You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by GitBox <gi...@apache.org> on 2022/11/14 06:29:33 UTC

[GitHub] [airflow] dstandish commented on a diff in pull request #24652: Add @task.snowpark decorator

dstandish commented on code in PR #24652:
URL: https://github.com/apache/airflow/pull/24652#discussion_r1021115584


##########
tests/system/providers/snowflake/example_snowflake_snowpark.py:
##########
@@ -0,0 +1,101 @@
+#
+# 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.
+"""
+Example use of Snowflake Snowpark related decorator.
+"""
+from __future__ import annotations
+
+import sys
+
+import pytest
+
+if not sys.version_info[0:2] == (3, 8):
+    pytest.skip("unsupported python version", allow_module_level=True)
+from datetime import datetime
+from random import uniform
+
+import snowflake.snowpark
+from snowflake.snowpark.functions import col, count, lit, random as spRandom, sproc, uniform as spUniform
+from snowflake.snowpark.types import FloatType
+
+from airflow import DAG, AirflowException
+from airflow.decorators import task
+
+SNOWFLAKE_CONN_ID = "my_snowflake_conn"
+DAG_ID = "example_snowflake_snowpark"
+
+with DAG(
+    DAG_ID,
+    start_date=datetime(2021, 1, 1),
+    schedule_interval="@once",

Review Comment:
   ```suggestion
       schedule="@once",
   ```



##########
airflow/providers/snowflake/decorators/snowpark.py:
##########
@@ -0,0 +1,154 @@
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements.  See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership.  The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License.  You may obtain a copy of the License at
+#
+#   http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+from __future__ import annotations
+
+from typing import TYPE_CHECKING, Callable, Sequence
+
+from airflow.decorators.base import DecoratedOperator, task_decorator_factory
+from airflow.exceptions import AirflowException
+from airflow.operators.python import PythonOperator
+from airflow.providers.snowflake.hooks.snowflake import SnowflakeHook
+
+if TYPE_CHECKING:
+    from airflow.decorators.base import TaskDecorator
+
+try:
+    import snowflake.snowpark  # noqa
+except ImportError:
+    raise AirflowException(
+        "The snowflake-snowpark-python package is not installed. Make sure you are using Python 3.8."
+    )
+
+
+class _SnowparkDecoratedOperator(DecoratedOperator, PythonOperator):
+    """
+    Wraps a Python callable and captures args/kwargs when called for execution.
+
+    :param snowflake_conn_id: Reference to
+        :ref:`Snowflake connection id<howto/connection:snowflake>`
+    :param parameters: (optional) the parameters to render the SQL query with.
+    :param warehouse: name of warehouse (will overwrite any warehouse defined in the connection's extra JSON)
+    :param database: name of database (will overwrite database defined in connection)
+    :param schema: name of schema (will overwrite schema defined in connection)
+    :param role: name of role (will overwrite any role defined in connection's extra JSON)
+    :param authenticator: authenticator for Snowflake.
+        'snowflake' (default) to use the internal Snowflake authenticator
+        'externalbrowser' to authenticate using your web browser and
+        Okta, ADFS or any other SAML 2.0-compliant identify provider
+        (IdP) that has been defined for your account
+        'https://<your_okta_account_name>.okta.com' to authenticate
+        through native Okta.
+    :param session_parameters: You can set session-level parameters at the time you connect to Snowflake
+    :param python_callable: A reference to an object that is callable
+    :param op_kwargs: a dictionary of keyword arguments that will get unpacked in your function (templated)
+    :param op_args: a list of positional arguments that will get unpacked when
+        calling your callable (templated)
+    :param multiple_outputs: if set, function return value will be
+        unrolled to multiple XCom values. Dict will unroll to xcom values with keys as keys.
+        Defaults to False.
+    """
+
+    custom_operator_name = "@task.snowpark"
+
+    template_fields: Sequence[str] = ("op_args", "op_kwargs")
+    template_fields_renderers = {"op_args": "py", "op_kwargs": "py"}
+
+    # since we won't mutate the arguments, we should just do the shallow copy
+    # there are some cases we can't deepcopy the objects (e.g protobuf).
+    shallow_copy_attrs: Sequence[str] = ("python_callable",)
+
+    def __init__(
+        self,
+        *,
+        snowflake_conn_id: str = "snowflake_default",
+        parameters: dict | None = None,
+        warehouse: str | None = None,
+        database: str | None = None,
+        role: str | None = None,
+        schema: str | None = None,
+        authenticator: str | None = None,
+        session_parameters: dict | None = None,
+        python_callable,
+        op_args,
+        op_kwargs: dict,
+        **kwargs,
+    ) -> None:
+        self.snowflake_conn_id = snowflake_conn_id
+        self.parameters = parameters
+        self.warehouse = warehouse
+        self.database = database
+        self.role = role
+        self.schema = schema
+        self.authenticator = authenticator
+        self.session_parameters = session_parameters
+
+        kwargs_to_upstream = {
+            "python_callable": python_callable,
+            "op_args": op_args,
+            "op_kwargs": op_kwargs,
+        }
+        super().__init__(
+            kwargs_to_upstream=kwargs_to_upstream,
+            python_callable=python_callable,
+            op_args=op_args,
+            # airflow.decorators.base.DecoratedOperator checks if the functions are bindable, so we have to
+            # add an artificial value to pass the validation. The real value is determined at runtime.
+            op_kwargs={**op_kwargs, "snowpark_session": None},
+            **kwargs,
+        )
+
+    def execute_callable(self):
+        hook = SnowflakeHook(
+            snowflake_conn_id=self.snowflake_conn_id,
+            parameters=self.parameters,
+            warehouse=self.warehouse,
+            database=self.database,
+            role=self.role,
+            schema=self.schema,
+            authenticator=self.authenticator,
+            session_parameters=self.session_parameters,
+        )
+        snowpark_session = hook.get_snowpark_session()
+        try:
+            op_kwargs = dict(self.op_kwargs)
+            # Set real sessions as an argument to the function.
+            op_kwargs["snowpark_session"] = snowpark_session
+            return self.python_callable(*self.op_args, **self.op_kwargs)
+        finally:
+            snowpark_session.close()

Review Comment:
   ```suggestion
           with hook.get_snowpark_session() as snowpark_session:
               op_kwargs = dict(self.op_kwargs)
               # Set real sessions as an argument to the function.
               op_kwargs["snowpark_session"] = snowpark_session
               return self.python_callable(*self.op_args, **self.op_kwargs)
   ```



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