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 2020/06/03 17:45:09 UTC

[GitHub] [airflow] samuelkhtu opened a new pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

samuelkhtu opened a new pull request #9124:
URL: https://github.com/apache/airflow/pull/9124


   ---
   Make sure to mark the boxes below before creating PR: [x]
   
   - [x] Description above provides context of the change
   - [x] Unit tests coverage for changes (not needed for documentation changes)
   - [x] Target Github ISSUE in description if exists
   - [x] Commits follow "[How to write a good git commit message](http://chris.beams.io/posts/git-commit/)"
   - [x] Relevant documentation is updated including usage instructions.
   - [x] I will engage committers as explained in [Contribution Workflow Example](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#contribution-workflow-example).
   
   ---
   In case of fundamental code change, Airflow Improvement Proposal ([AIP](https://cwiki.apache.org/confluence/display/AIRFLOW/Airflow+Improvements+Proposals)) is needed.
   In case of a new dependency, check compliance with the [ASF 3rd Party License Policy](https://www.apache.org/legal/resolved.html#category-x).
   In case of backwards incompatible changes please leave a note in [UPDATING.md](https://github.com/apache/airflow/blob/master/UPDATING.md).
   Read the [Pull Request Guidelines](https://github.com/apache/airflow/blob/master/CONTRIBUTING.rst#pull-request-guidelines) for more information.
   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on pull request #9124: Merging multiple sql operators

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


   This PR has failing tests


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] samuelkhtu commented on a change in pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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



##########
File path: airflow/operators/sql_branch_operator.py
##########
@@ -14,162 +14,15 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
-from distutils.util import strtobool
-from typing import Dict, Iterable, List, Mapping, Optional, Union
-
-from airflow.exceptions import AirflowException
-from airflow.hooks.base_hook import BaseHook
-from airflow.models import BaseOperator, SkipMixin
-from airflow.utils.decorators import apply_defaults
-
-ALLOWED_CONN_TYPE = {
-    "google_cloud_platform",
-    "jdbc",
-    "mssql",
-    "mysql",
-    "odbc",
-    "oracle",
-    "postgres",
-    "presto",
-    "sqlite",
-    "vertica",
-}
-
-
-class BranchSqlOperator(BaseOperator, SkipMixin):
-    """
-    Executes sql code in a specific database
-
-    :param sql: the sql code to be executed. (templated)
-    :type sql: Can receive a str representing a sql statement or reference to a template file.
-               Template reference are recognized by str ending in '.sql'.
-               Expected SQL query to return Boolean (True/False), integer (0 = False, Otherwise = 1)
-               or string (true/y/yes/1/on/false/n/no/0/off).
-    :param follow_task_ids_if_true: task id or task ids to follow if query return true
-    :type follow_task_ids_if_true: str or list
-    :param follow_task_ids_if_false: task id or task ids to follow if query return true
-    :type follow_task_ids_if_false: str or list
-    :param conn_id: reference to a specific database
-    :type conn_id: str
-    :param database: name of database which overwrite defined one in connection
-    :param parameters: (optional) the parameters to render the SQL query with.
-    :type parameters: mapping or iterable
-    """
-
-    template_fields = ("sql",)
-    template_ext = (".sql",)
-    ui_color = "#a22034"
-    ui_fgcolor = "#F7F7F7"
-
-    @apply_defaults
-    def __init__(
-        self,
-        sql: str,
-        follow_task_ids_if_true: List[str],
-        follow_task_ids_if_false: List[str],
-        conn_id: str = "default_conn_id",
-        database: Optional[str] = None,
-        parameters: Optional[Union[Mapping, Iterable]] = None,
-        *args,
-        **kwargs,
-    ) -> None:
-        super().__init__(*args, **kwargs)
-        self.conn_id = conn_id
-        self.sql = sql
-        self.parameters = parameters
-        self.follow_task_ids_if_true = follow_task_ids_if_true
-        self.follow_task_ids_if_false = follow_task_ids_if_false
-        self.database = database
-        self._hook = None
-
-    def _get_hook(self):
-        self.log.debug("Get connection for %s", self.conn_id)
-        conn = BaseHook.get_connection(self.conn_id)
-
-        if conn.conn_type not in ALLOWED_CONN_TYPE:
-            raise AirflowException(
-                "The connection type is not supported by BranchSqlOperator. "
-                + "Supported connection types: {}".format(list(ALLOWED_CONN_TYPE))
-            )
-
-        if not self._hook:
-            self._hook = conn.get_hook()
-            if self.database:
-                self._hook.schema = self.database
-
-        return self._hook
-
-    def execute(self, context: Dict):
-        # get supported hook
-        self._hook = self._get_hook()
-
-        if self._hook is None:
-            raise AirflowException(
-                "Failed to establish connection to '%s'" % self.conn_id
-            )
-
-        if self.sql is None:
-            raise AirflowException("Expected 'sql' parameter is missing.")
-
-        if self.follow_task_ids_if_true is None:
-            raise AirflowException(
-                "Expected 'follow_task_ids_if_true' paramter is missing."
-            )
-
-        if self.follow_task_ids_if_false is None:
-            raise AirflowException(
-                "Expected 'follow_task_ids_if_false' parameter is missing."
-            )
-
-        self.log.info(
-            "Executing: %s (with parameters %s) with connection: %s",
-            self.sql,
-            self.parameters,
-            self._hook,
-        )
-        record = self._hook.get_first(self.sql, self.parameters)
-        if not record:
-            raise AirflowException(
-                "No rows returned from sql query. Operator expected True or False return value."
-            )
-
-        if isinstance(record, list):
-            if isinstance(record[0], list):
-                query_result = record[0][0]
-            else:
-                query_result = record[0]
-        elif isinstance(record, tuple):
-            query_result = record[0]
-        else:
-            query_result = record
-
-        self.log.info("Query returns %s, type '%s'", query_result, type(query_result))
-
-        follow_branch = None
-        try:
-            if isinstance(query_result, bool):
-                if query_result:
-                    follow_branch = self.follow_task_ids_if_true
-            elif isinstance(query_result, str):
-                # return result is not Boolean, try to convert from String to Boolean
-                if bool(strtobool(query_result)):
-                    follow_branch = self.follow_task_ids_if_true
-            elif isinstance(query_result, int):
-                if bool(query_result):
-                    follow_branch = self.follow_task_ids_if_true
-            else:
-                raise AirflowException(
-                    "Unexpected query return result '%s' type '%s'"
-                    % (query_result, type(query_result))
-                )
-
-            if follow_branch is None:
-                follow_branch = self.follow_task_ids_if_false
-        except ValueError:
-            raise AirflowException(
-                "Unexpected query return result '%s' type '%s'"
-                % (query_result, type(query_result))
-            )
-
-        self.skip_all_except(context["ti"], follow_branch)
+"""This module is deprecated. Please use `airflow.operators.sql`."""
+import warnings
+
+# pylint: disable=unused-import
+# flake8: noqa F401
+from airflow.operators.sql import BranchSQLOperator as BranchSqlOperator

Review comment:
       Got it. Thanks. I updated the code.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] eladkal commented on pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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


   My thoughts (don't take any action for the moment lets wait for thoughts of others)
   1. I'm not sure if it requires deprecation warning for BranchSqlOperator. PR https://github.com/apache/airflow/pull/8942 wasn't released yet. So the question is will it be released with this PR or not. If both PRs are released together then there is no need to maintain `airflow/operators/sql_branch_operator.py`  with deprecation warning.
   2. PrestoCheckOperator(s) were deprecated in https://github.com/apache/airflow/pull/7884 and pointing to CheckOperator(s). By merging this PR they will point to another deprecated module. Maybe worth to fix the deprecation warning for the Presto Operators as well?


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] samuelkhtu commented on pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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


   When you have a chance, can you take a look? @mik-laj  @eladkal. Thanks!


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on pull request #9124: Merging multiple sql operators

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #9124:
URL: https://github.com/apache/airflow/pull/9124#issuecomment-645576111


   @eladkal I am not a release manager. I focus only on the development of Airflow 2.0


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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



##########
File path: tests/test_core_to_contrib.py
##########
@@ -1236,15 +1236,15 @@
         'airflow.operators.papermill_operator.PapermillOperator',
     ),
     (
-        'airflow.operators.check_operator.CheckOperator',
+        'airflow.operators.sql.SQLCheckOperator',

Review comment:
       You need to add new entry in this file. Now you have corrected the entry that changes 
   airflow.operators.presto_check_operator.PrestoCheckOperator => airflow.operators.sql.SQLCheckOperator
   , but we have no entry for 
   airflow.operators.check_operator.CheckOperator => airflow.operators.sql.SQLCheckOperator. 
   I have intentionally asked to add entries because it is the easiest way to check that you have done everything correctly.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] samuelkhtu commented on pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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


   Thanks @eladkal , @mik-laj  any thought? 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] samuelkhtu commented on a change in pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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



##########
File path: tests/test_core_to_contrib.py
##########
@@ -1236,15 +1236,15 @@
         'airflow.operators.papermill_operator.PapermillOperator',
     ),
     (
-        'airflow.operators.check_operator.CheckOperator',
+        'airflow.operators.sql.SQLCheckOperator',

Review comment:
       Sorry for misunderstand your instruction. I added new entries in the new commit.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on pull request #9124: Merging multiple sql operators

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #9124:
URL: https://github.com/apache/airflow/pull/9124#issuecomment-645548140


   Close https://github.com/apache/airflow/issues/9099


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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



##########
File path: airflow/operators/sql_branch_operator.py
##########
@@ -14,162 +14,15 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
-from distutils.util import strtobool
-from typing import Dict, Iterable, List, Mapping, Optional, Union
-
-from airflow.exceptions import AirflowException
-from airflow.hooks.base_hook import BaseHook
-from airflow.models import BaseOperator, SkipMixin
-from airflow.utils.decorators import apply_defaults
-
-ALLOWED_CONN_TYPE = {
-    "google_cloud_platform",
-    "jdbc",
-    "mssql",
-    "mysql",
-    "odbc",
-    "oracle",
-    "postgres",
-    "presto",
-    "sqlite",
-    "vertica",
-}
-
-
-class BranchSqlOperator(BaseOperator, SkipMixin):
-    """
-    Executes sql code in a specific database
-
-    :param sql: the sql code to be executed. (templated)
-    :type sql: Can receive a str representing a sql statement or reference to a template file.
-               Template reference are recognized by str ending in '.sql'.
-               Expected SQL query to return Boolean (True/False), integer (0 = False, Otherwise = 1)
-               or string (true/y/yes/1/on/false/n/no/0/off).
-    :param follow_task_ids_if_true: task id or task ids to follow if query return true
-    :type follow_task_ids_if_true: str or list
-    :param follow_task_ids_if_false: task id or task ids to follow if query return true
-    :type follow_task_ids_if_false: str or list
-    :param conn_id: reference to a specific database
-    :type conn_id: str
-    :param database: name of database which overwrite defined one in connection
-    :param parameters: (optional) the parameters to render the SQL query with.
-    :type parameters: mapping or iterable
-    """
-
-    template_fields = ("sql",)
-    template_ext = (".sql",)
-    ui_color = "#a22034"
-    ui_fgcolor = "#F7F7F7"
-
-    @apply_defaults
-    def __init__(
-        self,
-        sql: str,
-        follow_task_ids_if_true: List[str],
-        follow_task_ids_if_false: List[str],
-        conn_id: str = "default_conn_id",
-        database: Optional[str] = None,
-        parameters: Optional[Union[Mapping, Iterable]] = None,
-        *args,
-        **kwargs,
-    ) -> None:
-        super().__init__(*args, **kwargs)
-        self.conn_id = conn_id
-        self.sql = sql
-        self.parameters = parameters
-        self.follow_task_ids_if_true = follow_task_ids_if_true
-        self.follow_task_ids_if_false = follow_task_ids_if_false
-        self.database = database
-        self._hook = None
-
-    def _get_hook(self):
-        self.log.debug("Get connection for %s", self.conn_id)
-        conn = BaseHook.get_connection(self.conn_id)
-
-        if conn.conn_type not in ALLOWED_CONN_TYPE:
-            raise AirflowException(
-                "The connection type is not supported by BranchSqlOperator. "
-                + "Supported connection types: {}".format(list(ALLOWED_CONN_TYPE))
-            )
-
-        if not self._hook:
-            self._hook = conn.get_hook()
-            if self.database:
-                self._hook.schema = self.database
-
-        return self._hook
-
-    def execute(self, context: Dict):
-        # get supported hook
-        self._hook = self._get_hook()
-
-        if self._hook is None:
-            raise AirflowException(
-                "Failed to establish connection to '%s'" % self.conn_id
-            )
-
-        if self.sql is None:
-            raise AirflowException("Expected 'sql' parameter is missing.")
-
-        if self.follow_task_ids_if_true is None:
-            raise AirflowException(
-                "Expected 'follow_task_ids_if_true' paramter is missing."
-            )
-
-        if self.follow_task_ids_if_false is None:
-            raise AirflowException(
-                "Expected 'follow_task_ids_if_false' parameter is missing."
-            )
-
-        self.log.info(
-            "Executing: %s (with parameters %s) with connection: %s",
-            self.sql,
-            self.parameters,
-            self._hook,
-        )
-        record = self._hook.get_first(self.sql, self.parameters)
-        if not record:
-            raise AirflowException(
-                "No rows returned from sql query. Operator expected True or False return value."
-            )
-
-        if isinstance(record, list):
-            if isinstance(record[0], list):
-                query_result = record[0][0]
-            else:
-                query_result = record[0]
-        elif isinstance(record, tuple):
-            query_result = record[0]
-        else:
-            query_result = record
-
-        self.log.info("Query returns %s, type '%s'", query_result, type(query_result))
-
-        follow_branch = None
-        try:
-            if isinstance(query_result, bool):
-                if query_result:
-                    follow_branch = self.follow_task_ids_if_true
-            elif isinstance(query_result, str):
-                # return result is not Boolean, try to convert from String to Boolean
-                if bool(strtobool(query_result)):
-                    follow_branch = self.follow_task_ids_if_true
-            elif isinstance(query_result, int):
-                if bool(query_result):
-                    follow_branch = self.follow_task_ids_if_true
-            else:
-                raise AirflowException(
-                    "Unexpected query return result '%s' type '%s'"
-                    % (query_result, type(query_result))
-                )
-
-            if follow_branch is None:
-                follow_branch = self.follow_task_ids_if_false
-        except ValueError:
-            raise AirflowException(
-                "Unexpected query return result '%s' type '%s'"
-                % (query_result, type(query_result))
-            )
-
-        self.skip_all_except(context["ti"], follow_branch)
+"""This module is deprecated. Please use `airflow.operators.sql`."""
+import warnings
+
+# pylint: disable=unused-import
+# flake8: noqa F401
+from airflow.operators.sql import BranchSQLOperator as BranchSqlOperator

Review comment:
       This is not correct. When you change the name, you must create a dummy class to add a warning about the name change. Now you only have a message that the module has been moved, but you don't know the new class name. Look at other modules, please.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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



##########
File path: tests/test_core_to_contrib.py
##########
@@ -1236,15 +1236,15 @@
         'airflow.operators.papermill_operator.PapermillOperator',
     ),
     (
-        'airflow.operators.check_operator.CheckOperator',
+        'airflow.operators.sql.SQLCheckOperator',

Review comment:
       You need to add new entry in this file. Now you have corrected the entry that changes airflow.operators.presto_check_operator.PrestoCheckOperator => airflow.operators.sql.SQLCheckOperator, but we have no entry for airflow.operators.check_operator.CheckOperator => airflow.operators.sql.SQLCheckOperator. I have intentionally asked to add entries because it is the easiest way to check that you have done everything correctly.
   




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] samuelkhtu commented on pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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


   Yes @mik-laj    Thank you for the feedback. I will work on this ASAP. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj merged pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

Posted by GitBox <gi...@apache.org>.
mik-laj merged pull request #9124:
URL: https://github.com/apache/airflow/pull/9124


   


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] samuelkhtu commented on pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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


   Hi @mik-laj when you have a chance, can you take a look? Thanks. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #9124:
URL: https://github.com/apache/airflow/pull/9124#issuecomment-641106425


   Can you add these cases to the list of changed classes? The file name suggests something different, but that's a different problem. Here we have all the name changes that we then use to generate the documentation.
   https://github.com/apache/airflow/blob/master/tests/test_core_to_contrib.py#L427


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] kaxil commented on pull request #9124: Merging multiple sql operators

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


   ```
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_class_deprecated_100_airflow_operators_check_operator_CheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_class_deprecated_101_airflow_operators_check_operator_IntervalCheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_class_deprecated_102_airflow_operators_check_operator_ValueCheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_class_deprecated_103_airflow_operators_check_operator_ThresholdCheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_class_deprecated_104_airflow_operators_sql_branch_operator_BranchSqlOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_subclass_274_airflow_operators_check_operator_CheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_subclass_275_airflow_operators_check_operator_IntervalCheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_subclass_276_airflow_operators_check_operator_ValueCheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_subclass_277_airflow_operators_check_operator_ThresholdCheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_is_subclass_278_airflow_operators_sql_branch_operator_BranchSqlOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_warning_on_import_274_airflow_operators_check_operator_CheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_warning_on_import_275_airflow_operators_check_operator_IntervalCheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_warning_on_import_276_airflow_operators_check_operator_ValueCheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_warning_on_import_277_airflow_operators_check_operator_ThresholdCheckOperator
   FAILED tests/test_core_to_contrib.py::TestMovingCoreToContrib::test_warning_on_import_278_airflow_operators_sql_branch_operator_BranchSqlOperator
   
   ```


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on a change in pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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



##########
File path: airflow/operators/sql_branch_operator.py
##########
@@ -14,162 +14,15 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-
-from distutils.util import strtobool
-from typing import Dict, Iterable, List, Mapping, Optional, Union
-
-from airflow.exceptions import AirflowException
-from airflow.hooks.base_hook import BaseHook
-from airflow.models import BaseOperator, SkipMixin
-from airflow.utils.decorators import apply_defaults
-
-ALLOWED_CONN_TYPE = {
-    "google_cloud_platform",
-    "jdbc",
-    "mssql",
-    "mysql",
-    "odbc",
-    "oracle",
-    "postgres",
-    "presto",
-    "sqlite",
-    "vertica",
-}
-
-
-class BranchSqlOperator(BaseOperator, SkipMixin):
-    """
-    Executes sql code in a specific database
-
-    :param sql: the sql code to be executed. (templated)
-    :type sql: Can receive a str representing a sql statement or reference to a template file.
-               Template reference are recognized by str ending in '.sql'.
-               Expected SQL query to return Boolean (True/False), integer (0 = False, Otherwise = 1)
-               or string (true/y/yes/1/on/false/n/no/0/off).
-    :param follow_task_ids_if_true: task id or task ids to follow if query return true
-    :type follow_task_ids_if_true: str or list
-    :param follow_task_ids_if_false: task id or task ids to follow if query return true
-    :type follow_task_ids_if_false: str or list
-    :param conn_id: reference to a specific database
-    :type conn_id: str
-    :param database: name of database which overwrite defined one in connection
-    :param parameters: (optional) the parameters to render the SQL query with.
-    :type parameters: mapping or iterable
-    """
-
-    template_fields = ("sql",)
-    template_ext = (".sql",)
-    ui_color = "#a22034"
-    ui_fgcolor = "#F7F7F7"
-
-    @apply_defaults
-    def __init__(
-        self,
-        sql: str,
-        follow_task_ids_if_true: List[str],
-        follow_task_ids_if_false: List[str],
-        conn_id: str = "default_conn_id",
-        database: Optional[str] = None,
-        parameters: Optional[Union[Mapping, Iterable]] = None,
-        *args,
-        **kwargs,
-    ) -> None:
-        super().__init__(*args, **kwargs)
-        self.conn_id = conn_id
-        self.sql = sql
-        self.parameters = parameters
-        self.follow_task_ids_if_true = follow_task_ids_if_true
-        self.follow_task_ids_if_false = follow_task_ids_if_false
-        self.database = database
-        self._hook = None
-
-    def _get_hook(self):
-        self.log.debug("Get connection for %s", self.conn_id)
-        conn = BaseHook.get_connection(self.conn_id)
-
-        if conn.conn_type not in ALLOWED_CONN_TYPE:
-            raise AirflowException(
-                "The connection type is not supported by BranchSqlOperator. "
-                + "Supported connection types: {}".format(list(ALLOWED_CONN_TYPE))
-            )
-
-        if not self._hook:
-            self._hook = conn.get_hook()
-            if self.database:
-                self._hook.schema = self.database
-
-        return self._hook
-
-    def execute(self, context: Dict):
-        # get supported hook
-        self._hook = self._get_hook()
-
-        if self._hook is None:
-            raise AirflowException(
-                "Failed to establish connection to '%s'" % self.conn_id
-            )
-
-        if self.sql is None:
-            raise AirflowException("Expected 'sql' parameter is missing.")
-
-        if self.follow_task_ids_if_true is None:
-            raise AirflowException(
-                "Expected 'follow_task_ids_if_true' paramter is missing."
-            )
-
-        if self.follow_task_ids_if_false is None:
-            raise AirflowException(
-                "Expected 'follow_task_ids_if_false' parameter is missing."
-            )
-
-        self.log.info(
-            "Executing: %s (with parameters %s) with connection: %s",
-            self.sql,
-            self.parameters,
-            self._hook,
-        )
-        record = self._hook.get_first(self.sql, self.parameters)
-        if not record:
-            raise AirflowException(
-                "No rows returned from sql query. Operator expected True or False return value."
-            )
-
-        if isinstance(record, list):
-            if isinstance(record[0], list):
-                query_result = record[0][0]
-            else:
-                query_result = record[0]
-        elif isinstance(record, tuple):
-            query_result = record[0]
-        else:
-            query_result = record
-
-        self.log.info("Query returns %s, type '%s'", query_result, type(query_result))
-
-        follow_branch = None
-        try:
-            if isinstance(query_result, bool):
-                if query_result:
-                    follow_branch = self.follow_task_ids_if_true
-            elif isinstance(query_result, str):
-                # return result is not Boolean, try to convert from String to Boolean
-                if bool(strtobool(query_result)):
-                    follow_branch = self.follow_task_ids_if_true
-            elif isinstance(query_result, int):
-                if bool(query_result):
-                    follow_branch = self.follow_task_ids_if_true
-            else:
-                raise AirflowException(
-                    "Unexpected query return result '%s' type '%s'"
-                    % (query_result, type(query_result))
-                )
-
-            if follow_branch is None:
-                follow_branch = self.follow_task_ids_if_false
-        except ValueError:
-            raise AirflowException(
-                "Unexpected query return result '%s' type '%s'"
-                % (query_result, type(query_result))
-            )
-
-        self.skip_all_except(context["ti"], follow_branch)
+"""This module is deprecated. Please use `airflow.operators.sql`."""
+import warnings
+
+# pylint: disable=unused-import
+# flake8: noqa F401
+from airflow.operators.sql import BranchSQLOperator as BranchSqlOperator

Review comment:
       This is not correct. When you change the name, you must create a dummy class to add a warning about the name change. Now you only have a message that the module has been moved, but user don't know the new class name. Look at other modules, please.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] eladkal commented on pull request #9124: Merging multiple sql operators

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


   great work @samuelkhtu 
   @mik-laj  can you cherry pick both PRs https://github.com/apache/airflow/pull/8942 and current to next release? It's core operator so it won't be released in future providers package releases.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] mik-laj commented on pull request #9124: Merging multiple sql operators

Posted by GitBox <gi...@apache.org>.
mik-laj commented on pull request #9124:
URL: https://github.com/apache/airflow/pull/9124#issuecomment-645648971


   @kaxil I working on it now.


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] potiuk commented on pull request #9124: Merging multiple sql operators

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


   I added it to milestone 1.10.11 it seems easy to cherry pick. Let's see if we manage to do it. 


----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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



[GitHub] [airflow] samuelkhtu commented on a change in pull request #9124: [AIRFLOW-9099] Merging multiple sql operators

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



##########
File path: tests/test_core_to_contrib.py
##########
@@ -1236,15 +1236,15 @@
         'airflow.operators.papermill_operator.PapermillOperator',
     ),
     (
-        'airflow.operators.check_operator.CheckOperator',
+        'airflow.operators.sql.SQLCheckOperator',

Review comment:
       I am sorry I misunderstood your comment. I added the new entries in the new commit.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

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