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/11/28 07:14:08 UTC

[GitHub] [airflow] xinbinhuang opened a new pull request #12677: Refactor SQL/BigQuery Check operators

xinbinhuang opened a new pull request #12677:
URL: https://github.com/apache/airflow/pull/12677


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


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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -18,27 +18,59 @@
 from distutils.util import strtobool
 from typing import Any, Dict, Iterable, List, Mapping, Optional, SupportsAbs, Union
 
+from cached_property import cached_property
+
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
+from airflow.hooks.dbapi import DbApiHook
 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",
-    "snowflake",
-    "sqlite",
-    "vertica",
-}
-
-
-class SQLCheckOperator(BaseOperator):
+
+class BaseSQLOperator(BaseOperator):
+    """
+    This is a base class for generic SQL Operator to get a DB Hook
+
+    The provided method is .get_db_hook(). The default behavior will try to
+    retrieve the DB hook based on connection type.
+    You can custom the behavior by overriding the .get_db_hook() method.
+    """
+
+    @apply_defaults
+    def __init__(self, *, conn_id: Optional[str] = None, database: Optional[str] = None, **kwargs):
+        super().__init__(**kwargs)
+        self.conn_id = conn_id
+        self.database = database
+
+    @cached_property
+    def _hook(self):
+        """Get DB Hook based on connection type"""
+        self.log.debug("Get connection for %s", self.conn_id)
+        conn = BaseHook.get_connection(self.conn_id)
+
+        hook = conn.get_hook()
+        if not isinstance(hook, DbApiHook):
+            raise AirflowException(
+                f'The connection type is not supported by {self.__class__.__name__}. '
+                f'The associated hook should be a subclass of `DbApiHook`. Got {hook.__class__.__name__}'
+            )
+
+        if self.database:
+            hook.schema = self.database

Review comment:
       This is slightly confusing, database is not the same thing as schema




----------------------------------------------------------------
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] xinbinhuang edited a comment on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

Posted by GitBox <gi...@apache.org>.
xinbinhuang edited a comment on pull request #12677:
URL: https://github.com/apache/airflow/pull/12677#issuecomment-762371814


   > @xinbinhuang nice work! Just to be sure - there is no breaking change, right? @potiuk would you mind also taking a look?
   
   Yes, there is one place that may be considered fixing rather than breaking. For `airflow/providers/google/cloud/hooks/bigquery.py`, I changed the hook's `conn_type` and `hook_name`  from `google cloud platform`/`Google Cloud` to `bigquery`/`BigQuery`. This fixes the issue where `conn.get_hook` will never return the bigquery hook because `GoogleBaseHook` "hides" the `BigQueryHook` as they have the same `conn_type`.


----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   @potiuk I increased the `apache.druid` provider version to `1.0.2` because of deprecating a backward-compatible operator. It seems that I can also generate the documentation and releases with `./breeze prepare-provider-packages` and `./breeze prepare-provider-documentation` but I am not sure if I should do these. Can you help take a look and advise on anything else that you would want me to add?


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery Check operators

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


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


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

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



[GitHub] [airflow] xinbinhuang edited a comment on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

Posted by GitBox <gi...@apache.org>.
xinbinhuang edited a comment on pull request #12677:
URL: https://github.com/apache/airflow/pull/12677#issuecomment-781795819


   @kaxil @turbaszek @potiuk  @mik-laj  Can we merge this as it's already approved? It's pretty hard to get CI happy because it often fails for reasons not related to this PR.


----------------------------------------------------------------
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 edited a comment on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #12677:
URL: https://github.com/apache/airflow/pull/12677#issuecomment-764836879


   Hey @xinbinhuang . I am working on releasing providers and the "maintenance" process of them  - the PR and discussion is in #13767 I hope to get it merged tomorrow/Monday and release a new wave of providers early next week. I think I'd prefer to add this one afterwards - I would love to take a close look at the provider's backwards - compatibility then and we could work together to test the new provider mechanism (and updates to SEMVER versions of the providers that it brings) with you. 
   
   Would that be OK ?


----------------------------------------------------------------
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] xinbinhuang edited a comment on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

Posted by GitBox <gi...@apache.org>.
xinbinhuang edited a comment on pull request #12677:
URL: https://github.com/apache/airflow/pull/12677#issuecomment-781795819


   @kaxil @turbaszek @potiuk  Can we merge this as it's already approved? It's pretty hard to get CI happy because it often fails for reasons not related to this PR.


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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






----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       I left a comment about this in the thread previously, but I think it got buried over time. 
   
   This fixes the issue where conn.get_hook will never return the bigquery hook because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.
   
   Here I copy my original comment:
   
   > Yes, there is one place that may be considered fixing rather than breaking. For airflow/providers/google/cloud/hooks/bigquery.py, I changed the hook's conn_type and hook_name from google cloud platform/Google Cloud to bigquery/BigQuery. This fixes the issue where conn.get_hook will never return the bigquery hook because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.




----------------------------------------------------------------
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] xinbinhuang edited a comment on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

Posted by GitBox <gi...@apache.org>.
xinbinhuang edited a comment on pull request #12677:
URL: https://github.com/apache/airflow/pull/12677#issuecomment-781795819


   @kaxil @turbaszek @potiuk  @mik-laj  Can we merge this as it's already approved? It's pretty hard to get CI happy because it often fails for reasons not related to this PR.
   
   I can only get it green by luck by retriggering the job


----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       Does it needs any action from a user side is why I am thinking about it ? -- if yes, then we might need a note somewhere in Up:
   
   https://github.com/apache/airflow/blob/master/airflow/providers/google/cloud/ADDITIONAL_INFO.md




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

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



[GitHub] [airflow] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   @potiuk @kaxil  CI greenish except the MySQL transient failure...


----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/providers/apache/druid/operators/druid_check.py
##########
@@ -15,74 +15,23 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from typing import Any, Dict, Optional
+import warnings
 
-from airflow.exceptions import AirflowException
-from airflow.operators.check_operator import CheckOperator
-from airflow.providers.apache.druid.hooks.druid import DruidDbApiHook
-from airflow.utils.decorators import apply_defaults
+from airflow.operators.sql import SQLCheckOperator
 
 
-class DruidCheckOperator(CheckOperator):
+class DruidCheckOperator(SQLCheckOperator):
     """
-    Performs checks against Druid. The ``DruidCheckOperator`` expects
-    a sql query that will return a single row. Each value on that
-    first row is evaluated using python ``bool`` casting. If any of the
-    values return ``False`` the check is failed and errors out.
-
-    Note that Python bool casting evals the following as ``False``:
-
-    * ``False``
-    * ``0``
-    * Empty string (``""``)
-    * Empty list (``[]``)
-    * Empty dictionary or set (``{}``)
-
-    Given a query like ``SELECT COUNT(*) FROM foo``, it will fail only if
-    the count ``== 0``. You can craft much more complex query that could,
-    for instance, check that the table has the same number of rows as
-    the source table upstream, or that the count of today's partition is
-    greater than yesterday's partition, or that a set of metrics are less
-    than 3 standard deviation for the 7 day average.
-    This operator can be used as a data quality check in your pipeline, and
-    depending on where you put it in your DAG, you have the choice to
-    stop the critical path, preventing from
-    publishing dubious data, or on the side and receive email alerts
-    without stopping the progress of the DAG.
-
-    :param sql: the sql to be executed
-    :type sql: str
-    :param druid_broker_conn_id: reference to the druid broker
-    :type druid_broker_conn_id: str
+    This class is deprecated.
+    Please use `airflow.operators.sql.SQLCheckOperator`.

Review comment:
       There will be very shortly - I am working on adding SemVER versioning changes and procedure for providers.




----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery Check operators

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


   @turbaszek  there is a pylint error that is not relevant to this PR. Do you want me to take a look at that too?


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

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



[GitHub] [airflow] turbaszek commented on pull request #12677: Refactor SQL/BigQuery Check operators

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


   > @turbaszek there is a pylint error that is not relevant to this PR. Do you want me to take a look at that too?
   
   Please take a look at the issue as I think it's related to your change - you introduced a new level of inheritance. 


----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       Does it needs any action from a user side is why I am thinking about it -- if yes, then we might need a note somewhere in Up:
   
   https://github.com/apache/airflow/blob/master/airflow/providers/google/cloud/ADDITIONAL_INFO.md




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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


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


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


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


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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       Renamed, this is the new conn_type list for google:
   ![image](https://user-images.githubusercontent.com/27927454/109047564-de2d3080-768a-11eb-95ee-6e614ae6c27a.png)
   




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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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


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


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

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



[GitHub] [airflow] kaxil commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -120,16 +148,16 @@ def _convert_to_float_if_possible(s):
     return ret
 
 
-class SQLValueCheckOperator(BaseOperator):
+class SQLValueCheckOperator(BaseSQLOperator):

Review comment:
       cc @potiuk 




----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   @turbaszek @potiuk Hi do you have any further feedback or comments that would want me to address?


----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/providers/qubole/operators/qubole_check.py
##########
@@ -16,17 +16,40 @@
 # specific language governing permissions and limitations
 # under the License.
 #
-from typing import Iterable, Optional, Union
+from typing import Callable, Iterable, Optional, Union
 
 from airflow.exceptions import AirflowException
-from airflow.operators.check_operator import CheckOperator, ValueCheckOperator
-from airflow.providers.qubole.hooks.qubole import QuboleHook
+from airflow.operators.sql import SQLCheckOperator, SQLValueCheckOperator
 from airflow.providers.qubole.hooks.qubole_check import QuboleCheckHook
 from airflow.providers.qubole.operators.qubole import QuboleOperator
 from airflow.utils.decorators import apply_defaults
 
 
-class QuboleCheckOperator(CheckOperator, QuboleOperator):
+class _QuboleCheckOperatorMixin:
+    """This is a Mixin for Qubole related check operators"""
+
+    def execute(self, context=None) -> None:
+        """Execute a check operation againt Qubole"""
+        try:
+            self._hook_context = context
+            super().execute(context=context)
+        except AirflowException as e:
+            handle_airflow_exception(e, self.get_hook())
+
+    def get_db_hook(self) -> QuboleCheckHook:
+        """Get QuboleCheckHook"""
+        return self.get_hook()
+
+    # this overwrite the original QuboleOperator.get_hook() which returns a QuboleHook.
+    def get_hook(self) -> QuboleCheckHook:
+        """Reinitialising the hook, as some template fields might have changed"""
+        return QuboleCheckHook(
+            context=self._hook_context, results_parser_callable=self.results_parser_callable, **self.kwargs
+        )
+
+
+# pylint: disable=too-many-ancestors

Review comment:
       Yeah. for now it is good, but maybe leave a TODO(xinbinhuang)  note as comment?




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

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



[GitHub] [airflow] turbaszek commented on pull request #12677: Refactor SQL/BigQuery Check operators

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


   Hey @xinbinhuang there seems to be some pylint issues. Can you take a look?


----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       Does it needs any action from a user side is why I thinking about it -- if yes, then we might need a note somewhere in Up:
   
   https://github.com/apache/airflow/blob/master/airflow/providers/google/cloud/ADDITIONAL_INFO.md




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/qubole/operators/qubole.py
##########
@@ -267,7 +267,7 @@ def get_hook(self) -> QuboleHook:
         return QuboleHook(**self.kwargs)
 
     def __getattribute__(self, name: str) -> str:
-        if name in QuboleOperator.template_fields:
+        if name in _get_template_fields(self):

Review comment:
       It avoids hardcoding the class name `QuboleOperator` here, so you can inherit it safely. - so I can remove the `<class-name>.template_fields` in `qubole_check.py`




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       @potiuk I want to clarify a bit about the change here. The current Goold Providers in the master branch is broken because it doesn't return  `BigQueryHook` when we do `Connection.get_hook` (this is the original behavior). 
   
   Before these 2 PRs #12466 #12558, `Connection.get_hook` was using a static `conn_type` -> `HookClass` mapping to get the underlying hook. However, after #12558, `airflow.providers.google.common.hooks.base_google.GoogleBaseHook` was added after `BigQueryHook` so `BigQueryHook` will be overridden during providers discovery. https://github.com/PolideaInternal/airflow/blob/802159767baf1768d92c6047c2fdb2094ee7a2a8/airflow/providers/google/provider.yaml#L714-L720
   
   To solve this, I think we can either:
   - Fix the bug by putting  `BigQueryHook`  after `GoogleBaseHook`. This will give back us the old behavior.
   - Rename the `conn_type` as in this PR, which will involve some versioning consideration.
   
   I think we can fix the bug in this PR, and create a new issue/PR with the new `conn_type`. I am not familiar with the overall releasing schedule and planning, so please let me know which one do you prefer? @potiuk @kaxil 




----------------------------------------------------------------
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] RosterIn commented on pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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


   @xinbinhuang shouldn't [DruidCheckOperator](https://github.com/apache/airflow/blob/master/airflow/providers/apache/druid/operators/druid_check.py#L26) be also covered 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.

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery Check operators

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


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


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

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



[GitHub] [airflow] kaxil commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       @turbaszek @potiuk @mik-laj is this change ok?




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       Renamed, this is the new connect_type list for google:
   ![image](https://user-images.githubusercontent.com/27927454/109047564-de2d3080-768a-11eb-95ee-6e614ae6c27a.png)
   




----------------------------------------------------------------
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 merged pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   


----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   > Hey @xinbinhuang . I am working on releasing providers and the "maintenance" process of them - the PR and discussion is in #13767 I hope to get it merged tomorrow/Monday and release a new wave of providers early next week. I think I'd prefer to add this one afterwards - I would love to take a close look at the provider's backwards - compatibility then and we could work together to test the new provider mechanism (and updates to SEMVER versions of the providers that it brings) with you.
   > 
   > Would that be OK ?
   
   Sounds great to me! I will try to chime in a bit in that PR when I have time.


----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -38,7 +41,53 @@
 }
 
 
-class SQLCheckOperator(BaseOperator):
+class BaseSQLOperator(BaseOperator):
+    """
+    This is a base class for generic SQL Operator to get a DB Hook
+
+    The provided method is .get_db_hook(). The default behavior will try to
+    retrieve the DB hook based on connection type.
+    You can custom the behavior by overriding the .get_db_hook() method.
+    """
+
+    @apply_defaults
+    def __init__(self, *, conn_id: Optional[str] = None, database: Optional[str] = None, **kwargs):
+        super().__init__(**kwargs)
+        self.conn_id = conn_id
+        self.database = database
+
+    @cached_property
+    def _hook(self):
+        """Get DB Hook based on connection type"""
+        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:

Review comment:
       I think the reason for `ALLOWED_CONN_TYPE` is to only whitelist a subset of all connection types that are `DbApiHook` compatible. Maybe we can do something like this instead?
   
   
   ```python
   hook = conn.get_hook()
   if not isinstance(hook, DbApiHook):
       raise AirflowException('The connection type associated hook is not a subclass of `DbApiHook`') 
   ```
   
   However, there are edge cases that need to be resolved in future PRs to make this work.
   - Currently, bigQuery seems to be not discoverable because it overlaps with the GoogleBaseHook: https://github.com/apache/airflow/blob/5f81fc73c8ea3fc1c3b08080f439fa123926f250/airflow/providers/google/cloud/hooks/bigquery.py#L72
   - CloudSQLHook/CloudSQLDatabaseHook is not a subclass of `DbApiHook`
   https://github.com/apache/airflow/blob/5f81fc73c8ea3fc1c3b08080f439fa123926f250/airflow/providers/google/cloud/hooks/cloud_sql.py#L666
   - ... not sure if there are other edge cases




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: tests/core/test_core_to_contrib.py
##########
@@ -84,7 +84,7 @@ def test_is_class_deprecated(self, new_module, old_module):
                     # is used (=3 or =4 if @apply_defaults)
                     assert len(warnings) >= 1
                     assert any(warning.filename == __file__ for warning in warnings)
-                init_mock.assert_called_once_with()
+                init_mock.assert_called_once()

Review comment:
       Not sure if there is a better way to do this while maintain back compatibility.  `DruidCheckOperator` has a default `druid_broker_conn_id` that needs to pass down to `SQLCheckOperator.__init__(conn_id=druid_broker_conn_id, **kwargs)`

##########
File path: airflow/providers/apache/druid/operators/druid_check.py
##########
@@ -15,74 +15,22 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from typing import Any, Dict, Optional
+import warnings
 
-from airflow.exceptions import AirflowException
-from airflow.operators.check_operator import CheckOperator
-from airflow.providers.apache.druid.hooks.druid import DruidDbApiHook
-from airflow.utils.decorators import apply_defaults
+from airflow.operators.sql import SQLCheckOperator
 
 
-class DruidCheckOperator(CheckOperator):
+class DruidCheckOperator(SQLCheckOperator):
     """
-    Performs checks against Druid. The ``DruidCheckOperator`` expects
-    a sql query that will return a single row. Each value on that
-    first row is evaluated using python ``bool`` casting. If any of the
-    values return ``False`` the check is failed and errors out.
-
-    Note that Python bool casting evals the following as ``False``:
-
-    * ``False``
-    * ``0``
-    * Empty string (``""``)
-    * Empty list (``[]``)
-    * Empty dictionary or set (``{}``)
-
-    Given a query like ``SELECT COUNT(*) FROM foo``, it will fail only if
-    the count ``== 0``. You can craft much more complex query that could,
-    for instance, check that the table has the same number of rows as
-    the source table upstream, or that the count of today's partition is
-    greater than yesterday's partition, or that a set of metrics are less
-    than 3 standard deviation for the 7 day average.
-    This operator can be used as a data quality check in your pipeline, and
-    depending on where you put it in your DAG, you have the choice to
-    stop the critical path, preventing from
-    publishing dubious data, or on the side and receive email alerts
-    without stopping the progress of the DAG.
-
-    :param sql: the sql to be executed
-    :type sql: str
-    :param druid_broker_conn_id: reference to the druid broker
-    :type druid_broker_conn_id: str
+    This class is deprecated.
+    Please use `airflow.operators.sql.SQLCheckOperator`.
     """
 
-    @apply_defaults
-    def __init__(
-        self, *, sql: str, druid_broker_conn_id: str = 'druid_broker_default', **kwargs: Any
-    ) -> None:
-        super().__init__(sql=sql, **kwargs)
-        self.druid_broker_conn_id = druid_broker_conn_id
-        self.sql = sql
-
-    def get_db_hook(self) -> DruidDbApiHook:
-        """Return the druid db api hook."""
-        return DruidDbApiHook(druid_broker_conn_id=self.druid_broker_conn_id)
-
-    def get_first(self, sql: str) -> Any:
-        """
-        Executes the druid sql to druid broker and returns the first resulting row.
-
-        :param sql: the sql statement to be executed (str)
-        :type sql: str
-        """
-        with self.get_db_hook().get_conn() as cur:
-            cur.execute(sql)
-            return cur.fetchone()
-
-    def execute(self, context: Optional[Dict[Any, Any]] = None) -> None:
-        self.log.info('Executing SQL check: %s', self.sql)
-        record = self.get_first(self.sql)
-        self.log.info("Record: %s", str(record))
-        if not record:
-            raise AirflowException("The query returned None")
-        self.log.info("Success.")
+    def __init__(self, druid_broker_conn_id: str = 'druid_broker_default', **kwargs):
+        warnings.warn(
+            """This class is deprecated.
+            Please use `airflow.operators.sql.SQLCheckOperator`.""",
+            DeprecationWarning,
+            stacklevel=3,
+        )
+        super().__init__(conn_id=druid_broker_conn_id, **kwargs)

Review comment:
       Passdown `druid_broker_conn_id` to `SQLCheckOperator.__init__`

##########
File path: tests/deprecated_classes.py
##########
@@ -1008,7 +1008,7 @@
         'airflow.contrib.operators.sqoop_operator.SqoopOperator',
     ),
     (
-        'airflow.providers.apache.druid.operators.druid_check.DruidCheckOperator',
+        'airflow.operators.sql.SQLCheckOperator',

Review comment:
       Deprecate `DruidCheckOperator` in favor of `SQLCheckOperator`




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: tests/core/test_core_to_contrib.py
##########
@@ -84,7 +84,7 @@ def test_is_class_deprecated(self, new_module, old_module):
                     # is used (=3 or =4 if @apply_defaults)
                     assert len(warnings) >= 1
                     assert any(warning.filename == __file__ for warning in warnings)
-                init_mock.assert_called_once_with()
+                init_mock.assert_called_once()

Review comment:
       Not sure if there is a better way to do this while maintain back compatibility.  `DruidCheckOperator` has a default `druid_broker_conn_id` that needs to pass down to `SQLCheckOperator.__init__(conn_id=druid_broker_conn_id, **kwargs)`.
   
   So if we do `assert_called_once_with()` the test will fail because `DruidCheckOperator.__ini__()` == `SQLCheckOperator.__init__(conn_id=druid_broker_conn_id)`




----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       @xinbinhuang Do we need this 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.

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -120,16 +148,16 @@ def _convert_to_float_if_possible(s):
     return ret
 
 
-class SQLValueCheckOperator(BaseOperator):
+class SQLValueCheckOperator(BaseSQLOperator):

Review comment:
       I think so, the refactor on this operator is mainly to reduce duplicated code, and the logic, interface, and signature don't change in a backward incompatible one. But I don't know enough about how providers are released and how the core may affect it. cc @potiuk (sorry to ping you again..)




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -18,27 +18,59 @@
 from distutils.util import strtobool
 from typing import Any, Dict, Iterable, List, Mapping, Optional, SupportsAbs, Union
 
+from cached_property import cached_property
+
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
+from airflow.hooks.dbapi import DbApiHook
 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",
-    "snowflake",
-    "sqlite",
-    "vertica",
-}
-
-
-class SQLCheckOperator(BaseOperator):
+
+class BaseSQLOperator(BaseOperator):
+    """
+    This is a base class for generic SQL Operator to get a DB Hook
+
+    The provided method is .get_db_hook(). The default behavior will try to
+    retrieve the DB hook based on connection type.
+    You can custom the behavior by overriding the .get_db_hook() method.
+    """
+
+    @apply_defaults
+    def __init__(self, *, conn_id: Optional[str] = None, database: Optional[str] = None, **kwargs):
+        super().__init__(**kwargs)
+        self.conn_id = conn_id
+        self.database = database
+
+    @cached_property
+    def _hook(self):
+        """Get DB Hook based on connection type"""
+        self.log.debug("Get connection for %s", self.conn_id)
+        conn = BaseHook.get_connection(self.conn_id)
+
+        hook = conn.get_hook()
+        if not isinstance(hook, DbApiHook):
+            raise AirflowException(
+                f'The connection type is not supported by {self.__class__.__name__}. '
+                f'The associated hook should be a subclass of `DbApiHook`. Got {hook.__class__.__name__}'
+            )
+
+        if self.database:
+            hook.schema = self.database

Review comment:
       I totally agree!  But this is how the current mapping of Airflow works. Currently, the `Connection` object uses the field `schema` to specify the database (not schema) for database type connections, i.e. Postgres, MySQL, etc. And consequently for now, all `DbApiHook`s use the field schema to specify the database for the underlying DB API (i.e. `psycopg2`) and use `extra` to specify the schema if applicable. So the relationship looks like this:
   
   ```
   Connection(schema) -> DbApiHook (schema) -> DB API (dbname/database) 
   ```
   
   About the field `database`, it was already used for the `SQLBranchOperator` and also other SQL like operators. For this PR, I simply refactor it out to reduce duplicated code and add them to the `check*` operators for consistency.
   
   Though I think it will be nice to consolidate between `schema` and `database`, I think they should be addressed in a separate PR or worth a discussion in the `dev` list. I can send an email in the dev list if you don't object.




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

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



[GitHub] [airflow] turbaszek commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   @xinbinhuang nice work! Just to be sure - there is no breaking change, right? @potiuk would you mind also taking a look?


----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       I left a comment about this previously, but I think it got buried over time. 
   
   This fixes the issue where conn.get_hook will never return the bigquery hook because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.
   
   Here I copy my original comment:
   
   > Yes, there is one place that may be considered fixing rather than breaking. For airflow/providers/google/cloud/hooks/bigquery.py, I changed the hook's conn_type and hook_name from google cloud platform/Google Cloud to bigquery/BigQuery. This fixes the issue where conn.get_hook will never return the bigquery hook because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/providers/qubole/operators/qubole.py
##########
@@ -267,7 +267,7 @@ def get_hook(self) -> QuboleHook:
         return QuboleHook(**self.kwargs)
 
     def __getattribute__(self, name: str) -> str:
-        if name in QuboleOperator.template_fields:
+        if name in _get_template_fields(self):

Review comment:
       It avoids hardcoding the class name `QuboleOperator` here, so you can subclass it safely. i.e. then I can remove the `<class-name>.template_fields` in `qubole_check.py`




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       I left a comment about this previously, but I think it got buried over time. Here I copy my original comment here.
   
   This fixes the issue where conn.get_hook will never return the bigquery hook because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.
   
   > Yes, there is one place that may be considered fixing rather than breaking. For airflow/providers/google/cloud/hooks/bigquery.py, I changed the hook's conn_type and hook_name from google cloud platform/Google Cloud to bigquery/BigQuery. This fixes the issue where conn.get_hook will never return the bigquery hook because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.

##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       I left a comment about this previously, but I think it got buried over time. 
   
   This fixes the issue where conn.get_hook will never return the bigquery hook because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.
   
   > Yes, there is one place that may be considered fixing rather than breaking. For airflow/providers/google/cloud/hooks/bigquery.py, I changed the hook's conn_type and hook_name from google cloud platform/Google Cloud to bigquery/BigQuery. This fixes the issue where conn.get_hook will never return the bigquery hook because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.




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

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



[GitHub] [airflow] turbaszek commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   @ashb @mik-laj @potiuk would you mind taking a look?


----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/providers/qubole/operators/qubole_check.py
##########
@@ -16,17 +16,40 @@
 # specific language governing permissions and limitations
 # under the License.
 #
-from typing import Iterable, Optional, Union
+from typing import Callable, Iterable, Optional, Union
 
 from airflow.exceptions import AirflowException
-from airflow.operators.check_operator import CheckOperator, ValueCheckOperator
-from airflow.providers.qubole.hooks.qubole import QuboleHook
+from airflow.operators.sql import SQLCheckOperator, SQLValueCheckOperator
 from airflow.providers.qubole.hooks.qubole_check import QuboleCheckHook
 from airflow.providers.qubole.operators.qubole import QuboleOperator
 from airflow.utils.decorators import apply_defaults
 
 
-class QuboleCheckOperator(CheckOperator, QuboleOperator):
+class _QuboleCheckOperatorMixin:
+    """This is a Mixin for Qubole related check operators"""
+
+    def execute(self, context=None) -> None:
+        """Execute a check operation againt Qubole"""
+        try:
+            self._hook_context = context
+            super().execute(context=context)
+        except AirflowException as e:
+            handle_airflow_exception(e, self.get_hook())
+
+    def get_db_hook(self) -> QuboleCheckHook:
+        """Get QuboleCheckHook"""
+        return self.get_hook()
+
+    # this overwrite the original QuboleOperator.get_hook() which returns a QuboleHook.
+    def get_hook(self) -> QuboleCheckHook:
+        """Reinitialising the hook, as some template fields might have changed"""
+        return QuboleCheckHook(
+            context=self._hook_context, results_parser_callable=self.results_parser_callable, **self.kwargs
+        )
+
+
+# pylint: disable=too-many-ancestors

Review comment:
       Temporarily disable pylint for these 2 operators. Will submit a second PR in the future to simplify the inheritance.




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -120,16 +148,16 @@ def _convert_to_float_if_possible(s):
     return ret
 
 
-class SQLValueCheckOperator(BaseOperator):
+class SQLValueCheckOperator(BaseSQLOperator):

Review comment:
       I think so, the refactor on this operator is mainly to reduce duplicated code, and the logic, interface, and signature don't change in a backward incompatible one. But I don't know enough about how providers are released and how the core may affect it. @potiuk (sorry to ping you again..)




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       > Does it needs any action from a user side is why I am thinking about it ? -- if yes, then we might need a note somewhere in Up: https://github.com/apache/airflow/blob/master/airflow/providers/google/cloud/ADDITIONAL_INFO.md
   
   Hmm, I think we need a note for it. From sole user perspective, the conn_type for BigQuery changes from `Google Cloud` -> `Google BigQuery`.
   
   Should I update the providers' version by 0.0.1 and add a note to the `ADDITIONAL_INFO.md` or other documentation?




----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -120,16 +148,16 @@ def _convert_to_float_if_possible(s):
     return ret
 
 
-class SQLValueCheckOperator(BaseOperator):
+class SQLValueCheckOperator(BaseSQLOperator):

Review comment:
       Since this operator is in Airflow Core, can we release providers with the changes in this PR without corresponding core changes




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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery Check operators

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


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


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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -18,27 +18,59 @@
 from distutils.util import strtobool
 from typing import Any, Dict, Iterable, List, Mapping, Optional, SupportsAbs, Union
 
+from cached_property import cached_property
+
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
+from airflow.hooks.dbapi import DbApiHook
 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",
-    "snowflake",
-    "sqlite",
-    "vertica",
-}
-
-
-class SQLCheckOperator(BaseOperator):
+
+class BaseSQLOperator(BaseOperator):
+    """
+    This is a base class for generic SQL Operator to get a DB Hook
+
+    The provided method is .get_db_hook(). The default behavior will try to
+    retrieve the DB hook based on connection type.
+    You can custom the behavior by overriding the .get_db_hook() method.
+    """
+
+    @apply_defaults
+    def __init__(self, *, conn_id: Optional[str] = None, database: Optional[str] = None, **kwargs):
+        super().__init__(**kwargs)
+        self.conn_id = conn_id
+        self.database = database
+
+    @cached_property
+    def _hook(self):
+        """Get DB Hook based on connection type"""
+        self.log.debug("Get connection for %s", self.conn_id)
+        conn = BaseHook.get_connection(self.conn_id)
+
+        hook = conn.get_hook()
+        if not isinstance(hook, DbApiHook):
+            raise AirflowException(
+                f'The connection type is not supported by {self.__class__.__name__}. '
+                f'The associated hook should be a subclass of `DbApiHook`. Got {hook.__class__.__name__}'
+            )
+
+        if self.database:
+            hook.schema = self.database

Review comment:
       I totally agree!  But this is how the current mapping of Airflow works. Currently, the `Connection` object uses the field `schema` to specify the database (not schema) for database type connections, i.e. Postgres, MySQL, etc. And as a consequence, all `DbApiHook`s use the field schema to specify the database for the underlying DB API (i.e. `psycopg2`) and use `extra` to specify the schema if applicable. So the relationship looks like this:
   
   ```
   Connection(schema) -> DbApiHook (schema) -> DB API (dbname/database) 
   ```
   
   About the field `database`, it was already used for the `SQLBranchOperator` and also other SQL like operators. For this PR, I simply refactor it out to reduce duplicated code and add them to the `check*` operators for consistency.
   
   Though I think it will be nice to consolidate between `schema` and `database`, I think they should be addressed in a separate PR or worth a discussion in the `dev` list. I can send an email in the dev list if you don't object.

##########
File path: airflow/operators/sql.py
##########
@@ -18,27 +18,59 @@
 from distutils.util import strtobool
 from typing import Any, Dict, Iterable, List, Mapping, Optional, SupportsAbs, Union
 
+from cached_property import cached_property
+
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
+from airflow.hooks.dbapi import DbApiHook
 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",
-    "snowflake",
-    "sqlite",
-    "vertica",
-}
-
-
-class SQLCheckOperator(BaseOperator):
+
+class BaseSQLOperator(BaseOperator):
+    """
+    This is a base class for generic SQL Operator to get a DB Hook
+
+    The provided method is .get_db_hook(). The default behavior will try to
+    retrieve the DB hook based on connection type.
+    You can custom the behavior by overriding the .get_db_hook() method.
+    """
+
+    @apply_defaults
+    def __init__(self, *, conn_id: Optional[str] = None, database: Optional[str] = None, **kwargs):
+        super().__init__(**kwargs)
+        self.conn_id = conn_id
+        self.database = database
+
+    @cached_property
+    def _hook(self):
+        """Get DB Hook based on connection type"""
+        self.log.debug("Get connection for %s", self.conn_id)
+        conn = BaseHook.get_connection(self.conn_id)
+
+        hook = conn.get_hook()
+        if not isinstance(hook, DbApiHook):
+            raise AirflowException(
+                f'The connection type is not supported by {self.__class__.__name__}. '
+                f'The associated hook should be a subclass of `DbApiHook`. Got {hook.__class__.__name__}'
+            )
+
+        if self.database:
+            hook.schema = self.database

Review comment:
       I totally agree!  But this is how the current mapping of Airflow works. Currently, the `Connection` object uses the field `schema` to specify the database (not schema) for database type connections, i.e. Postgres, MySQL, etc. And as a result, all `DbApiHook`s use the field schema to specify the database for the underlying DB API (i.e. `psycopg2`) and use `extra` to specify the schema if applicable. So the relationship looks like this:
   
   ```
   Connection(schema) -> DbApiHook (schema) -> DB API (dbname/database) 
   ```
   
   About the field `database`, it was already used for the `SQLBranchOperator` and also other SQL like operators. For this PR, I simply refactor it out to reduce duplicated code and add them to the `check*` operators for consistency.
   
   Though I think it will be nice to consolidate between `schema` and `database`, I think they should be addressed in a separate PR or worth a discussion in the `dev` list. I can send an email in the dev list if you don't object.




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       I left a comment about this in the thread previously, but I think it got buried over time. 
   
   This fixes the issue that `conn.get_hook` will never return the bigquery hook after we start to use the `ProvidersManager` because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.
   
   Here I copy my original comment:
   
   > Yes, there is one place that may be considered fixing rather than breaking. For airflow/providers/google/cloud/hooks/bigquery.py, I changed the hook's conn_type and hook_name from google cloud platform/Google Cloud to bigquery/BigQuery. This fixes the issue where conn.get_hook will never return the bigquery hook because GoogleBaseHook "hides" the BigQueryHook as they have the same conn_type.




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #12677: Refactor SQL/BigQuery Check operators

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



##########
File path: airflow/providers/google/cloud/operators/bigquery.py
##########
@@ -87,7 +87,17 @@ def get_link(self, operator: BaseOperator, dttm: datetime):
         return BIGQUERY_JOB_DETAILS_LINK_FMT.format(job_id=job_id)
 
 
-class BigQueryCheckOperator(CheckOperator):
+class BigQueryDbHookMixin:
+    def get_db_hook(self) -> BigQueryHook:
+        return BigQueryHook(
+            gcp_conn_id=self.gcp_conn_id,
+            use_legacy_sql=self.use_legacy_sql,
+            location=self.location,
+            impersonation_chain=self.impersonation_chain,
+        )
+
+
+class BigQueryCheckOperator(CheckOperator, BigQueryDbHookMixin):

Review comment:
       This is not a pattern recommended by Google:
   https://s.apache.org/google-operators
   
   It is rather preferred to instantiated the hook in execute method in operator. In this way refactoring and future changes are simpler and we reduce cross dependencies between classes. 




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

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



[GitHub] [airflow] turbaszek commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: tests/core/test_core_to_contrib.py
##########
@@ -84,7 +84,7 @@ def test_is_class_deprecated(self, new_module, old_module):
                     # is used (=3 or =4 if @apply_defaults)
                     assert len(warnings) >= 1
                     assert any(warning.filename == __file__ for warning in warnings)
-                init_mock.assert_called_once_with()
+                init_mock.assert_called_once()

Review comment:
       @xinbinhuang should this help?
   ```
   assert_called_once_with(conn_id=mock.ANY)
   ```




----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/druid_check_operator.py
##########
@@ -23,7 +23,7 @@
 from airflow.providers.apache.druid.operators.druid_check import DruidCheckOperator  # noqa
 
 warnings.warn(
-    "This module is deprecated. Please use `airflow.providers.apache.druid.operators.druid_check`.",
+    "This module is deprecated. Please use `airflow.operators.sql.SQLCheckOperator`.",

Review comment:
       nvm saw the change later in the PR




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/qubole/operators/qubole.py
##########
@@ -267,7 +267,7 @@ def get_hook(self) -> QuboleHook:
         return QuboleHook(**self.kwargs)
 
     def __getattribute__(self, name: str) -> str:
-        if name in QuboleOperator.template_fields:
+        if name in _get_template_fields(self):

Review comment:
       It avoids hardcoding the class name `QuboleOperator` here, so you can inherit it safely. i.e. so I can remove the `<class-name>.template_fields` in `qubole_check.py`




----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/druid_check_operator.py
##########
@@ -23,7 +23,7 @@
 from airflow.providers.apache.druid.operators.druid_check import DruidCheckOperator  # noqa
 
 warnings.warn(
-    "This module is deprecated. Please use `airflow.providers.apache.druid.operators.druid_check`.",
+    "This module is deprecated. Please use `airflow.operators.sql.SQLCheckOperator`.",

Review comment:
       Oh why is this?




----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -38,7 +41,53 @@
 }
 
 
-class SQLCheckOperator(BaseOperator):
+class BaseSQLOperator(BaseOperator):
+    """
+    This is a base class for generic SQL Operator to get a DB Hook
+
+    The provided method is .get_db_hook(). The default behavior will try to
+    retrieve the DB hook based on connection type.
+    You can custom the behavior by overriding the .get_db_hook() method.
+    """
+
+    @apply_defaults
+    def __init__(self, *, conn_id: Optional[str] = None, database: Optional[str] = None, **kwargs):
+        super().__init__(**kwargs)
+        self.conn_id = conn_id
+        self.database = database
+
+    @cached_property
+    def _hook(self):
+        """Get DB Hook based on connection type"""
+        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:

Review comment:
       Hmm. I think we should rather check the povider's manager hook property for that. We've added automated discovery of providers so that they can register their own connection -> Hook mapping so maybe we should utilise that? https://github.com/apache/airflow/blob/6f246b0d54ccaf733b7c5951a8955adda6719acb/airflow/providers_manager.py#L387 
   




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/providers/apache/druid/operators/druid_check.py
##########
@@ -15,74 +15,23 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from typing import Any, Dict, Optional
+import warnings
 
-from airflow.exceptions import AirflowException
-from airflow.operators.check_operator import CheckOperator
-from airflow.providers.apache.druid.hooks.druid import DruidDbApiHook
-from airflow.utils.decorators import apply_defaults
+from airflow.operators.sql import SQLCheckOperator
 
 
-class DruidCheckOperator(CheckOperator):
+class DruidCheckOperator(SQLCheckOperator):
     """
-    Performs checks against Druid. The ``DruidCheckOperator`` expects
-    a sql query that will return a single row. Each value on that
-    first row is evaluated using python ``bool`` casting. If any of the
-    values return ``False`` the check is failed and errors out.
-
-    Note that Python bool casting evals the following as ``False``:
-
-    * ``False``
-    * ``0``
-    * Empty string (``""``)
-    * Empty list (``[]``)
-    * Empty dictionary or set (``{}``)
-
-    Given a query like ``SELECT COUNT(*) FROM foo``, it will fail only if
-    the count ``== 0``. You can craft much more complex query that could,
-    for instance, check that the table has the same number of rows as
-    the source table upstream, or that the count of today's partition is
-    greater than yesterday's partition, or that a set of metrics are less
-    than 3 standard deviation for the 7 day average.
-    This operator can be used as a data quality check in your pipeline, and
-    depending on where you put it in your DAG, you have the choice to
-    stop the critical path, preventing from
-    publishing dubious data, or on the side and receive email alerts
-    without stopping the progress of the DAG.
-
-    :param sql: the sql to be executed
-    :type sql: str
-    :param druid_broker_conn_id: reference to the druid broker
-    :type druid_broker_conn_id: str
+    This class is deprecated.
+    Please use `airflow.operators.sql.SQLCheckOperator`.

Review comment:
       So I will just keep the deprecation inside the `class`  for now. Do you think that is ok?




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       > Does it needs any action from a user side is why I am thinking about it ? -- if yes, then we might need a note somewhere in Up: https://github.com/apache/airflow/blob/master/airflow/providers/google/cloud/ADDITIONAL_INFO.md
   
   Hmm, I think we need a note for it. From sole user perspective, the conn_type for BigQuery changes from `Google Cloud` -> `Google BigQuery`. However, if we don't change it the current master branch will break anything that relies on `Connection.get_hook` to return a `BigQueryHook`.
   
   Should I update the providers' version by 0.0.1 and add a note to the `ADDITIONAL_INFO.md` or other documentation?




----------------------------------------------------------------
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 #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   It is failing unit tests:
   
   https://github.com/apache/airflow/pull/12677/checks?check_run_id=1932643565#step:6:942
   
   ```
     _ TestMovingCoreToContrib.test_is_class_deprecated_097_airflow_operators_sql_SQLCheckOperator _
   
     >           raise AssertionError(_error_message()) from cause
     E           AssertionError: expected call not found.
     E           Expected: __init__()
     E           Actual: __init__(conn_id='druid_broker_default')
     
     /usr/local/lib/python3.8/unittest/mock.py:913: AssertionError
     ________ TestMovingCoreToContrib.test_no_redirect_to_deprecated_classes ________
   
   
   ....
   
   
                 if new in all_classes_by_old:
     >               raise AssertionError(
                         f'Deprecation "{old}" to "{new}" is incorrect. '
                         f'Please use \"{all_classes_by_old[new]}\" instead of "{old}".'
                     )
     E               AssertionError: Deprecation "airflow.operators.druid_check_operator.DruidCheckOperator" to "airflow.providers.apache.druid.operators.druid_check.DruidCheckOperator" is incorrect. Please use "airflow.operators.sql.SQLCheckOperator" instead of "airflow.operators.druid_check_operator.DruidCheckOperator".
     
     tests/core/test_core_to_contrib.py:116: AssertionError
     _ TestMovingCoreToContrib.test_warning_on_import_243_airflow_operators_sql_SQLCheckOperator _
   ```


----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       As similar to other Google cloud connection type (before this change, you can't find bigquery in the list)
   
   ![image](https://user-images.githubusercontent.com/27927454/109046868-12ecb800-768a-11eb-970c-6d52905825fa.png)
   ![image](https://user-images.githubusercontent.com/27927454/109046902-1bdd8980-768a-11eb-9cc2-155f26ef7898.png)
   
   Maybe I should rename it to `gcpbigquery` & `Google Bigquery` instead?




----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -120,16 +148,16 @@ def _convert_to_float_if_possible(s):
     return ret
 
 
-class SQLValueCheckOperator(BaseOperator):
+class SQLValueCheckOperator(BaseSQLOperator):

Review comment:
       I think it should work




----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   Hi @potiuk , I noticed that #13767 has been merged. I wonder would be your suggestion on upgrading this PR to follow the versioning policy? Can you give me some pointers? 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] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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


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


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

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



[GitHub] [airflow] msumit commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/providers/qubole/operators/qubole.py
##########
@@ -267,7 +267,7 @@ def get_hook(self) -> QuboleHook:
         return QuboleHook(**self.kwargs)
 
     def __getattribute__(self, name: str) -> str:
-        if name in QuboleOperator.template_fields:
+        if name in _get_template_fields(self):

Review comment:
       what is the rationale behind this 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.

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



[GitHub] [airflow] turbaszek commented on a change in pull request #12677: Refactor SQL/BigQuery Check operators

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



##########
File path: tests/operators/test_sql.py
##########
@@ -322,7 +359,7 @@ def test_pass_min_value_max_sql(self, mock_get_db_hook):
 
         operator.execute()
 
-    @mock.patch.object(ThresholdCheckOperator, "get_db_hook")
+    @mock.patch.object(SQLThresholdCheckOperator, "get_db_hook")

Review comment:
       > tests were forgotten to refactor I believe.
   
   That was what I though, wanted to check. Thanks!

##########
File path: tests/operators/test_sql.py
##########
@@ -322,7 +359,7 @@ def test_pass_min_value_max_sql(self, mock_get_db_hook):
 
         operator.execute()
 
-    @mock.patch.object(ThresholdCheckOperator, "get_db_hook")
+    @mock.patch.object(SQLThresholdCheckOperator, "get_db_hook")

Review comment:
       > tests were forgotten to refactor I believe.
   
   That was what I thought, wanted to check. 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] turbaszek commented on a change in pull request #12677: Refactor SQL/BigQuery Check operators

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



##########
File path: tests/operators/test_sql.py
##########
@@ -322,7 +359,7 @@ def test_pass_min_value_max_sql(self, mock_get_db_hook):
 
         operator.execute()
 
-    @mock.patch.object(ThresholdCheckOperator, "get_db_hook")
+    @mock.patch.object(SQLThresholdCheckOperator, "get_db_hook")

Review comment:
       Was this name changed?




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/qubole/operators/qubole.py
##########
@@ -267,7 +267,7 @@ def get_hook(self) -> QuboleHook:
         return QuboleHook(**self.kwargs)
 
     def __getattribute__(self, name: str) -> str:
-        if name in QuboleOperator.template_fields:
+        if name in _get_template_fields(self):

Review comment:
       It avoids hardcoding the class name `QuboleOperator` here, so you can inherit it safely. i.e. I can remove the `<class-name>.template_fields` in `qubole_check.py`




----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   Hi @potiuk, just a gentle ping.
   
   Do you think the PR is safe to merge? Or are there other feedback or tests that you would want me to take a look?


----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery Check operators

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



##########
File path: tests/operators/test_sql.py
##########
@@ -322,7 +359,7 @@ def test_pass_min_value_max_sql(self, mock_get_db_hook):
 
         operator.execute()
 
-    @mock.patch.object(ThresholdCheckOperator, "get_db_hook")
+    @mock.patch.object(SQLThresholdCheckOperator, "get_db_hook")

Review comment:
       You can see the original imports and the refactored imports at the top of the file - `test_sql.py`




----------------------------------------------------------------
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 a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       Yes probably add an Updating.md style note in `ADDITIONAL_INFO.md` with version ad header -- same as https://github.com/apache/airflow/blob/master/UPDATING.md

##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       Yes probably add an Updating.md style note in `ADDITIONAL_INFO.md` with version as header -- same as https://github.com/apache/airflow/blob/master/UPDATING.md




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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       I have reverted the `BigQueryHook` conn_type change and put `BigQueryHook` after `GoogleBaseHook` for 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] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/555344814) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       As similar to other Google cloud connection type (before this change, you can't find bigquery in the list)
   
   ![image](https://user-images.githubusercontent.com/27927454/109046868-12ecb800-768a-11eb-970c-6d52905825fa.png)
   ![image](https://user-images.githubusercontent.com/27927454/109046902-1bdd8980-768a-11eb-9cc2-155f26ef7898.png)
   
   Maybe I should rename it to `Google Bigquery` instead?




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: tests/core/test_core_to_contrib.py
##########
@@ -84,7 +84,7 @@ def test_is_class_deprecated(self, new_module, old_module):
                     # is used (=3 or =4 if @apply_defaults)
                     assert len(warnings) >= 1
                     assert any(warning.filename == __file__ for warning in warnings)
-                init_mock.assert_called_once_with()
+                init_mock.assert_called_once()

Review comment:
       Not sure if there is a better way to do this while maintain backward compatibility.  `DruidCheckOperator` has a default `druid_broker_conn_id` that needs to pass down to `SQLCheckOperator.__init__(conn_id=druid_broker_conn_id, **kwargs)`.
   
   So if we do `assert_called_once_with()` the test will fail because `DruidCheckOperator.__ini__()` == `SQLCheckOperator.__init__(conn_id=druid_broker_conn_id)`

##########
File path: tests/core/test_core_to_contrib.py
##########
@@ -84,7 +84,7 @@ def test_is_class_deprecated(self, new_module, old_module):
                     # is used (=3 or =4 if @apply_defaults)
                     assert len(warnings) >= 1
                     assert any(warning.filename == __file__ for warning in warnings)
-                init_mock.assert_called_once_with()
+                init_mock.assert_called_once()

Review comment:
       Not sure if there is a better way to do this while maintaining backward compatibility.  `DruidCheckOperator` has a default `druid_broker_conn_id` that needs to pass down to `SQLCheckOperator.__init__(conn_id=druid_broker_conn_id, **kwargs)`.
   
   So if we do `assert_called_once_with()` the test will fail because `DruidCheckOperator.__ini__()` == `SQLCheckOperator.__init__(conn_id=druid_broker_conn_id)`




----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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






----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/providers/apache/druid/operators/druid_check.py
##########
@@ -15,74 +15,23 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from typing import Any, Dict, Optional
+import warnings
 
-from airflow.exceptions import AirflowException
-from airflow.operators.check_operator import CheckOperator
-from airflow.providers.apache.druid.hooks.druid import DruidDbApiHook
-from airflow.utils.decorators import apply_defaults
+from airflow.operators.sql import SQLCheckOperator
 
 
-class DruidCheckOperator(CheckOperator):
+class DruidCheckOperator(SQLCheckOperator):
     """
-    Performs checks against Druid. The ``DruidCheckOperator`` expects
-    a sql query that will return a single row. Each value on that
-    first row is evaluated using python ``bool`` casting. If any of the
-    values return ``False`` the check is failed and errors out.
-
-    Note that Python bool casting evals the following as ``False``:
-
-    * ``False``
-    * ``0``
-    * Empty string (``""``)
-    * Empty list (``[]``)
-    * Empty dictionary or set (``{}``)
-
-    Given a query like ``SELECT COUNT(*) FROM foo``, it will fail only if
-    the count ``== 0``. You can craft much more complex query that could,
-    for instance, check that the table has the same number of rows as
-    the source table upstream, or that the count of today's partition is
-    greater than yesterday's partition, or that a set of metrics are less
-    than 3 standard deviation for the 7 day average.
-    This operator can be used as a data quality check in your pipeline, and
-    depending on where you put it in your DAG, you have the choice to
-    stop the critical path, preventing from
-    publishing dubious data, or on the side and receive email alerts
-    without stopping the progress of the DAG.
-
-    :param sql: the sql to be executed
-    :type sql: str
-    :param druid_broker_conn_id: reference to the druid broker
-    :type druid_broker_conn_id: str
+    This class is deprecated.
+    Please use `airflow.operators.sql.SQLCheckOperator`.

Review comment:
       Is there a deprecation process for Provider packages? i.e. any changes to `providers.yaml` and the README file?
   
   @potiuk 

##########
File path: airflow/providers/apache/druid/operators/druid_check.py
##########
@@ -15,74 +15,23 @@
 # KIND, either express or implied.  See the License for the
 # specific language governing permissions and limitations
 # under the License.
-from typing import Any, Dict, Optional
+import warnings
 
-from airflow.exceptions import AirflowException
-from airflow.operators.check_operator import CheckOperator
-from airflow.providers.apache.druid.hooks.druid import DruidDbApiHook
-from airflow.utils.decorators import apply_defaults
+from airflow.operators.sql import SQLCheckOperator
 
 
-class DruidCheckOperator(CheckOperator):
+class DruidCheckOperator(SQLCheckOperator):
     """
-    Performs checks against Druid. The ``DruidCheckOperator`` expects
-    a sql query that will return a single row. Each value on that
-    first row is evaluated using python ``bool`` casting. If any of the
-    values return ``False`` the check is failed and errors out.
-
-    Note that Python bool casting evals the following as ``False``:
-
-    * ``False``
-    * ``0``
-    * Empty string (``""``)
-    * Empty list (``[]``)
-    * Empty dictionary or set (``{}``)
-
-    Given a query like ``SELECT COUNT(*) FROM foo``, it will fail only if
-    the count ``== 0``. You can craft much more complex query that could,
-    for instance, check that the table has the same number of rows as
-    the source table upstream, or that the count of today's partition is
-    greater than yesterday's partition, or that a set of metrics are less
-    than 3 standard deviation for the 7 day average.
-    This operator can be used as a data quality check in your pipeline, and
-    depending on where you put it in your DAG, you have the choice to
-    stop the critical path, preventing from
-    publishing dubious data, or on the side and receive email alerts
-    without stopping the progress of the DAG.
-
-    :param sql: the sql to be executed
-    :type sql: str
-    :param druid_broker_conn_id: reference to the druid broker
-    :type druid_broker_conn_id: str
+    This class is deprecated.
+    Please use `airflow.operators.sql.SQLCheckOperator`.

Review comment:
       Is there a deprecation process for Provider packages? i.e. any changes to `provider.yaml` and the README file?
   
   @potiuk 




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery Check operators

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



##########
File path: tests/operators/test_sql.py
##########
@@ -322,7 +359,7 @@ def test_pass_min_value_max_sql(self, mock_get_db_hook):
 
         operator.execute()
 
-    @mock.patch.object(ThresholdCheckOperator, "get_db_hook")
+    @mock.patch.object(SQLThresholdCheckOperator, "get_db_hook")

Review comment:
       Yeah. The original operators were moved to another module and renamed but the tests were forgotten to refactor I believe.




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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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


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


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

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



[GitHub] [airflow] kaxil commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       or in https://github.com/apache/airflow/blob/master/airflow/providers/google/CHANGELOG.rst actually




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       As similar to other Google cloud connection type:
   
   ![image](https://user-images.githubusercontent.com/27927454/109046868-12ecb800-768a-11eb-970c-6d52905825fa.png)
   ![image](https://user-images.githubusercontent.com/27927454/109046902-1bdd8980-768a-11eb-9cc2-155f26ef7898.png)
   
   Maybe I should rename it to `gcpbigquery` & `Google Bigquery` instead?




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/providers/qubole/operators/qubole.py
##########
@@ -267,7 +267,7 @@ def get_hook(self) -> QuboleHook:
         return QuboleHook(**self.kwargs)
 
     def __getattribute__(self, name: str) -> str:
-        if name in QuboleOperator.template_fields:
+        if name in _get_template_fields(self):

Review comment:
       It avoids hardcoding the class name `QuboleOperator` here, so you can subclass it safely. i.e. you can see it in `qubole_check.py`




----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   > @xinbinhuang nice work! Just to be sure - there is no breaking change, right? @potiuk would you mind also taking a look?
   
   Yes, there is one place that may be considered fixing rather than breaking. For `airflow/providers/google/cloud/hooks/bigquery.py`, I changed the hook's `conn_type` and `hook_name`  from `google cloud platform`/`Google Cloud` to `bigquery`/`BigQuery`. This fixes the issue where `conn.get_hook` will never return the bigquery hook because `BaseGcpHook` "hides" the `BigQueryHook` as they have the same `conn_type`.


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


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


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

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



[GitHub] [airflow] potiuk commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       Just a comment on this one. Yeah. I wanted to avoid this change when we switched to 2.0 (not to add backwards incompatibilities), but this change indeed breaks the compatibility of Google Provider, so we should mention that in the CHANGELOG @kaxil mentioned. I think however, whether this is a good time to "merge" this change now - we are planning a bigger release of Google Provider (3.0.0) around March/April when we will drop "backports" altogether. We need to update dependencies and upgrade a few remaining APIs of python libraries of Google - and such a backwards-incompatible change could be added then. 
   
   I think if we could make this change without this breaking compatibility now, that would be ideal.
   




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/providers/google/cloud/hooks/bigquery.py
##########
@@ -69,8 +69,8 @@ class BigQueryHook(GoogleBaseHook, DbApiHook):
 
     conn_name_attr = 'gcp_conn_id'
     default_conn_name = 'google_cloud_default'
-    conn_type = 'google_cloud_platform'
-    hook_name = 'Google Cloud'
+    conn_type = 'bigquery'
+    hook_name = 'BigQuery'

Review comment:
       > Does it needs any action from a user side is why I am thinking about it ? -- if yes, then we might need a note somewhere in Up: https://github.com/apache/airflow/blob/master/airflow/providers/google/cloud/ADDITIONAL_INFO.md
   
   Hmm, I think we need a note for it. From sole user perspective, the conn_type for BigQuery changes from `Google Cloud` -> `Google BigQuery`. However, if we don't change it the current master branch will break or make it impossible on things that rely on `Connection.get_hook` to return a `BigQueryHook`.
   
   Should I update the providers' version by 0.0.1 and add a note to the `ADDITIONAL_INFO.md` or other documentation?




----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole Check operators

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



##########
File path: airflow/providers/qubole/operators/qubole.py
##########
@@ -267,7 +267,7 @@ def get_hook(self) -> QuboleHook:
         return QuboleHook(**self.kwargs)
 
     def __getattribute__(self, name: str) -> str:
-        if name in QuboleOperator.template_fields:
+        if name in _get_template_fields(self):

Review comment:
       It avoids hardcoding the class name `QuboleOperator` here, so you can subclass it safely. i.e. I can remove the `<class-name>.template_fields` in `qubole_check.py`




----------------------------------------------------------------
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 edited a comment on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

Posted by GitBox <gi...@apache.org>.
potiuk edited a comment on pull request #12677:
URL: https://github.com/apache/airflow/pull/12677#issuecomment-764836879


   Hey @xinbinhuang . I am working on releasing providers and the "maintenance" process of them  - the PR and discussion is in #13767 I hope to get it merged tomorrow/Monday and release a new wave of providers early next week. I think I'd prefer to add this one afterwards - I would love to take a close look at the provider's backwards - compatibility then and we could work together to test the new provider mechanism (and updates to SEMVER versions of the providers that it brings) with you. 
   
   Would that be OK ?


----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -18,27 +18,59 @@
 from distutils.util import strtobool
 from typing import Any, Dict, Iterable, List, Mapping, Optional, SupportsAbs, Union
 
+from cached_property import cached_property
+
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
+from airflow.hooks.dbapi import DbApiHook
 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",
-    "snowflake",
-    "sqlite",
-    "vertica",
-}
-
-
-class SQLCheckOperator(BaseOperator):
+
+class BaseSQLOperator(BaseOperator):
+    """
+    This is a base class for generic SQL Operator to get a DB Hook
+
+    The provided method is .get_db_hook(). The default behavior will try to
+    retrieve the DB hook based on connection type.
+    You can custom the behavior by overriding the .get_db_hook() method.
+    """
+
+    @apply_defaults
+    def __init__(self, *, conn_id: Optional[str] = None, database: Optional[str] = None, **kwargs):
+        super().__init__(**kwargs)
+        self.conn_id = conn_id
+        self.database = database
+
+    @cached_property
+    def _hook(self):
+        """Get DB Hook based on connection type"""
+        self.log.debug("Get connection for %s", self.conn_id)
+        conn = BaseHook.get_connection(self.conn_id)
+
+        hook = conn.get_hook()
+        if not isinstance(hook, DbApiHook):
+            raise AirflowException(
+                f'The connection type is not supported by {self.__class__.__name__}. '
+                f'The associated hook should be a subclass of `DbApiHook`. Got {hook.__class__.__name__}'
+            )
+
+        if self.database:
+            hook.schema = self.database

Review comment:
       I totally agree!  But this is how the current mapping of Airflow works. Currently, the `Connection` object uses the field `schema` to specify the database (not schema) for database type connections, i.e. Postgres, MySQL, etc. And as a result, all `DbApiHook`s use the field `schema` to specify the database to connect to using the underlying DB API (i.e. `psycopg2`). `extra` is used to specify the schema if applicable. So the relationship looks like this:
   
   ```
   Connection(schema) -> DbApiHook (schema) -> DB API (dbname/database) 
   ```
   
   About the field `database`, it was already used for the `SQLBranchOperator` and also other SQL like operators. For this PR, I simply refactor it out to reduce duplicated code and add them to the `check*` operators for consistency.
   
   Though I think it will be nice to consolidate between `schema` and `database`, I think they should be addressed in a separate PR or worth a discussion in the `dev` list. I can send an email in the dev list if you don't object.




----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   @turbaszek @potiuk @kaxil Thank you for reviewing!


----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery Check operators

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


   @turbaszek thanks for taking a look! I submitted the PR pretty late last night so I didn't  have time to add description to the PR. I will update the description later today and double check the CI failure.


----------------------------------------------------------------
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 edited a comment on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

Posted by GitBox <gi...@apache.org>.
kaxil edited a comment on pull request #12677:
URL: https://github.com/apache/airflow/pull/12677#issuecomment-782371508


   It is failing unit tests that are related to this PR though:
   
   https://github.com/apache/airflow/pull/12677/checks?check_run_id=1932643565#step:6:942
   
   ```
     _ TestMovingCoreToContrib.test_is_class_deprecated_097_airflow_operators_sql_SQLCheckOperator _
   
     >           raise AssertionError(_error_message()) from cause
     E           AssertionError: expected call not found.
     E           Expected: __init__()
     E           Actual: __init__(conn_id='druid_broker_default')
     
     /usr/local/lib/python3.8/unittest/mock.py:913: AssertionError
     ________ TestMovingCoreToContrib.test_no_redirect_to_deprecated_classes ________
   
   
   ....
   
   
                 if new in all_classes_by_old:
     >               raise AssertionError(
                         f'Deprecation "{old}" to "{new}" is incorrect. '
                         f'Please use \"{all_classes_by_old[new]}\" instead of "{old}".'
                     )
     E               AssertionError: Deprecation "airflow.operators.druid_check_operator.DruidCheckOperator" to "airflow.providers.apache.druid.operators.druid_check.DruidCheckOperator" is incorrect. Please use "airflow.operators.sql.SQLCheckOperator" instead of "airflow.operators.druid_check_operator.DruidCheckOperator".
     
     tests/core/test_core_to_contrib.py:116: AssertionError
     _ TestMovingCoreToContrib.test_warning_on_import_243_airflow_operators_sql_SQLCheckOperator _
   ```


----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery Check operators

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


   > > @turbaszek there is a pylint error that is not relevant to this PR. Do you want me to take a look at that too?
   > 
   > Please take a look at the issue as I think it's related to your change - you introduced a new level of inheritance.
   
   Ahhh, you are right. Thanks for the tips. Let me fix it in a bit


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


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


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

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



[GitHub] [airflow] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -18,27 +18,59 @@
 from distutils.util import strtobool
 from typing import Any, Dict, Iterable, List, Mapping, Optional, SupportsAbs, Union
 
+from cached_property import cached_property
+
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
+from airflow.hooks.dbapi import DbApiHook
 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",
-    "snowflake",
-    "sqlite",
-    "vertica",
-}
-
-
-class SQLCheckOperator(BaseOperator):
+
+class BaseSQLOperator(BaseOperator):
+    """
+    This is a base class for generic SQL Operator to get a DB Hook
+
+    The provided method is .get_db_hook(). The default behavior will try to
+    retrieve the DB hook based on connection type.
+    You can custom the behavior by overriding the .get_db_hook() method.
+    """
+
+    @apply_defaults
+    def __init__(self, *, conn_id: Optional[str] = None, database: Optional[str] = None, **kwargs):
+        super().__init__(**kwargs)
+        self.conn_id = conn_id
+        self.database = database
+
+    @cached_property
+    def _hook(self):
+        """Get DB Hook based on connection type"""
+        self.log.debug("Get connection for %s", self.conn_id)
+        conn = BaseHook.get_connection(self.conn_id)
+
+        hook = conn.get_hook()
+        if not isinstance(hook, DbApiHook):
+            raise AirflowException(
+                f'The connection type is not supported by {self.__class__.__name__}. '
+                f'The associated hook should be a subclass of `DbApiHook`. Got {hook.__class__.__name__}'
+            )
+
+        if self.database:
+            hook.schema = self.database

Review comment:
       I totally agree!  But this is how the current mapping of Airflow works. Currently, the `Connection` object uses the field `schema` to specify the database (not schema) for database type connections, i.e. Postgres, MySQL, etc. And consequently for now, all `DbApiHook`s use the field schema to specify the database to connect to and use `extra` to specify the schema if applicable. Here is a visual mapping:
   
   ```
   Connection(schema) -> DbApiHook (schema) -> DB API (dbname/database) 
   ```
   
   About the field `database`, it was already used for the `SQLBranchOperator` and also other SQL like operators. For this PR, I simply refactor it out to reduce duplicated code and add them to the `check*` operators for consistency.
   
   Though I think it will be nice to consolidate between `schema` and `database`, I think they should be addressed in a separate PR or worth a discussion in the `dev` list. I can send an email in the dev list if you don't object.




----------------------------------------------------------------
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] xinbinhuang commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   @kaxil @turbaszek @potiuk  Can we merge this as it's already approved? It's pretty hard to get CI happy because it often fails for reasons that not related to this PR.


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

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



[GitHub] [airflow] github-actions[bot] commented on pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   [The Workflow run](https://github.com/apache/airflow/actions/runs/584828588) is cancelling this PR. Building images for the PR has failed. Follow the the workflow link to check the reason.


----------------------------------------------------------------
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] xinbinhuang commented on a change in pull request #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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



##########
File path: airflow/operators/sql.py
##########
@@ -18,27 +18,59 @@
 from distutils.util import strtobool
 from typing import Any, Dict, Iterable, List, Mapping, Optional, SupportsAbs, Union
 
+from cached_property import cached_property
+
 from airflow.exceptions import AirflowException
 from airflow.hooks.base import BaseHook
+from airflow.hooks.dbapi import DbApiHook
 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",
-    "snowflake",
-    "sqlite",
-    "vertica",
-}
-
-
-class SQLCheckOperator(BaseOperator):
+
+class BaseSQLOperator(BaseOperator):
+    """
+    This is a base class for generic SQL Operator to get a DB Hook
+
+    The provided method is .get_db_hook(). The default behavior will try to
+    retrieve the DB hook based on connection type.
+    You can custom the behavior by overriding the .get_db_hook() method.
+    """
+
+    @apply_defaults
+    def __init__(self, *, conn_id: Optional[str] = None, database: Optional[str] = None, **kwargs):
+        super().__init__(**kwargs)
+        self.conn_id = conn_id
+        self.database = database
+
+    @cached_property
+    def _hook(self):
+        """Get DB Hook based on connection type"""
+        self.log.debug("Get connection for %s", self.conn_id)
+        conn = BaseHook.get_connection(self.conn_id)
+
+        hook = conn.get_hook()
+        if not isinstance(hook, DbApiHook):
+            raise AirflowException(
+                f'The connection type is not supported by {self.__class__.__name__}. '
+                f'The associated hook should be a subclass of `DbApiHook`. Got {hook.__class__.__name__}'
+            )
+
+        if self.database:
+            hook.schema = self.database

Review comment:
       I totally agree!  But this is how the current mapping of Airflow works. Currently, the `Connection` object uses the field `schema` to specify the database (not schema) for the database type connection, i.e. Postgres, MySQL, etc. And consequently for now, all `DbApiHook`s use the field schema to specify the database to connect to and use `extra` to specify the schema if applicable. Here is a visual mapping:
   
   ```
   Connection(schema) -> DbApiHook (schema) -> DB API (dbname/database) 
   ```
   
   About the field `database`, it's also used across the codebase to means an actual `database`.
   
   Though I think it will be nice to consolidate these concepts, they should be addressed in a separate PR or worth a discussion in the `dev` list. I can send an email in the dev list if you don't object.




----------------------------------------------------------------
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 #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   Hey @xinbinhuang . I am working on relasing providers and the "maintenance" process of them  - the PR and discussion is in #13767 I hope to get it merged tomorrow/Monday and relase a new wave of providers early next week. I think I'd prefer to add this one afterwards - I would love to take a close look at the provider's backwards - compatibility then and we could work together to test the new provider mechanism (and updates to semver versions of the providers that it brings) with you. 
   
   Woudl that be OK ?


----------------------------------------------------------------
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 #12677: Refactor SQL/BigQuery/Qubole/Druid Check operators

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


   Hey @xinbinhuang . I am working on relasing providers and the "maintenance" process of them  - the PR and discussion is in #13767 I hope to get it merged tomorrow/Monday and relase a new wave of providers early next week. I think I'd prefer to add this one afterwards - I would love to take a close look at the provider's backwards - compatibility then and we could work together to test the new provider mechanism (and updates to semver versions of the providers that it brings) with you. 
   
   Woudl that be OK ?


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