You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "josh-fell (via GitHub)" <gi...@apache.org> on 2023/03/01 03:38:00 UTC

[GitHub] [airflow] josh-fell commented on a diff in pull request #28950: Sensor for Databricks partition and table changes

josh-fell commented on code in PR #28950:
URL: https://github.com/apache/airflow/pull/28950#discussion_r1121044045


##########
airflow/providers/databricks/sensors/sql.py:
##########
@@ -0,0 +1,144 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+"""This module contains Databricks sensors."""
+
+from __future__ import annotations
+
+from typing import Any, Callable, Sequence
+
+from airflow.providers.common.sql.hooks.sql import fetch_all_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context

Review Comment:
   Since this import is only used for typing it should be put behind `typing.TYPE_CHECKING`. One fewer import at runtime. Applicable to all of the other net-new modules in this PR too.



##########
airflow/providers/databricks/sensors/table_changes.py:
##########
@@ -0,0 +1,152 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+"""This module contains Databricks sensors."""
+
+from __future__ import annotations
+
+from datetime import datetime
+from typing import Sequence
+
+from airflow.exceptions import AirflowException
+from airflow.providers.databricks.sensors.sql import DatabricksSqlSensor
+from airflow.utils.context import Context
+
+
+class DatabricksTableChangesSensor(DatabricksSqlSensor):
+    """Sensor to detect changes in a Delta table.
+
+
+    :param databricks_conn_id: Reference to :ref:`Databricks
+        connection id<howto/connection:databricks>` (templated), defaults to
+        DatabricksSqlHook.default_conn_name
+    :param http_path: Optional string specifying HTTP path of Databricks SQL Endpoint or cluster.
+        If not specified, it should be either specified in the Databricks connection's
+        extra parameters, or ``sql_endpoint_name`` must be specified.
+    :param sql_endpoint_name: Optional name of Databricks SQL Endpoint. If not specified, ``http_path`` must
+        be provided as described above, defaults to None
+    :param session_configuration: An optional dictionary of Spark session parameters. If not specified,
+        it could be specified in the Databricks connection's extra parameters., defaults to None
+    :param http_headers: An optional list of (k, v) pairs
+        that will be set as HTTP headers on every request. (templated).
+    :param catalog: An optional initial catalog to use.
+        Requires DBR version 9.0+ (templated), defaults to ""
+    :param schema: An optional initial schema to use.
+        Requires DBR version 9.0+ (templated), defaults to "default"
+    :param table_name: Table name to generate the SQL query, defaults to ""
+    :param handler: Handler for DbApiHook.run() to return results, defaults to fetch_all_handler
+    :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+    :param timestamp: Timestamp to check event history for a Delta table.
+    :param change_filter_operator: Operator to specify filter condition to check table changes,
+        defaults to >=.
+    """
+
+    template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", "table_name")

Review Comment:
   IMO it would be useful to have `timestamp` as a template field too. I could foresee users wanting to use one of the built-in Jinja variables for this to the task is idempotent (like `{{ data_interval_start }}` for example). Or, be used as a dynamic input from a previous task.



##########
airflow/providers/databricks/sensors/table_changes.py:
##########
@@ -0,0 +1,152 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+"""This module contains Databricks sensors."""
+
+from __future__ import annotations
+
+from datetime import datetime
+from typing import Sequence
+
+from airflow.exceptions import AirflowException
+from airflow.providers.databricks.sensors.sql import DatabricksSqlSensor
+from airflow.utils.context import Context
+
+
+class DatabricksTableChangesSensor(DatabricksSqlSensor):
+    """Sensor to detect changes in a Delta table.
+
+
+    :param databricks_conn_id: Reference to :ref:`Databricks
+        connection id<howto/connection:databricks>` (templated), defaults to
+        DatabricksSqlHook.default_conn_name
+    :param http_path: Optional string specifying HTTP path of Databricks SQL Endpoint or cluster.
+        If not specified, it should be either specified in the Databricks connection's
+        extra parameters, or ``sql_endpoint_name`` must be specified.
+    :param sql_endpoint_name: Optional name of Databricks SQL Endpoint. If not specified, ``http_path`` must
+        be provided as described above, defaults to None
+    :param session_configuration: An optional dictionary of Spark session parameters. If not specified,
+        it could be specified in the Databricks connection's extra parameters., defaults to None
+    :param http_headers: An optional list of (k, v) pairs
+        that will be set as HTTP headers on every request. (templated).
+    :param catalog: An optional initial catalog to use.
+        Requires DBR version 9.0+ (templated), defaults to ""
+    :param schema: An optional initial schema to use.
+        Requires DBR version 9.0+ (templated), defaults to "default"
+    :param table_name: Table name to generate the SQL query, defaults to ""
+    :param handler: Handler for DbApiHook.run() to return results, defaults to fetch_all_handler
+    :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+    :param timestamp: Timestamp to check event history for a Delta table.
+    :param change_filter_operator: Operator to specify filter condition to check table changes,
+        defaults to >=.
+    """
+
+    template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", "table_name")
+
+    def __init__(
+        self,
+        table_name: str,
+        timestamp: datetime | None = None,
+        change_filter_operator: str = ">=",
+        *args,
+        **kwargs,
+    ) -> None:
+        super().__init__(*args, **kwargs)
+        self.timestamp = timestamp
+        self.caller = "DatabricksTableChangesSensor"
+        self.change_filter_operator = change_filter_operator
+        self.table_name = table_name
+
+    def _sql_sensor(self, sql):
+        hook = self._get_hook()
+        sql_result = hook.run(
+            sql,
+            handler=self.handler if self.do_xcom_push else None,
+        )
+        return sql_result
+

Review Comment:
   ```suggestion
   ```
   This method exists in the inherited `DatabricksSqlSensor`. No need to redefine here.



##########
airflow/providers/databricks/sensors/table_changes.py:
##########
@@ -0,0 +1,152 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+"""This module contains Databricks sensors."""
+
+from __future__ import annotations
+
+from datetime import datetime
+from typing import Sequence
+
+from airflow.exceptions import AirflowException
+from airflow.providers.databricks.sensors.sql import DatabricksSqlSensor
+from airflow.utils.context import Context
+
+
+class DatabricksTableChangesSensor(DatabricksSqlSensor):
+    """Sensor to detect changes in a Delta table.
+
+
+    :param databricks_conn_id: Reference to :ref:`Databricks
+        connection id<howto/connection:databricks>` (templated), defaults to
+        DatabricksSqlHook.default_conn_name
+    :param http_path: Optional string specifying HTTP path of Databricks SQL Endpoint or cluster.
+        If not specified, it should be either specified in the Databricks connection's
+        extra parameters, or ``sql_endpoint_name`` must be specified.
+    :param sql_endpoint_name: Optional name of Databricks SQL Endpoint. If not specified, ``http_path`` must
+        be provided as described above, defaults to None
+    :param session_configuration: An optional dictionary of Spark session parameters. If not specified,
+        it could be specified in the Databricks connection's extra parameters., defaults to None
+    :param http_headers: An optional list of (k, v) pairs
+        that will be set as HTTP headers on every request. (templated).
+    :param catalog: An optional initial catalog to use.
+        Requires DBR version 9.0+ (templated), defaults to ""
+    :param schema: An optional initial schema to use.
+        Requires DBR version 9.0+ (templated), defaults to "default"
+    :param table_name: Table name to generate the SQL query, defaults to ""
+    :param handler: Handler for DbApiHook.run() to return results, defaults to fetch_all_handler
+    :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+    :param timestamp: Timestamp to check event history for a Delta table.
+    :param change_filter_operator: Operator to specify filter condition to check table changes,
+        defaults to >=.
+    """
+
+    template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", "table_name")
+
+    def __init__(
+        self,
+        table_name: str,
+        timestamp: datetime | None = None,
+        change_filter_operator: str = ">=",
+        *args,
+        **kwargs,
+    ) -> None:
+        super().__init__(*args, **kwargs)
+        self.timestamp = timestamp
+        self.caller = "DatabricksTableChangesSensor"
+        self.change_filter_operator = change_filter_operator
+        self.table_name = table_name
+
+    def _sql_sensor(self, sql):
+        hook = self._get_hook()
+        sql_result = hook.run(
+            sql,
+            handler=self.handler if self.do_xcom_push else None,
+        )
+        return sql_result
+
+    def _generate_query(
+        self,
+        prefix: str,
+        suffix: str,
+        table_name: str,
+        time_range_filter: str,
+    ) -> str:
+        formatted_opts = f"{prefix} {table_name}{suffix}{time_range_filter}"
+        self.log.debug("Formatted options: %s", formatted_opts)
+
+        return formatted_opts.strip()
+
+    @staticmethod
+    def get_previous_version(context: Context, lookup_key):
+        return context["ti"].xcom_pull(key=lookup_key, include_prior_dates=True)

Review Comment:
   I don't believe there is a guarantee that the most recent XCom will be pulled here. Behind the scenes `XCom.get_many()` is called and just retrieves the first record. At the mercy of the metadatabase being used there.
   
   Also, what happens in a mapped operator situation? If the XCom key is always the same, it seems possible this can pull an XCom key for an entirely different task since `task_ids` and `map_index` is not specified in the `xcom_pull()` call.
   
   Another question then would be what if the input args are the same (i.e. checking for changes in the same table) but a user simply updates the `task_id`. Would this sensor yield a false positive that there was indeed a change?
   
   I don't necessarily have answers to these questions on the top of my head, but some things to think about with using XComs in this way.



##########
tests/providers/databricks/sensors/test_partition.py:
##########
@@ -0,0 +1,59 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+from __future__ import annotations
+
+import unittest
+from datetime import datetime, timedelta
+from unittest import mock
+
+from airflow.providers.common.sql.hooks.sql import fetch_all_handler
+from airflow.providers.databricks.sensors.partition import DatabricksPartitionSensor
+
+TASK_ID = "db-sensor"
+DEFAULT_CONN_ID = "databricks_default"
+DEFAULT_SCHEMA = "schema1"
+DEFAULT_CATALOG = "catalog1"
+DEFAULT_TABLE = "table1"
+DEFAULT_SQL_ENDPOINT = "sql_warehouse_default"
+DEFAULT_PARTITION = {"date": "2023-01-01"}
+
+TIMESTAMP_TEST = datetime.now() - timedelta(days=30)
+
+sql_sensor = DatabricksPartitionSensor(
+    databricks_conn_id=DEFAULT_CONN_ID,
+    sql_endpoint_name=DEFAULT_SQL_ENDPOINT,
+    task_id=TASK_ID,
+    table_name=DEFAULT_TABLE,
+    schema=DEFAULT_SCHEMA,
+    catalog=DEFAULT_CATALOG,
+    partitions=DEFAULT_PARTITION,
+    handler=fetch_all_handler,
+)
+
+
+class TestDatabricksPartitionSensor(unittest.TestCase):

Review Comment:
   There is an ongoing effort to move away from `unittest` in favor of `pytest`. Since these tests are net-new, could you change this and the other tests in the PR to `pytest` please?



##########
airflow/providers/databricks/sensors/partition.py:
##########
@@ -0,0 +1,153 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+"""This module contains Databricks sensors."""
+
+from __future__ import annotations
+
+import datetime
+from typing import Sequence
+
+from databricks.sql.utils import ParamEscaper
+
+from airflow.exceptions import AirflowException
+from airflow.providers.databricks.sensors.sql import DatabricksSqlSensor
+from airflow.utils.context import Context
+
+
+class DatabricksPartitionSensor(DatabricksSqlSensor):
+    """Sensor to detect the existence of partitions in a Delta table.
+
+    :param databricks_conn_id: Reference to :ref:`Databricks
+        connection id<howto/connection:databricks>` (templated).
+    :param http_path: Optional string specifying HTTP path of Databricks SQL Endpoint or cluster.
+        If not specified, it should be either specified in the Databricks connection's extra parameters,
+        or ``sql_endpoint_name`` must be specified.
+    :param sql_endpoint_name: Optional name of Databricks SQL Endpoint.
+        If not specified, ``http_path`` must be provided as described above.
+    :param session_configuration: An optional dictionary of Spark session parameters. If not specified,
+        it could be specified in the Databricks connection's extra parameters.
+    :param http_headers: An optional list of (k, v) pairs that will be set
+        as HTTP headers on every request. (templated)
+    :param catalog: An optional initial catalog to use. Requires DBR version 9.0+ (templated)
+    :param schema: An optional initial schema to use. Requires DBR version 9.0+ (templated)
+    :param table_name: Table name to generate the SQL query.
+    :param partitions: Partitions to check, supplied via a dict.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to fetch_one_handler
+    :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+    :param partition_operator: Comparison operator for partitions.
+    """
+
+    template_fields: Sequence[str] = (
+        "databricks_conn_id",
+        "schema",
+        "http_headers",
+        "catalog",
+        "table_name",
+        "partitions",
+    )
+
+    def __init__(
+        self,
+        table_name: str,
+        partitions: dict,
+        partition_operator: str = "=",
+        *args,
+        **kwargs,
+    ) -> None:
+        super().__init__(*args, **kwargs)
+        self.caller = "DatabricksPartitionSensor"
+        self.partitions = partitions
+        self.partition_operator = partition_operator
+        self.table_name = table_name
+        self.escaper = ParamEscaper()
+
+    def _sql_sensor(self, sql):
+        hook = self._get_hook()
+        sql_result = hook.run(
+            sql,
+            handler=self.handler if self.do_xcom_push else None,
+        )
+        return sql_result
+

Review Comment:
   ```suggestion
   ```
   Same idea here. This method exists in DatabricksSqlSensor.



##########
airflow/providers/databricks/sensors/sql.py:
##########
@@ -0,0 +1,144 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+"""This module contains Databricks sensors."""
+
+from __future__ import annotations
+
+from typing import Any, Callable, Sequence
+
+from airflow.providers.common.sql.hooks.sql import fetch_all_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class DatabricksSqlSensor(BaseSensorOperator):
+    """Generic SQL sensor for Databricks
+
+    :param databricks_conn_id: connection id from Airflow to databricks,
+        defaults to DatabricksSqlHook.default_conn_name
+    :param http_path: Optional string specifying HTTP path of
+        Databricks SQL Endpoint or cluster.If not specified, it should be either specified
+        in the Databricks connection's extra parameters, or ``sql_endpoint_name`` must be specified.
+    :param sql_endpoint_name: Optional name of Databricks SQL Endpoint.
+        If not specified, ``http_path`` must be provided as described above.
+    :param session_configuration: An optional dictionary of Spark session parameters.
+        Defaults to None. If not specified, it could be specified in the
+        Databricks connection's extra parameters.
+    :param http_headers: An optional list of (k, v) pairs that will be set as HTTP headers on every request.
+    :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+    :param sql: SQL query to be executed.
+    :param catalog: Database name/catalog name, defaults to "hive_metastore"
+    :param schema: Schema name, defaults to "default"
+    :param handler: The result handler which is called with the result of each statement.
+    """
+
+    template_fields: Sequence[str] = (
+        "databricks_conn_id",
+        "sql",
+        "catalog",
+        "schema",
+        "http_headers",
+    )
+
+    template_ext: Sequence[str] = (".sql",)
+    template_fields_renderers = {"sql": "sql"}
+
+    def __init__(
+        self,
+        *,
+        databricks_conn_id: str = DatabricksSqlHook.default_conn_name,
+        http_path: str | None = None,
+        sql_endpoint_name: str | None = None,
+        session_configuration=None,
+        http_headers: list[tuple[str, str]] | None = None,
+        catalog: str = "hive_metastore",
+        schema: str = "default",
+        sql: str | None = None,
+        handler: Callable[[Any], Any] = fetch_all_handler,
+        client_parameters: dict[str, Any] | None = None,
+        **kwargs,
+    ) -> None:
+        """Sensor to execute SQL statements on a Delta table via Databricks.
+
+        :param databricks_conn_id: Reference to :ref:`Databricks
+            connection id<howto/connection:databricks>` (templated), defaults to
+            DatabricksSqlHook.default_conn_name
+        :param http_path: Optional string specifying HTTP path of Databricks SQL Endpoint or cluster.
+            If not specified, it should be either specified in the Databricks connection's
+            extra parameters, or ``sql_endpoint_name`` must be specified.
+        :param sql_endpoint_name: Optional name of Databricks SQL Endpoint. If not specified, ``http_path``
+            must be provided as described above, defaults to None
+        :param session_configuration: An optional dictionary of Spark session parameters. If not specified,
+            it could be specified in the Databricks connection's extra parameters., defaults to None
+        :param http_headers: An optional list of (k, v) pairs
+            that will be set as HTTP headers on every request. (templated).
+        :param catalog: An optional initial catalog to use.
+            Requires DBR version 9.0+ (templated), defaults to ""
+        :param schema: An optional initial schema to use.
+            Requires DBR version 9.0+ (templated), defaults to "default"
+        :param sql: SQL statement to be executed.
+        :param handler: Handler for DbApiHook.run() to return results, defaults to fetch_all_handler
+        :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+        """

Review Comment:
   There are two sets of docstrings for this sensor's construction. Can you consolidate please?



##########
airflow/providers/databricks/sensors/sql.py:
##########
@@ -0,0 +1,144 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+"""This module contains Databricks sensors."""
+
+from __future__ import annotations
+
+from typing import Any, Callable, Sequence
+
+from airflow.providers.common.sql.hooks.sql import fetch_all_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.sensors.base import BaseSensorOperator
+from airflow.utils.context import Context
+
+
+class DatabricksSqlSensor(BaseSensorOperator):
+    """Generic SQL sensor for Databricks
+
+    :param databricks_conn_id: connection id from Airflow to databricks,
+        defaults to DatabricksSqlHook.default_conn_name
+    :param http_path: Optional string specifying HTTP path of
+        Databricks SQL Endpoint or cluster.If not specified, it should be either specified
+        in the Databricks connection's extra parameters, or ``sql_endpoint_name`` must be specified.
+    :param sql_endpoint_name: Optional name of Databricks SQL Endpoint.
+        If not specified, ``http_path`` must be provided as described above.
+    :param session_configuration: An optional dictionary of Spark session parameters.
+        Defaults to None. If not specified, it could be specified in the
+        Databricks connection's extra parameters.
+    :param http_headers: An optional list of (k, v) pairs that will be set as HTTP headers on every request.
+    :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+    :param sql: SQL query to be executed.
+    :param catalog: Database name/catalog name, defaults to "hive_metastore"
+    :param schema: Schema name, defaults to "default"
+    :param handler: The result handler which is called with the result of each statement.
+    """
+
+    template_fields: Sequence[str] = (
+        "databricks_conn_id",
+        "sql",
+        "catalog",
+        "schema",
+        "http_headers",
+    )
+
+    template_ext: Sequence[str] = (".sql",)
+    template_fields_renderers = {"sql": "sql"}
+
+    def __init__(
+        self,
+        *,
+        databricks_conn_id: str = DatabricksSqlHook.default_conn_name,
+        http_path: str | None = None,
+        sql_endpoint_name: str | None = None,
+        session_configuration=None,
+        http_headers: list[tuple[str, str]] | None = None,
+        catalog: str = "hive_metastore",
+        schema: str = "default",
+        sql: str | None = None,
+        handler: Callable[[Any], Any] = fetch_all_handler,
+        client_parameters: dict[str, Any] | None = None,
+        **kwargs,
+    ) -> None:
+        """Sensor to execute SQL statements on a Delta table via Databricks.
+
+        :param databricks_conn_id: Reference to :ref:`Databricks
+            connection id<howto/connection:databricks>` (templated), defaults to
+            DatabricksSqlHook.default_conn_name
+        :param http_path: Optional string specifying HTTP path of Databricks SQL Endpoint or cluster.
+            If not specified, it should be either specified in the Databricks connection's
+            extra parameters, or ``sql_endpoint_name`` must be specified.
+        :param sql_endpoint_name: Optional name of Databricks SQL Endpoint. If not specified, ``http_path``
+            must be provided as described above, defaults to None
+        :param session_configuration: An optional dictionary of Spark session parameters. If not specified,
+            it could be specified in the Databricks connection's extra parameters., defaults to None
+        :param http_headers: An optional list of (k, v) pairs
+            that will be set as HTTP headers on every request. (templated).
+        :param catalog: An optional initial catalog to use.
+            Requires DBR version 9.0+ (templated), defaults to ""
+        :param schema: An optional initial schema to use.
+            Requires DBR version 9.0+ (templated), defaults to "default"
+        :param sql: SQL statement to be executed.
+        :param handler: Handler for DbApiHook.run() to return results, defaults to fetch_all_handler
+        :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+        """
+        super().__init__(**kwargs)
+        self.databricks_conn_id = databricks_conn_id
+        self._http_path = http_path
+        self._sql_endpoint_name = sql_endpoint_name
+        self.session_config = session_configuration
+        self.http_headers = http_headers
+        self.catalog = catalog
+        self.schema = schema
+        self.sql = sql
+        self.caller = "DatabricksSqlSensor"
+        self.client_parameters = client_parameters or {}
+        self.hook_params = kwargs.pop("hook_params", {})
+        self.handler = handler
+
+    def _get_hook(self) -> DatabricksSqlHook:
+        return DatabricksSqlHook(
+            self.databricks_conn_id,
+            self._http_path,
+            self._sql_endpoint_name,
+            self.session_config,
+            self.http_headers,
+            self.catalog,
+            self.schema,
+            self.caller,
+            **self.client_parameters,
+            **self.hook_params,
+        )
+
+    def _sql_sensor(self, sql):
+        hook = self._get_hook()
+        sql_result = hook.run(
+            sql,
+            handler=self.handler if self.do_xcom_push else None,
+        )
+        return sql_result
+
+    def _get_results(self) -> bool:
+        result = self._sql_sensor(self.sql)
+        self.log.debug("SQL result: %s", result)
+        if len(result) < 1:
+            return False
+        return True

Review Comment:
   ```suggestion
           return bool(result)
   ```
   Small optimization.



##########
airflow/providers/databricks/sensors/partition.py:
##########
@@ -0,0 +1,153 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+"""This module contains Databricks sensors."""
+
+from __future__ import annotations
+
+import datetime
+from typing import Sequence
+
+from databricks.sql.utils import ParamEscaper
+
+from airflow.exceptions import AirflowException
+from airflow.providers.databricks.sensors.sql import DatabricksSqlSensor
+from airflow.utils.context import Context
+
+
+class DatabricksPartitionSensor(DatabricksSqlSensor):
+    """Sensor to detect the existence of partitions in a Delta table.
+
+    :param databricks_conn_id: Reference to :ref:`Databricks
+        connection id<howto/connection:databricks>` (templated).
+    :param http_path: Optional string specifying HTTP path of Databricks SQL Endpoint or cluster.
+        If not specified, it should be either specified in the Databricks connection's extra parameters,
+        or ``sql_endpoint_name`` must be specified.
+    :param sql_endpoint_name: Optional name of Databricks SQL Endpoint.
+        If not specified, ``http_path`` must be provided as described above.
+    :param session_configuration: An optional dictionary of Spark session parameters. If not specified,
+        it could be specified in the Databricks connection's extra parameters.
+    :param http_headers: An optional list of (k, v) pairs that will be set
+        as HTTP headers on every request. (templated)
+    :param catalog: An optional initial catalog to use. Requires DBR version 9.0+ (templated)
+    :param schema: An optional initial schema to use. Requires DBR version 9.0+ (templated)
+    :param table_name: Table name to generate the SQL query.
+    :param partitions: Partitions to check, supplied via a dict.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to fetch_one_handler
+    :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+    :param partition_operator: Comparison operator for partitions.
+    """
+
+    template_fields: Sequence[str] = (
+        "databricks_conn_id",
+        "schema",
+        "http_headers",
+        "catalog",
+        "table_name",
+        "partitions",
+    )
+
+    def __init__(
+        self,
+        table_name: str,
+        partitions: dict,
+        partition_operator: str = "=",
+        *args,
+        **kwargs,
+    ) -> None:
+        super().__init__(*args, **kwargs)
+        self.caller = "DatabricksPartitionSensor"
+        self.partitions = partitions
+        self.partition_operator = partition_operator
+        self.table_name = table_name
+        self.escaper = ParamEscaper()
+
+    def _sql_sensor(self, sql):
+        hook = self._get_hook()
+        sql_result = hook.run(
+            sql,
+            handler=self.handler if self.do_xcom_push else None,
+        )
+        return sql_result
+
+    def _generate_partition_query(
+        self,
+        prefix: str,
+        suffix: str,
+        joiner_val: str,
+        table_name: str,
+        opts: dict[str, str] | None = None,
+        escape_key: bool = False,
+    ) -> str:
+        partition_columns = self._sql_sensor(f"DESCRIBE DETAIL {table_name}")[0][7]
+        self.log.info("table_info: %s", partition_columns)
+        if len(partition_columns) < 1:
+            raise AirflowException("Table %s does not have partitions", table_name)
+        formatted_opts = ""
+        if opts is not None and len(opts) > 0:
+            output_list = []
+            for partition_col, partition_value in self.partitions.items():
+                if escape_key:
+                    partition_col = self.escaper.escape_item(partition_col)
+                if partition_col in partition_columns:
+                    if isinstance(partition_value, list):
+                        output_list.append(f"""{partition_col} in {tuple(partition_value)}""")
+                    if isinstance(partition_value, (int, float, complex)):
+                        output_list.append(
+                            f"""{partition_col}{self.partition_operator}{self.escaper.escape_item(partition_value)}"""
+                        )
+                    if isinstance(partition_value, (str, datetime.datetime)):
+                        output_list.append(
+                            f"""{partition_col}{self.partition_operator}{self.escaper.escape_item(partition_value)}"""
+                        )
+                    # TODO: Check date types.
+                else:
+                    raise AirflowException(
+                        "Column %s not part of table partitions: %s", partition_col, partition_columns
+                    )
+        self.log.debug("Formatted options: %s", formatted_opts)
+        formatted_opts = f"{prefix} {joiner_val.join(output_list)} {suffix}"
+
+        return formatted_opts.strip()
+
+    def _check_table_partitions(self) -> list:
+        _fully_qualified_table_name = str(self.catalog + "." + self.schema + "." + self.table_name)
+        self.log.debug("Table name generated from arguments: %s", _fully_qualified_table_name)
+        _joiner_val = " AND "
+        _prefix = f"SELECT 1 FROM {_fully_qualified_table_name} WHERE"
+        _suffix = " LIMIT 1"
+
+        partition_sql = self._generate_partition_query(
+            prefix=_prefix,
+            suffix=_suffix,
+            joiner_val=_joiner_val,
+            opts=self.partitions,
+            table_name=_fully_qualified_table_name,
+            escape_key=False,
+        )
+        return self._sql_sensor(partition_sql)
+
+    def _get_results(self) -> bool:
+        result = self._check_table_partitions()
+        self.log.debug("Partition sensor result: %s", result)
+        if len(result) < 1:
+            return False
+        return True
+
+    def poke(self, context: Context) -> bool:
+        return self._get_results()

Review Comment:
   _Technically_ could remove this too.



##########
tests/system/providers/databricks/example_databricks_sensors.py:
##########
@@ -0,0 +1,111 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+"""
+This is an example DAG which uses the DatabricksSqlSensor,
+DatabricksPartitionSensor and DatabricksTableChangesSensor.
+The first task checks for a generic SQL statement against a Delta table,
+and if a result is returned, the task succeeds, else it times out.
+The second task checks for the specified partitions' presence in a
+Delta table. If it exists, the task succeeds, else it times out.
+The third task checks for data related changes in a Delta
+table via versions. If the version retrieved from hive metastore
+is different from the version stored in Airflow metadata, the DAG
+succeeds, else it times out.
+"""
+from __future__ import annotations
+
+import os
+from datetime import datetime, timedelta
+
+from airflow import DAG
+from airflow.providers.databricks.sensors.partition import DatabricksPartitionSensor
+from airflow.providers.databricks.sensors.sql import DatabricksSqlSensor
+from airflow.providers.databricks.sensors.table_changes import DatabricksTableChangesSensor
+
+# [docs]
+ENV_ID = os.environ.get("SYSTEM_TESTS_ENV_ID")
+# [docs]
+DAG_ID = "example_databricks_sensor"
+
+with DAG(
+    dag_id=DAG_ID,
+    schedule="@daily",
+    start_date=datetime(2021, 1, 1),
+    tags=["example"],
+    catchup=False,
+) as dag:
+    # [docs]
+    connection_id = "databricks_default"
+    sql_endpoint_name = "Starter Warehouse"
+
+    # [START howto_sensor_databricks_sql]

Review Comment:
   These `START/END` markers are used to include code snippets in guides (generally). It would be great if there was accompanying documentation for these new sensors and ones that take advantage of the snippets being outlined in this DAG. There are a lot of examples in operator guides on how this is done throughout the providers.



##########
airflow/providers/databricks/sensors/table_changes.py:
##########
@@ -0,0 +1,152 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+#
+"""This module contains Databricks sensors."""
+
+from __future__ import annotations
+
+from datetime import datetime
+from typing import Sequence
+
+from airflow.exceptions import AirflowException
+from airflow.providers.databricks.sensors.sql import DatabricksSqlSensor
+from airflow.utils.context import Context
+
+
+class DatabricksTableChangesSensor(DatabricksSqlSensor):
+    """Sensor to detect changes in a Delta table.
+
+
+    :param databricks_conn_id: Reference to :ref:`Databricks
+        connection id<howto/connection:databricks>` (templated), defaults to
+        DatabricksSqlHook.default_conn_name
+    :param http_path: Optional string specifying HTTP path of Databricks SQL Endpoint or cluster.
+        If not specified, it should be either specified in the Databricks connection's
+        extra parameters, or ``sql_endpoint_name`` must be specified.
+    :param sql_endpoint_name: Optional name of Databricks SQL Endpoint. If not specified, ``http_path`` must
+        be provided as described above, defaults to None
+    :param session_configuration: An optional dictionary of Spark session parameters. If not specified,
+        it could be specified in the Databricks connection's extra parameters., defaults to None
+    :param http_headers: An optional list of (k, v) pairs
+        that will be set as HTTP headers on every request. (templated).
+    :param catalog: An optional initial catalog to use.
+        Requires DBR version 9.0+ (templated), defaults to ""
+    :param schema: An optional initial schema to use.
+        Requires DBR version 9.0+ (templated), defaults to "default"
+    :param table_name: Table name to generate the SQL query, defaults to ""
+    :param handler: Handler for DbApiHook.run() to return results, defaults to fetch_all_handler
+    :param client_parameters: Additional parameters internal to Databricks SQL Connector parameters.
+    :param timestamp: Timestamp to check event history for a Delta table.
+    :param change_filter_operator: Operator to specify filter condition to check table changes,
+        defaults to >=.
+    """
+
+    template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", "table_name")
+
+    def __init__(
+        self,
+        table_name: str,
+        timestamp: datetime | None = None,
+        change_filter_operator: str = ">=",
+        *args,
+        **kwargs,
+    ) -> None:
+        super().__init__(*args, **kwargs)
+        self.timestamp = timestamp
+        self.caller = "DatabricksTableChangesSensor"
+        self.change_filter_operator = change_filter_operator
+        self.table_name = table_name
+
+    def _sql_sensor(self, sql):
+        hook = self._get_hook()
+        sql_result = hook.run(
+            sql,
+            handler=self.handler if self.do_xcom_push else None,
+        )
+        return sql_result
+
+    def _generate_query(
+        self,
+        prefix: str,
+        suffix: str,
+        table_name: str,
+        time_range_filter: str,
+    ) -> str:
+        formatted_opts = f"{prefix} {table_name}{suffix}{time_range_filter}"
+        self.log.debug("Formatted options: %s", formatted_opts)
+
+        return formatted_opts.strip()
+
+    @staticmethod
+    def get_previous_version(context: Context, lookup_key):
+        return context["ti"].xcom_pull(key=lookup_key, include_prior_dates=True)
+
+    @staticmethod
+    def set_version(context: Context, lookup_key, version):
+        context["ti"].xcom_push(key=lookup_key, value=version)
+
+    def get_current_table_version(self, table_name):
+        _prefix = "SELECT MAX(version) AS versions FROM (DESCRIBE HISTORY"
+        _data_operations_filter = ") WHERE operation NOT IN ('CONVERT', 'OPTIMIZE', 'CLONE', \
+        'RESTORE', 'FSCK') AND operation NOT LIKE 'VACUUM%'"
+        if self.timestamp is not None:
+            if self.change_filter_operator not in ("=", ">", "<", ">=", "<=", "!="):
+                raise AirflowException("Invalid comparison operator specified for time range filter.")
+            _timestamp_literal = f" AND timestamp {self.change_filter_operator} '{self.timestamp}'"
+        else:
+            _timestamp_literal = ""
+        query = self._generate_query(
+            prefix=_prefix,
+            suffix=_data_operations_filter,
+            time_range_filter=_timestamp_literal,
+            table_name=table_name,
+        )
+        self.log.debug("Query to be executed: %s", query)
+        result = self._sql_sensor(query)[0][0]
+        self.log.debug("Query result: %s", result)
+        return result
+
+    def _get_results_table_changes(self, context) -> bool:
+        complete_table_name = str(self.catalog + "." + self.schema + "." + self.table_name)
+        self.log.debug("Table name generated from arguments: %s", complete_table_name)
+
+        prev_version = -1
+        if context is not None:
+            lookup_key = complete_table_name
+            prev_data = self.get_previous_version(lookup_key=lookup_key, context=context)
+            self.log.debug("prev_data: %s, type=%s", str(prev_data), type(prev_data))
+            if isinstance(prev_data, int):
+                prev_version = prev_data
+            elif prev_data is not None:
+                raise AirflowException("Incorrect type for previous XCom data: %s", type(prev_data))
+            version = self.get_current_table_version(table_name=complete_table_name)
+            self.log.debug("Current version: %s", version)
+            if version is None:
+                return False
+            if prev_version < version:
+                result = True
+            else:
+                return False
+            if prev_version != version:
+                self.set_version(lookup_key=lookup_key, version=version, context=context)
+            self.log.debug("Result: %s", result)
+            return result
+        return False
+
+    def poke(self, context: Context) -> bool:
+        return self._get_results_table_changes(context=context)

Review Comment:
   +1. Some of these functions look like they could be handy if made generally available as part of a hook.



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

To unsubscribe, e-mail: commits-unsubscribe@airflow.apache.org

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