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 2022/06/13 13:58:31 UTC

[GitHub] [airflow] phanikumv opened a new pull request, #24415: Add TrinoOperator

phanikumv opened a new pull request, #24415:
URL: https://github.com/apache/airflow/pull/24415

   This PR adds a TrinoOperator to enable submitting SQL to the Trino Query Engine.
   
   <!--
   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/main/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 a newsfragement file, named `{pr_number}.significant.rst`, in [newsfragments](https://github.com/apache/airflow/tree/main/newsfragments).
   


-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r901480914


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+        to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+        before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+

Review Comment:
   Addressed in https://github.com/apache/airflow/pull/24559



-- 
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] kaxil commented on pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
kaxil commented on PR #24415:
URL: https://github.com/apache/airflow/pull/24415#issuecomment-1155078506

   @phanikumv Doc test is failing - https://github.com/apache/airflow/runs/6875362076?check_suite_focus=true


-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r899130990


##########
tests/system/providers/trino/example_trino_query.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ 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.
+*/
+
+SELECT * FROM {{ params.SCHEMA }}.{{ params.TABLE }}

Review Comment:
   Removed the redundant 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: commits-unsubscribe@airflow.apache.org

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


[GitHub] [airflow] eladkal commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r898742602


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   We are defenetly not going to set dependency on snowflake for TrinoOperator. I'll check this later today as I'm not near my laptop.



-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r899133257


##########
docs/apache-airflow-providers-trino/operators/trino.rst:
##########
@@ -0,0 +1,56 @@
+ .. 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.
+
+.. _howto/operator:TrinoOperator:
+
+TrinoOperator
+=============
+
+Use the :class:`TrinoOperator <airflow.providers.trino.operators.trino>` to execute
+SQL commands in a `Trino <https://trino.io/>`__ query engine.
+
+
+Using the Operator
+^^^^^^^^^^^^^^^^^^
+
+Use the ``trino_conn_id`` argument to connect to your Trino instance where
+the connection metadata is structured as follows:
+
+.. list-table:: Trino Airflow Connection Metadata
+   :widths: 25 25
+   :header-rows: 1
+
+   * - Parameter
+     - Input
+   * - Host: string
+     - Trino server host name (or) ip address
+   * - Login: string
+     - Trino user name
+   * - Port: string
+     - Set Port to connect to the Trino server, for example on AWS EMR, Trino uses port 8889
+
+An example usage of the TrinoOperator is as follows:
+
+.. exampleinclude:: /../../tests/system/providers/trino/example_trino.py
+    :language: python
+    :start-after: [START howto_operator_trino]
+    :end-before: [END howto_operator_trino]
+
+.. note::
+
+  This Operator can be used to run any syntactically correct Trino query(one at a time), but we cannot run
+  multiple queries in the same task.

Review Comment:
   we can now run multiple queries with the commit [6cb8b37](https://github.com/apache/airflow/pull/24415/commits/6cb8b37bad883f785686983d465608904e014b28) , hence I have modified the documentation accordingly



-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r898922540


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   Thank you, will check and get back in case of any questions



-- 
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] eladkal commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r895756818


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   This will work only with a single query won't it?
   I believe This PR is depended on https://github.com/apache/airflow/issues/23112
   
   I think it's worth waiting for https://github.com/apache/airflow/pull/23971 to be completed?



-- 
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] eladkal commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r898916770


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   Check https://github.com/apache/airflow/pull/22391 



-- 
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] kaxil commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
kaxil commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r896717023


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   I don't think it is worth waiting for https://github.com/apache/airflow/pull/23971 as that change will require Airflow 2.4 (which is at least 1-2 months down the line) but this doesn't and can be released sooner 



-- 
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 #24415: Add TrinoOperator

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


-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r896737861


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   I will try to add support to run multiple queries to this PR , and it will be specific to TrinoOperator



-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r898713143


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   The `split_statements` method referenced in #23112 is from the snowflake library, so there are two options to use it in TrinoHook, 
   
   - create the same method again in TrinoHoook 
   - we will need to do an import using `from snowflake.connector.util_text import split_statements` in the TrinoHook which will create a dependency on the snowflake library and will not work if the user doesnt have snowflake library installed 
   
   According to me , option 1 is OK until #23971 is merged. 
   
   Please let me know your thoughts @eladkal 



##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   The `split_statements` method referenced in #23112 is from the snowflake library, so there are two options to use it in TrinoHook, 
   
   - create the same method again in TrinoHook 
   - we will need to do an import using `from snowflake.connector.util_text import split_statements` in the TrinoHook which will create a dependency on the snowflake library and will not work if the user doesnt have snowflake library installed 
   
   According to me , option 1 is OK until #23971 is merged. 
   
   Please let me know your thoughts @eladkal 



-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r899134611


##########
docs/apache-airflow-providers-trino/operators/trino.rst:
##########
@@ -0,0 +1,56 @@
+ .. 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.
+
+.. _howto/operator:TrinoOperator:
+
+TrinoOperator
+=============
+
+Use the :class:`TrinoOperator <airflow.providers.trino.operators.trino>` to execute
+SQL commands in a `Trino <https://trino.io/>`__ query engine.
+
+
+Using the Operator
+^^^^^^^^^^^^^^^^^^
+
+Use the ``trino_conn_id`` argument to connect to your Trino instance where
+the connection metadata is structured as follows:
+
+.. list-table:: Trino Airflow Connection Metadata
+   :widths: 25 25
+   :header-rows: 1
+
+   * - Parameter
+     - Input
+   * - Host: string
+     - Trino server host name (or) ip address
+   * - Login: string
+     - Trino user name
+   * - Port: string
+     - Set Port to connect to the Trino server, for example on AWS EMR, Trino uses port 8889

Review Comment:
   Since the connection doc is already present, I have removed this section from the [docs/apache-airflow-providers-trino/operators/trino.rst](https://github.com/apache/airflow/pull/24415/files/102a346e28d91f1c80aafc22d583a47ba4cdb831#diff-c26e56d548a65aecc1719aa9816888cf1507640d7e9947f4a9cd236d3fb2cbb4)



-- 
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] eladkal commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r896739291


##########
docs/apache-airflow-providers-trino/operators/trino.rst:
##########
@@ -0,0 +1,56 @@
+ .. 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.
+
+.. _howto/operator:TrinoOperator:
+
+TrinoOperator
+=============
+
+Use the :class:`TrinoOperator <airflow.providers.trino.operators.trino>` to execute
+SQL commands in a `Trino <https://trino.io/>`__ query engine.
+
+
+Using the Operator
+^^^^^^^^^^^^^^^^^^
+
+Use the ``trino_conn_id`` argument to connect to your Trino instance where
+the connection metadata is structured as follows:
+
+.. list-table:: Trino Airflow Connection Metadata
+   :widths: 25 25
+   :header-rows: 1
+
+   * - Parameter
+     - Input
+   * - Host: string
+     - Trino server host name (or) ip address
+   * - Login: string
+     - Trino user name
+   * - Port: string
+     - Set Port to connect to the Trino server, for example on AWS EMR, Trino uses port 8889
+
+An example usage of the TrinoOperator is as follows:
+
+.. exampleinclude:: /../../tests/system/providers/trino/example_trino.py
+    :language: python
+    :start-after: [START howto_operator_trino]
+    :end-before: [END howto_operator_trino]
+
+.. note::
+
+  This Operator can be used to run any syntactically correct Trino query(one at a time), but we cannot run
+  multiple queries in the same task.

Review Comment:
   If we can lets point to Trino documentation that states this. This is a limitation coming from Trino not from Airflow so it's important that users will understand this.



##########
docs/apache-airflow-providers-trino/operators/trino.rst:
##########
@@ -0,0 +1,56 @@
+ .. 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.
+
+.. _howto/operator:TrinoOperator:
+
+TrinoOperator
+=============
+
+Use the :class:`TrinoOperator <airflow.providers.trino.operators.trino>` to execute
+SQL commands in a `Trino <https://trino.io/>`__ query engine.
+
+
+Using the Operator
+^^^^^^^^^^^^^^^^^^
+
+Use the ``trino_conn_id`` argument to connect to your Trino instance where
+the connection metadata is structured as follows:
+
+.. list-table:: Trino Airflow Connection Metadata
+   :widths: 25 25
+   :header-rows: 1
+
+   * - Parameter
+     - Input
+   * - Host: string
+     - Trino server host name (or) ip address
+   * - Login: string
+     - Trino user name
+   * - Port: string
+     - Set Port to connect to the Trino server, for example on AWS EMR, Trino uses port 8889

Review Comment:
   This is information for connection doc not the operator doc.
   Example: https://github.com/apache/airflow/blob/5c0e98cc770b4f055dbd1c0b60ccbd69f3166da7/docs/apache-airflow-providers-salesforce/connections/salesforce.rst



##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+        to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+        before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+

Review Comment:
   We need `on_kill()` implementation to kill Trino query if Airflow task is killed.



##########
tests/system/providers/trino/example_trino_query.sql:
##########
@@ -0,0 +1,20 @@
+/*
+ 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.
+*/
+
+SELECT * FROM {{ params.SCHEMA }}.{{ params.TABLE }}

Review Comment:
   I think this file is redundant. You can just place this query in the sql parameter of the operator being used.
   There is no reason to test/show example of how to use templating from `.sql` file in TrinoOperator example.
   It's also hard for people who will read the example from the docs as they won't see what .sql file contains.



##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+        to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+        before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)

Review Comment:
   I think in other operators we also template `parameters`?



##########
tests/system/providers/trino/example_trino.py:
##########
@@ -0,0 +1,89 @@
+#
+# 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.
+"""
+Example DAG using TrinoOperator.
+"""
+
+from datetime import datetime
+
+from airflow import models
+from airflow.providers.trino.operators.trino import TrinoOperator
+
+SCHEMA = "hive.cities"
+TABLE = "city"

Review Comment:
   Some of the queries use hard coded schema and table though you defined it globally.



-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r897829824


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   sure I will check https://github.com/apache/airflow/issues/23112 and implement the change



-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r899130433


##########
tests/system/providers/trino/example_trino.py:
##########
@@ -0,0 +1,89 @@
+#
+# 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.
+"""
+Example DAG using TrinoOperator.
+"""
+
+from datetime import datetime
+
+from airflow import models
+from airflow.providers.trino.operators.trino import TrinoOperator
+
+SCHEMA = "hive.cities"
+TABLE = "city"

Review Comment:
   Changed the DAG accordingly



-- 
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] eladkal commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r896755290


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   Check https://github.com/apache/airflow/issues/23112 i refrenced how Snowflake and Redshift implement it so it's very similar if not identical 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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r899031687


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   Sample logs for single and multi query execution
   
   `[2022-06-16, 12:27:38 UTC] {base.py:68} INFO - Using connection ID 'trino_default' for task execution.
   [2022-06-16, 12:27:38 UTC] {base.py:68} INFO - Using connection ID 'trino_default' for task execution.
   [2022-06-16, 12:27:38 UTC] {dbapi.py:208} INFO - Running statement: SELECT * FROM hive.cities.city100, parameters: None
   [2022-06-16, 12:27:40 UTC] {trino.py:312} INFO - Query Execution Result: [[2, 'San Jose']]`
   
   `[2022-06-16, 12:27:27 UTC] {base.py:68} INFO - Using connection ID 'trino_default' for task execution.
   [2022-06-16, 12:27:27 UTC] {base.py:68} INFO - Using connection ID 'trino_default' for task execution.
   [2022-06-16, 12:27:27 UTC] {dbapi.py:208} INFO - Running statement: CREATE TABLE hive.cities.city100(cityid bigint,cityname varchar), parameters: None
   [2022-06-16, 12:27:29 UTC] {dbapi.py:208} INFO - Running statement: INSERT INTO hive.cities.city100 VALUES (2, 'San Jose'), parameters: None
   [2022-06-16, 12:27:31 UTC] {dbapi.py:208} INFO - Running statement: CREATE TABLE hive.cities.city101(cityid bigint,cityname varchar), parameters: None
   [2022-06-16, 12:27:32 UTC] {dbapi.py:208} INFO - Running statement: INSERT INTO hive.cities.city101 VALUES (3, 'San Diego'), parameters: None
   [2022-06-16, 12:27:35 UTC] {trino.py:312} INFO - Query Execution Result: [[True], [1], [True], [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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r900020686


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+        to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+        before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)

Review Comment:
   thats when the parameters is a `dict`, here it is `tuple`



-- 
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 #24415: Add TrinoOperator

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

   The PR most likely needs to run full matrix of tests because it modifies parts of the core of Airflow. However, committers might decide to merge it quickly and take the risk. If they don't merge it quickly - please rebase it to the latest main at your convenience, 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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r898713143


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   The `split_statements` method referenced in #23112 is from the snowflake library, so there are two options to use it in TrinoHook, 
   
   - create the same method again in TrionHoook 
   - we will need to do an import using `from snowflake.connector.util_text import split_statements` in the TrinoHook which will create a dependency on the snowflake library and will not work if the user doesnt have snowflake library installed 
   
   According to me , option 1 is OK until #23971 is merged. 
   
   Please let me know your thoughts @eladkal 



-- 
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] eladkal commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r898742602


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   We are defenetly not going to set dependency on snowflake for TrinoOperwtor. I'll check this later today as I'm not near my laptop.



-- 
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] eladkal commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r898916770


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   @phanikumv  Check https://github.com/apache/airflow/pull/22391 



-- 
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] phanikumv commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
phanikumv commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r895761906


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   Exactly @eladkal , for now it will run with single query only. 



-- 
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] eladkal commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r896733610


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   It shouldn't...
   The solution should be compatible with Airflow >= 2.2 (min version for providers)
   so it can have a code for Airflow 2.4 and backport for lower versions which we will eventually remove.
   
   As a heavy Trino user the use case of running single query is rare.. very rare so I prefer to get the ability to run multiple queries. if we decide not to wait for https://github.com/apache/airflow/pull/23971 can we try to add support for multiple queries here? (It should be simple)
   I'm OK either way - just making a suggestion :)



-- 
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] eladkal commented on a diff in pull request #24415: Add TrinoOperator

Posted by GitBox <gi...@apache.org>.
eladkal commented on code in PR #24415:
URL: https://github.com/apache/airflow/pull/24415#discussion_r896733610


##########
airflow/providers/trino/operators/trino.py:
##########
@@ -0,0 +1,81 @@
+#
+# 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.
+"""This module contains the Trino operator."""
+
+from typing import TYPE_CHECKING, Any, Callable, Optional, Sequence
+
+from airflow.models import BaseOperator
+from airflow.providers.trino.hooks.trino import TrinoHook
+
+if TYPE_CHECKING:
+    from airflow.utils.context import Context
+
+
+class TrinoOperator(BaseOperator):
+    """
+    Executes sql code using a specific Trino query Engine.
+
+    .. seealso::
+        For more information on how to use this operator, take a look at the guide:
+        :ref:`howto/operator:TrinoOperator`
+
+    :param sql: the SQL code to be executed as a single string, or a reference
+    to a template file.Template references are recognized by str ending in '.sql'
+    :param trino_conn_id: id of the connection config for the target Trino
+        environment
+    :param autocommit: What to set the connection's autocommit setting to
+            before executing the query
+    :param handler: The result handler which is called with the result of each statement.
+    :param parameters: (optional) the parameters to render the SQL query with.
+    """
+
+    template_fields: Sequence[str] = ('sql',)
+    template_fields_renderers = {'sql': 'sql'}
+    template_ext: Sequence[str] = ('.sql',)
+    ui_color = '#ededed'
+
+    def __init__(
+        self,
+        *,
+        sql: str,
+        trino_conn_id: str = "trino_default",
+        autocommit: bool = False,
+        parameters: Optional[tuple] = None,
+        handler: Optional[Callable] = None,
+        **kwargs: Any,
+    ) -> None:
+        super().__init__(**kwargs)
+        self.sql = sql
+        self.trino_conn_id = trino_conn_id
+        self.hook: Optional[TrinoHook] = None
+        self.autocommit = autocommit
+        self.parameters = parameters
+        self.handler = handler
+
+    def get_hook(self) -> TrinoHook:
+        """Get Trino hook"""
+        return TrinoHook(
+            trino_conn_id=self.trino_conn_id,
+        )
+
+    def execute(self, context: 'Context') -> None:
+        """Execute Trino SQL"""
+        self.hook = self.get_hook()
+        self.hook.run(
+            sql=self.sql, autocommit=self.autocommit, parameters=self.parameters, handler=self.handler

Review Comment:
   It shouldn't...
   The solution should be compatible with Airflow >= 2.2 (min version for providers)
   so it can have a code for Airflow 2.4 and backport for lower versions which we will eventually remove.
   
   As a heavy Trino user the use case of running single query is rare.. very rare so I prefer to get the ability to run multiple queries so if we decide not to wait for https://github.com/apache/airflow/pull/23971 can we try to add support for multiple queries here? (It should be simple)
   I'm OK either way - just making a suggestion :)



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