You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by "harishkrao (via GitHub)" <gi...@apache.org> on 2023/02/06 03:36:37 UTC

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

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


##########
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:
   @alexott For this change, I am thinking of generating the SQL instead of receiving it as an argument, to prevent injections: `SELECT {expression_list, formatted — list} FROM {table_name} (generated by arguments) WHERE {filter_vals, formatted — dict} GROUP BY {group_by vals, formatted — list} ORDER BY {order_by vals, formatted — list}` where the where , group by and order by are optional, but will be validated by the formatter if it receives values for the arguments in the class.
   
   Thought of two options: Generating the SQL like the above, which has a downside of restricting the SQLs we can allow to be passed to the sensor, but it also prevents unexpected injections as a result.
   
   Another option is to breakdown the SQL received from the user and do the validation via the SQL formatter before executing it via the hook.
   
   I am leaning towards the first option, any thoughts/preferences or alternative suggestions? 



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