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

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

o-nikolas commented on code in PR #28950:
URL: https://github.com/apache/airflow/pull/28950#discussion_r1121074566


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

Review Comment:
   We usually add test code to setup the resource under test, or at least make it configurable (os env var etc) so that users can setup their own and supply the correct config to test against it.



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

Review Comment:
   What is the purpose of these tags?



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

Review Comment:
   I don't think I fully understand this logic. The two False cases can certainly be collapsed to be more compact, but also, shouldn't the false case be setting `result` rather than returning? If they return then the code to store the version in xcom is not executed. You're basically always comparing `version` with the `prev_version` default value of `-1` from what I can tell. 



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

Review Comment:
   ```suggestion
           if opts:
   ```



##########
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):

Review Comment:
   Can any or all of this be pushed into the hook? Validating things like operators doesn't seem like the right thing



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

Review Comment:
   IMHO all this logic should be inside `get_previous_version()` rather than 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)
+
+    @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:

Review Comment:
   Are we really worried about `context` being missing?
   If so, then just add a statement like:
   ```python
   if not context:
       return False
   ```
   This way the whole main block of code doesn't have to be indented.
   Also if context is really missing you may want to throw an exception instead of just returning False.



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