vincbeck commented on code in PR #23881:
URL: https://github.com/apache/airflow/pull/23881#discussion_r886134984


##########
airflow/operators/s3_to_sql.py:
##########
@@ -0,0 +1,117 @@
+#

Review Comment:
   In my opinion this file should be in the Amazon provider package. It should 
be in `/airflow/airflow/providers/amazon/aws/transfers`



##########
airflow/operators/s3_to_sql.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.
+
+from typing import List, Optional, Sequence, Union
+
+import numpy
+import pandas
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base import BaseHook
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.utils.context import Context
+
+
+class S3ToSqlOperator(BaseOperator):
+    """
+    Moves data from s3 to sql.
+
+    :param source_path: path to s3 file
+    :param destination_table: target table on sql
+    :param file_format: input file format. CSV, JSON or Parquet
+    :param file_options: file reader options
+    :param source_conn_id: source connection
+    :param destination_conn_id: destination connection
+    :param preoperator: sql statement or list of statements to be
+        executed prior to loading the data. (templated)
+    :param insert_args: extra params for `insert_rows` method.
+    """
+
+    template_fields: Sequence[str] = ('source_key', 'destination_table', 
'file_format', 'preoperator')
+    template_ext: Sequence[str] = (
+        '.sql',
+        '.hql',
+    )
+    template_fields_renderers = {"preoperator": "sql"}
+    ui_color = '#b0f07c'
+
+    def __init__(
+        self,
+        *,
+        source_path: str,
+        destination_table: str,
+        file_format: str,
+        file_options: Optional[dict] = None,
+        source_conn_id: str = 'aws_default',
+        destination_conn_id: str = 'sql_default',
+        preoperator: Optional[Union[str, List[str]]] = None,
+        insert_args: Optional[dict] = None,
+        **kwargs,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.source_key = source_path
+        self.destination_table = destination_table
+        self.file_format = file_format
+        self.file_options = file_options or {}
+        self.source_conn_id = source_conn_id
+        self.destination_conn_id = destination_conn_id
+        self.preoperator = preoperator
+        self.insert_args = insert_args or {}
+
+    @staticmethod
+    def _fix_int_dtypes(df: pandas.DataFrame) -> None:

Review Comment:
   Please use the function `SqlToS3Operator._fix_int_dtypes` instead of copy 
pasting it. If doing so, please also rename the method to `fix_int_dtypes` 



##########
airflow/operators/s3_to_sql.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.
+
+from typing import List, Optional, Sequence, Union
+
+import numpy
+import pandas
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base import BaseHook
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.utils.context import Context
+
+
+class S3ToSqlOperator(BaseOperator):

Review Comment:
   There is no unit tests for this operator



##########
airflow/operators/s3_to_sql.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.
+
+from typing import List, Optional, Sequence, Union
+
+import numpy
+import pandas
+
+from airflow.exceptions import AirflowException
+from airflow.hooks.base import BaseHook
+from airflow.models import BaseOperator
+from airflow.providers.amazon.aws.hooks.s3 import S3Hook
+from airflow.utils.context import Context
+
+
+class S3ToSqlOperator(BaseOperator):
+    """
+    Moves data from s3 to sql.
+
+    :param source_path: path to s3 file

Review Comment:
   The operator `SqlToS3Operator` already exists, this one is just the 
opposite. Should we try to use same parameter name?
   ```suggestion
       :param s3_key: path to s3 file
   ```



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to