alexott commented on code in PR #28950:
URL: https://github.com/apache/airflow/pull/28950#discussion_r1089952944


##########
airflow/providers/databricks/sensors/databricks_partition.py:
##########
@@ -0,0 +1,157 @@
+#
+# 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.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_one_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_sql import 
DatabricksSqlSensor
+from airflow.utils.context import Context
+
+template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", 
"table_name", "partition_name")
+
+
+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 partition_name: Partition to check.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to 
fetch_one_handler
+    :param caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :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",
+        "partition_name",
+    )
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        partition_name: dict,
+        handler: Callable[[Any], Any] = fetch_one_handler,
+        caller: str = "DatabricksPartitionSensor",

Review Comment:
   let hardcode `caller` instead of passing it as an argument.



##########
airflow/providers/databricks/provider.yaml:
##########
@@ -92,6 +92,16 @@ hooks:
     python-modules:
       - airflow.providers.databricks.hooks.databricks_sql
 
+sensors:
+  - integration-name: Databricks SQL
+    python-modules:
+      - airflow.providers.databricks.sensors.databricks_sql
+  - integration-name: Databricks Partition
+    python-modules:
+      - airflow.providers.databricks.sensors.databricks_partition
+  - integration-name: Databricks Table Changes
+    python-modules:
+      - airflow.providers.databricks.sensors.databricks_table_changes

Review Comment:
   We can combine all of them under the `Databricks SQL` umbrella



##########
airflow/providers/databricks/sensors/databricks_partition.py:
##########
@@ -0,0 +1,157 @@
+#
+# 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.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_one_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_sql import 
DatabricksSqlSensor
+from airflow.utils.context import Context
+
+template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", 
"table_name", "partition_name")
+
+
+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 partition_name: Partition to check.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to 
fetch_one_handler
+    :param caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :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",
+        "partition_name",
+    )
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        partition_name: dict,
+        handler: Callable[[Any], Any] = fetch_one_handler,
+        caller: str = "DatabricksPartitionSensor",
+        client_parameters: dict[str, Any] | None = None,
+        partition_operator: str = "=",
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.partition_name = partition_name
+        self.caller = caller
+        self.client_parameters = client_parameters or {}
+        self.hook_params = kwargs.pop("hook_params", {})
+        self.handler = handler
+        self.partition_operator = partition_operator
+
+    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 _check_table_partitions(self) -> list:
+        if self.catalog is not None:
+            complete_table_name = str(self.catalog + "." + self.schema + "." + 
self.table_name)
+            self.log.debug("Table name generated from arguments: %s", 
complete_table_name)
+        else:
+            raise AirflowException("Catalog name not specified, aborting query 
execution.")
+        partition_columns = self._sql_sensor(f"describe detail 
{complete_table_name}")[7]
+        self.log.debug("table_info: %s", partition_columns)
+        if len(partition_columns) < 1:
+            raise AirflowException("Table %s does not have partitions", 
complete_table_name)
+        partitions_list = []
+        for partition_col, partition_value in self.partition_name.items():
+            if partition_col in partition_columns:
+                if isinstance(partition_value, list):
+                    partitions_list.append(f"""{partition_col} in 
{tuple(partition_value)}""")
+                if isinstance(partition_value, (int, float, complex)):
+                    
partitions_list.append(f"""{partition_col}{self.partition_operator}{partition_value}""")
+                if isinstance(partition_value, str):
+                    partitions_list.append(
+                        
f"""{partition_col}{self.partition_operator}\"{partition_value}\""""
+                    )
+            else:
+                raise AirflowException(
+                    "Column %s not part of table partitions: %s", 
partition_col, partition_columns
+                )
+        partitions = " AND ".join(partitions_list)
+        partition_sql = f"SELECT 1 FROM {complete_table_name} WHERE 
{partitions}"

Review Comment:
   we should add `LIMIT 1` to avoid selecting too much from the table...



##########
airflow/providers/databricks/sensors/databricks_partition.py:
##########
@@ -0,0 +1,157 @@
+#
+# 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.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_one_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_sql import 
DatabricksSqlSensor
+from airflow.utils.context import Context
+
+template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", 
"table_name", "partition_name")
+
+
+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 partition_name: Partition to check.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to 
fetch_one_handler
+    :param caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :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",
+        "partition_name",
+    )
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        partition_name: dict,
+        handler: Callable[[Any], Any] = fetch_one_handler,
+        caller: str = "DatabricksPartitionSensor",
+        client_parameters: dict[str, Any] | None = None,
+        partition_operator: str = "=",
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.partition_name = partition_name
+        self.caller = caller
+        self.client_parameters = client_parameters or {}
+        self.hook_params = kwargs.pop("hook_params", {})
+        self.handler = handler
+        self.partition_operator = partition_operator
+
+    def _get_hook(self) -> DatabricksSqlHook:

Review Comment:
   If we pass all parameters to the `DatabricksSqlSensor`, then we can simply 
inherit that hook from it.



##########
airflow/providers/databricks/sensors/databricks_table_changes.py:
##########
@@ -0,0 +1,164 @@
+#
+# 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, timedelta
+from typing import Any, Callable, Sequence
+
+from airflow.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_all_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_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 caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :param client_parameters: Additional parameters internal to Databricks SQL 
Connector parameters.
+    :param timestamp: Timestamp to check event history for a Delta table,
+        defaults to datetime.now()-timedelta(days=7)
+    """
+
+    template_fields: Sequence[str] = ("databricks_conn_id", "catalog", 
"schema", "table_name", "timestamp")
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        handler: Callable[[Any], Any] = fetch_all_handler,
+        timestamp: datetime = datetime.now() - timedelta(days=7),
+        caller: str = "DatabricksTableChangesSensor",
+        client_parameters: dict[str, Any] | None = None,

Review Comment:
   Same comments as for partition sensor



##########
airflow/providers/databricks/sensors/databricks_table_changes.py:
##########
@@ -0,0 +1,164 @@
+#
+# 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, timedelta
+from typing import Any, Callable, Sequence
+
+from airflow.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_all_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_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 caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :param client_parameters: Additional parameters internal to Databricks SQL 
Connector parameters.
+    :param timestamp: Timestamp to check event history for a Delta table,
+        defaults to datetime.now()-timedelta(days=7)
+    """
+
+    template_fields: Sequence[str] = ("databricks_conn_id", "catalog", 
"schema", "table_name", "timestamp")
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        handler: Callable[[Any], Any] = fetch_all_handler,
+        timestamp: datetime = datetime.now() - timedelta(days=7),
+        caller: str = "DatabricksTableChangesSensor",
+        client_parameters: dict[str, Any] | None = None,
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.timestamp = timestamp
+        self.caller = caller
+        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
+
+    @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, time_range):
+        change_sql = (
+            f"SELECT COUNT(version) as versions from "

Review Comment:
   what about selecting `max(version)` instead of count?



##########
airflow/providers/databricks/sensors/databricks_partition.py:
##########
@@ -0,0 +1,157 @@
+#
+# 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.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_one_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_sql import 
DatabricksSqlSensor
+from airflow.utils.context import Context
+
+template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", 
"table_name", "partition_name")
+
+
+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 partition_name: Partition to check.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to 
fetch_one_handler
+    :param caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :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",
+        "partition_name",
+    )
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        partition_name: dict,
+        handler: Callable[[Any], Any] = fetch_one_handler,
+        caller: str = "DatabricksPartitionSensor",
+        client_parameters: dict[str, Any] | None = None,
+        partition_operator: str = "=",
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.partition_name = partition_name
+        self.caller = caller
+        self.client_parameters = client_parameters or {}
+        self.hook_params = kwargs.pop("hook_params", {})
+        self.handler = handler
+        self.partition_operator = partition_operator
+
+    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 _check_table_partitions(self) -> list:
+        if self.catalog is not None:

Review Comment:
   it will be always executed this way because we're passing empty string as 
default. Also, we can continue to rely on two-level and even one level naming, 
relying on default catalog & default schema...



##########
airflow/providers/databricks/sensors/databricks_partition.py:
##########
@@ -0,0 +1,157 @@
+#
+# 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.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_one_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_sql import 
DatabricksSqlSensor
+from airflow.utils.context import Context
+
+template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", 
"table_name", "partition_name")
+
+
+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 partition_name: Partition to check.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to 
fetch_one_handler
+    :param caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :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",
+        "partition_name",
+    )
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        partition_name: dict,

Review Comment:
   `partition_name` isn't intuitively understandable. I suggest to rename it to 
something like `partitions` or `partition_spec`...



##########
airflow/providers/databricks/sensors/databricks_partition.py:
##########
@@ -0,0 +1,157 @@
+#
+# 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.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_one_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_sql import 
DatabricksSqlSensor
+from airflow.utils.context import Context
+
+template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", 
"table_name", "partition_name")
+
+
+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 partition_name: Partition to check.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to 
fetch_one_handler
+    :param caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :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",
+        "partition_name",
+    )
+
+    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 = "",
+        schema: str = "default",

Review Comment:
   I would prefer not to hardcode default schema but relying on the 
cluster/warehouse defaults.



##########
airflow/providers/databricks/sensors/databricks_partition.py:
##########
@@ -0,0 +1,157 @@
+#
+# 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.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_one_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_sql import 
DatabricksSqlSensor
+from airflow.utils.context import Context
+
+template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", 
"table_name", "partition_name")
+
+
+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 partition_name: Partition to check.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to 
fetch_one_handler
+    :param caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :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",
+        "partition_name",
+    )
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",

Review Comment:
   Can we inherit most of the parameters from `DatabricksSqlSensor`?



##########
airflow/providers/databricks/sensors/databricks_partition.py:
##########
@@ -0,0 +1,157 @@
+#
+# 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.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_one_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_sql import 
DatabricksSqlSensor
+from airflow.utils.context import Context
+
+template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", 
"table_name", "partition_name")
+
+
+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 partition_name: Partition to check.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to 
fetch_one_handler
+    :param caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :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",
+        "partition_name",
+    )
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        partition_name: dict,
+        handler: Callable[[Any], Any] = fetch_one_handler,
+        caller: str = "DatabricksPartitionSensor",
+        client_parameters: dict[str, Any] | None = None,
+        partition_operator: str = "=",
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.partition_name = partition_name
+        self.caller = caller
+        self.client_parameters = client_parameters or {}
+        self.hook_params = kwargs.pop("hook_params", {})
+        self.handler = handler
+        self.partition_operator = partition_operator
+
+    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 _check_table_partitions(self) -> list:
+        if self.catalog is not None:
+            complete_table_name = str(self.catalog + "." + self.schema + "." + 
self.table_name)
+            self.log.debug("Table name generated from arguments: %s", 
complete_table_name)
+        else:
+            raise AirflowException("Catalog name not specified, aborting query 
execution.")
+        partition_columns = self._sql_sensor(f"describe detail 
{complete_table_name}")[7]
+        self.log.debug("table_info: %s", partition_columns)
+        if len(partition_columns) < 1:
+            raise AirflowException("Table %s does not have partitions", 
complete_table_name)
+        partitions_list = []
+        for partition_col, partition_value in self.partition_name.items():
+            if partition_col in partition_columns:
+                if isinstance(partition_value, list):
+                    partitions_list.append(f"""{partition_col} in 
{tuple(partition_value)}""")
+                if isinstance(partition_value, (int, float, complex)):
+                    
partitions_list.append(f"""{partition_col}{self.partition_operator}{partition_value}""")
+                if isinstance(partition_value, str):
+                    partitions_list.append(
+                        
f"""{partition_col}{self.partition_operator}\"{partition_value}\""""
+                    )

Review Comment:
   it's better to use formatting functions from sql connector, similar to what 
is done in the `DatabricksCopyIntoOperator` - otherwise we have a chance of 
getting SQL injection.



##########
airflow/providers/databricks/sensors/databricks_partition.py:
##########
@@ -0,0 +1,157 @@
+#
+# 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.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_one_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_sql import 
DatabricksSqlSensor
+from airflow.utils.context import Context
+
+template_fields: Sequence[str] = ("databricks_conn_id", "catalog", "schema", 
"table_name", "partition_name")
+
+
+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 partition_name: Partition to check.
+    :param handler: Handler for DbApiHook.run() to return results, defaults to 
fetch_one_handler
+    :param caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :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",
+        "partition_name",
+    )
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        partition_name: dict,
+        handler: Callable[[Any], Any] = fetch_one_handler,
+        caller: str = "DatabricksPartitionSensor",
+        client_parameters: dict[str, Any] | None = None,
+        partition_operator: str = "=",
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.partition_name = partition_name
+        self.caller = caller
+        self.client_parameters = client_parameters or {}
+        self.hook_params = kwargs.pop("hook_params", {})
+        self.handler = handler
+        self.partition_operator = partition_operator
+
+    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 _check_table_partitions(self) -> list:
+        if self.catalog is not None:
+            complete_table_name = str(self.catalog + "." + self.schema + "." + 
self.table_name)
+            self.log.debug("Table name generated from arguments: %s", 
complete_table_name)
+        else:
+            raise AirflowException("Catalog name not specified, aborting query 
execution.")
+        partition_columns = self._sql_sensor(f"describe detail 
{complete_table_name}")[7]
+        self.log.debug("table_info: %s", partition_columns)
+        if len(partition_columns) < 1:
+            raise AirflowException("Table %s does not have partitions", 
complete_table_name)
+        partitions_list = []
+        for partition_col, partition_value in self.partition_name.items():
+            if partition_col in partition_columns:
+                if isinstance(partition_value, list):
+                    partitions_list.append(f"""{partition_col} in 
{tuple(partition_value)}""")
+                if isinstance(partition_value, (int, float, complex)):
+                    
partitions_list.append(f"""{partition_col}{self.partition_operator}{partition_value}""")
+                if isinstance(partition_value, str):
+                    partitions_list.append(
+                        
f"""{partition_col}{self.partition_operator}\"{partition_value}\""""
+                    )
+            else:
+                raise AirflowException(
+                    "Column %s not part of table partitions: %s", 
partition_col, partition_columns
+                )
+        partitions = " AND ".join(partitions_list)
+        partition_sql = f"SELECT 1 FROM {complete_table_name} WHERE 
{partitions}"
+        return self._sql_sensor(partition_sql)
+
+    def poke(self, context: Context) -> bool:
+        result = self._check_table_partitions()
+        self.log.debug("Partition sensor result: %s", result)
+        if len(result) < 1:
+            raise AirflowException("Databricks SQL partition sensor failed.")
+        return True

Review Comment:
   We can use the same function defined in the parent, but simply have an 
overridable function something like, `_get_results` that will implement the 
logic.



##########
airflow/providers/databricks/sensors/databricks_table_changes.py:
##########
@@ -0,0 +1,164 @@
+#
+# 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, timedelta
+from typing import Any, Callable, Sequence
+
+from airflow.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_all_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_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 caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :param client_parameters: Additional parameters internal to Databricks SQL 
Connector parameters.
+    :param timestamp: Timestamp to check event history for a Delta table,
+        defaults to datetime.now()-timedelta(days=7)
+    """
+
+    template_fields: Sequence[str] = ("databricks_conn_id", "catalog", 
"schema", "table_name", "timestamp")
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        handler: Callable[[Any], Any] = fetch_all_handler,
+        timestamp: datetime = datetime.now() - timedelta(days=7),
+        caller: str = "DatabricksTableChangesSensor",
+        client_parameters: dict[str, Any] | None = None,
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.timestamp = timestamp
+        self.caller = caller
+        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
+
+    @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, time_range):
+        change_sql = (
+            f"SELECT COUNT(version) as versions from "
+            f"(DESCRIBE HISTORY {table_name}) "
+            f"WHERE timestamp >= '{time_range}'"

Review Comment:
   if time_range doesn't change between calls, we may return true all the time. 
Why not compare with the latest version instead?



##########
airflow/providers/databricks/sensors/databricks_table_changes.py:
##########
@@ -0,0 +1,164 @@
+#
+# 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, timedelta
+from typing import Any, Callable, Sequence
+
+from airflow.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_all_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_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 caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :param client_parameters: Additional parameters internal to Databricks SQL 
Connector parameters.
+    :param timestamp: Timestamp to check event history for a Delta table,
+        defaults to datetime.now()-timedelta(days=7)
+    """
+
+    template_fields: Sequence[str] = ("databricks_conn_id", "catalog", 
"schema", "table_name", "timestamp")
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        handler: Callable[[Any], Any] = fetch_all_handler,
+        timestamp: datetime = datetime.now() - timedelta(days=7),
+        caller: str = "DatabricksTableChangesSensor",
+        client_parameters: dict[str, Any] | None = None,
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.timestamp = timestamp
+        self.caller = caller
+        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
+
+    @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, time_range):
+        change_sql = (
+            f"SELECT COUNT(version) as versions from "
+            f"(DESCRIBE HISTORY {table_name}) "
+            f"WHERE timestamp >= '{time_range}'"
+        )
+        result = self._sql_sensor(change_sql)[0][0]
+        return result
+
+    def _check_table_changes(self, context: Context) -> bool:
+        if self.catalog is not None:
+            complete_table_name = str(self.catalog + "." + self.schema + "." + 
self.table_name)
+            self.log.debug("Table name generated from arguments: %s", 
complete_table_name)
+        else:
+            raise AirflowException("Catalog name not specified, aborting query 
execution.")

Review Comment:
   Same comment as for partition sensor.



##########
airflow/providers/databricks/sensors/databricks_table_changes.py:
##########
@@ -0,0 +1,164 @@
+#
+# 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, timedelta
+from typing import Any, Callable, Sequence
+
+from airflow.exceptions import AirflowException
+from airflow.providers.common.sql.hooks.sql import fetch_all_handler
+from airflow.providers.databricks.hooks.databricks_sql import DatabricksSqlHook
+from airflow.providers.databricks.sensors.databricks_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 caller: String passed to name a hook to Databricks, defaults to 
"DatabricksPartitionSensor"
+    :param client_parameters: Additional parameters internal to Databricks SQL 
Connector parameters.
+    :param timestamp: Timestamp to check event history for a Delta table,
+        defaults to datetime.now()-timedelta(days=7)
+    """
+
+    template_fields: Sequence[str] = ("databricks_conn_id", "catalog", 
"schema", "table_name", "timestamp")
+
+    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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        handler: Callable[[Any], Any] = fetch_all_handler,
+        timestamp: datetime = datetime.now() - timedelta(days=7),
+        caller: str = "DatabricksTableChangesSensor",
+        client_parameters: dict[str, Any] | None = None,
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.timestamp = timestamp
+        self.caller = caller
+        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
+
+    @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, time_range):
+        change_sql = (
+            f"SELECT COUNT(version) as versions from "
+            f"(DESCRIBE HISTORY {table_name}) "
+            f"WHERE timestamp >= '{time_range}'"

Review Comment:
   Does it make sense to report only data changes? Otherwise we'll get not 
necessary changes like after `VACUUM`, `OPTIMIZE`, ...



##########
airflow/providers/databricks/sensors/databricks_sql.py:
##########
@@ -0,0 +1,123 @@
+#
+# 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 caller: String passed to name a hook to Databricks, defaults to 
"DatabricksSqlSensor"
+    :param client_parameters: Additional parameters internal to Databricks SQL 
Connector parameters.
+    :param sql: SQL query to be executed.
+    """
+
+    template_fields: Sequence[str] = (
+        "databricks_conn_id",
+        "sql",
+        "catalog",
+        "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 = "",
+        schema: str = "default",
+        table_name: str = "",
+        handler: Callable[[Any], Any] = fetch_all_handler,
+        caller: str = "DatabricksSqlSensor",
+        client_parameters: dict[str, Any] | None = None,
+        sql: str = "",
+        **kwargs,
+    ) -> None:
+        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.table_name = table_name
+        self.sql = sql
+        self.caller = caller
+        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 poke(self, context: Context) -> bool:
+        result = self._sql_sensor(self.sql)
+        self.log.debug("SQL result: %s", result)
+        if isinstance(result, list):
+            if len(result) >= 1:
+                return True
+            else:
+                return False

Review Comment:
   this is really just `return len(result) > 0`



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

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

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

Reply via email to