softyoungha commented on code in PR #40633: URL: https://github.com/apache/airflow/pull/40633#discussion_r1667494067
########## airflow/providers/apache/spark/decorators/pyspark_submit.py: ########## @@ -0,0 +1,205 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import inspect +import os +import pickle +import warnings +from tempfile import TemporaryDirectory +from textwrap import dedent +from typing import TYPE_CHECKING, Callable, Sequence + +import dill + +from airflow.configuration import conf +from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory +from airflow.exceptions import AirflowException +from airflow.providers.apache.spark.decorators.pyspark import SPARK_CONTEXT_KEYS +from airflow.providers.apache.spark.operators.spark_submit import SparkSubmitOperator +from airflow.utils.python_virtualenv import write_python_script + +if TYPE_CHECKING: + from airflow.utils.context import Context + + +class _PysparkSubmitDecoratedOperator(DecoratedOperator, SparkSubmitOperator): + custom_operator_name = "@task.pyspark_submit" + + template_fields: Sequence[str] = ( + "conf", + "files", + "py_files", + "jars", + "driver_class_path", + "packages", + "exclude_packages", + "keytab", + "principal", + "proxy_user", + "name", + "env_vars", + "properties_file", + "op_args", + "op_kwargs", + ) + + def __init__( + self, + python_callable: Callable, + op_args: Sequence | None = None, + op_kwargs: dict | None = None, + use_dill: bool = False, + expect_airflow: bool = False, + **kwargs, + ): + self.use_dill = use_dill + self.expect_airflow = expect_airflow + + signature = inspect.signature(python_callable) + parameters = [ + param.replace(default=None) if param.name in SPARK_CONTEXT_KEYS else param + for param in signature.parameters.values() + ] + # mypy does not understand __signature__ attribute + # see https://github.com/python/mypy/issues/12472 + python_callable.__signature__ = signature.replace(parameters=parameters) # type: ignore[attr-defined] + + if kwargs.get("application"): + if not conf.getboolean("operators", "ALLOW_ILLEGAL_ARGUMENTS"): + raise AirflowException( + "Invalid argument 'application' were passed to `@task.pyspark_submit`." + ) + warnings.warn( + "Invalid argument 'application' were passed to @task.pyspark_submit.", + UserWarning, + stacklevel=2, + ) + if kwargs.get("application_args"): + if not conf.getboolean("operators", "ALLOW_ILLEGAL_ARGUMENTS"): + raise AirflowException( + "Invalid argument 'application_args' were passed to `@task.pyspark_submit`." + ) + warnings.warn( + "Invalid argument 'application_args' were passed to `@task.pyspark_submit`.", + UserWarning, + stacklevel=2, + ) + for key in SPARK_CONTEXT_KEYS: + if key in kwargs: + if not conf.getboolean("operators", "ALLOW_ILLEGAL_ARGUMENTS"): + raise AirflowException( + f"Invalid key '{key}' in op_kwargs. You don't need to set it because it's a " + "variable that will be automatically set within the Python process of the Spark " + "job submitted via spark-submit." + ) + warnings.warn( + f"Invalid key '{key}' in op_kwargs. You don't need to set it because it's a " + "variable that will be automatically set within the Python process of the Spark " + "job submitted via spark-submit.", + UserWarning, + stacklevel=2, + ) + + super().__init__( + python_callable=python_callable, + op_args=op_args, + op_kwargs=op_kwargs, + **kwargs, + ) + + def execute(self, context: Context): + with TemporaryDirectory() as tmp_dir: + script_filename = os.path.join(tmp_dir, "script.py") + input_filename = os.path.join(tmp_dir, "SCRIPT__GENERATED__AIRFLOW.IN") Review Comment: This pattern follows `_DockerDecoratedOperator`. https://github.com/apache/airflow/blob/e827bfbd2c79f3432411e14351c245a06410766a/airflow/providers/docker/decorators/docker.py#L95-L115 ########## airflow/decorators/__init__.pyi: ########## @@ -726,6 +726,94 @@ class TaskDecoratorCollection: self, python_callable: Callable[FParams, FReturn] | None = None ) -> Task[FParams, FReturn]: ... @overload + def pyspark_submit( + self, + *, + multiple_outputs: bool | None = None, + use_dill: bool = False, # Added by _PysparkSubmitDecoratedOperator. + conf: dict[str, Any] | None = None, + conn_id: str = "spark_default", + files: str | None = None, + py_files: str | None = None, + archives: str | None = None, + driver_class_path: str | None = None, + jars: str | None = None, + java_class: str | None = None, + packages: str | None = None, + exclude_packages: str | None = None, + repositories: str | None = None, + total_executor_cores: int | None = None, + executor_cores: int | None = None, + executor_memory: str | None = None, + driver_memory: str | None = None, + keytab: str | None = None, + principal: str | None = None, + proxy_user: str | None = None, + name: str = "arrow-spark", + num_executors: int | None = None, + status_poll_interval: int = 1, + env_vars: dict[str, Any] | None = None, + verbose: bool = False, + spark_binary: str | None = None, + properties_file: str | None = None, + yarn_queue: str | None = None, + deploy_mode: str | None = None, + use_krb5ccache: bool = False, Review Comment: Retrieve all arguments of `SparkSubmitOperator` except for `application` and `application_args`. https://github.com/apache/airflow/blob/e827bfbd2c79f3432411e14351c245a06410766a/airflow/providers/apache/spark/operators/spark_submit.py#L38-L80 ########## airflow/providers/apache/spark/decorators/pyspark_submit.py: ########## @@ -0,0 +1,205 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import inspect +import os +import pickle +import warnings +from tempfile import TemporaryDirectory +from textwrap import dedent +from typing import TYPE_CHECKING, Callable, Sequence + +import dill + +from airflow.configuration import conf +from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory +from airflow.exceptions import AirflowException +from airflow.providers.apache.spark.decorators.pyspark import SPARK_CONTEXT_KEYS +from airflow.providers.apache.spark.operators.spark_submit import SparkSubmitOperator +from airflow.utils.python_virtualenv import write_python_script + +if TYPE_CHECKING: + from airflow.utils.context import Context + + +class _PysparkSubmitDecoratedOperator(DecoratedOperator, SparkSubmitOperator): + custom_operator_name = "@task.pyspark_submit" + + template_fields: Sequence[str] = ( + "conf", + "files", + "py_files", + "jars", + "driver_class_path", + "packages", + "exclude_packages", + "keytab", + "principal", + "proxy_user", + "name", + "env_vars", + "properties_file", + "op_args", + "op_kwargs", + ) + + def __init__( + self, + python_callable: Callable, + op_args: Sequence | None = None, + op_kwargs: dict | None = None, + use_dill: bool = False, + expect_airflow: bool = False, + **kwargs, + ): + self.use_dill = use_dill + self.expect_airflow = expect_airflow + + signature = inspect.signature(python_callable) + parameters = [ + param.replace(default=None) if param.name in SPARK_CONTEXT_KEYS else param + for param in signature.parameters.values() + ] + # mypy does not understand __signature__ attribute + # see https://github.com/python/mypy/issues/12472 + python_callable.__signature__ = signature.replace(parameters=parameters) # type: ignore[attr-defined] + + if kwargs.get("application"): + if not conf.getboolean("operators", "ALLOW_ILLEGAL_ARGUMENTS"): + raise AirflowException( + "Invalid argument 'application' were passed to `@task.pyspark_submit`." + ) + warnings.warn( + "Invalid argument 'application' were passed to @task.pyspark_submit.", + UserWarning, + stacklevel=2, + ) + if kwargs.get("application_args"): + if not conf.getboolean("operators", "ALLOW_ILLEGAL_ARGUMENTS"): + raise AirflowException( + "Invalid argument 'application_args' were passed to `@task.pyspark_submit`." + ) + warnings.warn( + "Invalid argument 'application_args' were passed to `@task.pyspark_submit`.", + UserWarning, + stacklevel=2, + ) Review Comment: I wanted to include it in the kwargs-related exceptions of `BaseOperator`, but I haven't yet found a way to exclude it from the reserved kwargs of the `SparkSubmitOperator`. Any ideas? https://github.com/apache/airflow/blob/e827bfbd2c79f3432411e14351c245a06410766a/airflow/models/baseoperator.py#L931-L944 ########## airflow/providers/apache/spark/decorators/pyspark_submit.py: ########## @@ -0,0 +1,205 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, +# software distributed under the License is distributed on an +# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +# KIND, either express or implied. See the License for the +# specific language governing permissions and limitations +# under the License. +from __future__ import annotations + +import inspect +import os +import pickle +import warnings +from tempfile import TemporaryDirectory +from textwrap import dedent +from typing import TYPE_CHECKING, Callable, Sequence + +import dill + +from airflow.configuration import conf +from airflow.decorators.base import DecoratedOperator, TaskDecorator, task_decorator_factory +from airflow.exceptions import AirflowException +from airflow.providers.apache.spark.decorators.pyspark import SPARK_CONTEXT_KEYS +from airflow.providers.apache.spark.operators.spark_submit import SparkSubmitOperator +from airflow.utils.python_virtualenv import write_python_script + +if TYPE_CHECKING: + from airflow.utils.context import Context + + +class _PysparkSubmitDecoratedOperator(DecoratedOperator, SparkSubmitOperator): + custom_operator_name = "@task.pyspark_submit" + + template_fields: Sequence[str] = ( + "conf", + "files", + "py_files", + "jars", + "driver_class_path", + "packages", + "exclude_packages", + "keytab", + "principal", + "proxy_user", + "name", + "env_vars", + "properties_file", + "op_args", + "op_kwargs", + ) + + def __init__( + self, + python_callable: Callable, + op_args: Sequence | None = None, + op_kwargs: dict | None = None, + use_dill: bool = False, + expect_airflow: bool = False, + **kwargs, + ): + self.use_dill = use_dill + self.expect_airflow = expect_airflow + + signature = inspect.signature(python_callable) + parameters = [ + param.replace(default=None) if param.name in SPARK_CONTEXT_KEYS else param + for param in signature.parameters.values() + ] + # mypy does not understand __signature__ attribute + # see https://github.com/python/mypy/issues/12472 + python_callable.__signature__ = signature.replace(parameters=parameters) # type: ignore[attr-defined] + + if kwargs.get("application"): + if not conf.getboolean("operators", "ALLOW_ILLEGAL_ARGUMENTS"): + raise AirflowException( + "Invalid argument 'application' were passed to `@task.pyspark_submit`." + ) + warnings.warn( + "Invalid argument 'application' were passed to @task.pyspark_submit.", + UserWarning, + stacklevel=2, + ) + if kwargs.get("application_args"): + if not conf.getboolean("operators", "ALLOW_ILLEGAL_ARGUMENTS"): + raise AirflowException( + "Invalid argument 'application_args' were passed to `@task.pyspark_submit`." + ) + warnings.warn( + "Invalid argument 'application_args' were passed to `@task.pyspark_submit`.", + UserWarning, + stacklevel=2, + ) + for key in SPARK_CONTEXT_KEYS: + if key in kwargs: + if not conf.getboolean("operators", "ALLOW_ILLEGAL_ARGUMENTS"): + raise AirflowException( + f"Invalid key '{key}' in op_kwargs. You don't need to set it because it's a " + "variable that will be automatically set within the Python process of the Spark " + "job submitted via spark-submit." + ) + warnings.warn( + f"Invalid key '{key}' in op_kwargs. You don't need to set it because it's a " + "variable that will be automatically set within the Python process of the Spark " + "job submitted via spark-submit.", + UserWarning, + stacklevel=2, + ) + + super().__init__( + python_callable=python_callable, + op_args=op_args, + op_kwargs=op_kwargs, + **kwargs, + ) + + def execute(self, context: Context): + with TemporaryDirectory() as tmp_dir: + script_filename = os.path.join(tmp_dir, "script.py") + input_filename = os.path.join(tmp_dir, "SCRIPT__GENERATED__AIRFLOW.IN") + + if self.op_args or self.op_kwargs: + with open(input_filename, "wb") as file: + self.pickling_library.dump({"args": self.op_args, "kwargs": self.op_kwargs}, file) + files = (self.files or "").split(",") + self.files = ",".join(files + [input_filename]) + + parameters = inspect.signature(self.python_callable).parameters + use_spark_context = use_spark_session = False + if "sc" in parameters: + use_spark_context = True + if "spark" in parameters: + use_spark_session = True + + py_source = dedent( + f"""\ + from pyspark import SparkFiles + from pyspark.sql import SparkSession + + # Script + {{python_callable_source}} + + if {bool(self.op_args or self.op_kwargs)}: + with open(SparkFiles.get("{os.path.basename(input_filename)}"), 'rb') as file: + arg_dict = {self.pickling_library.__name__}.load(file) + else: + arg_dict = {{default_arg_dict}} + + if {use_spark_session}: + arg_dict["kwargs"]["spark"] = SparkSession.builder.getOrCreate() + if {use_spark_context}: + spark = arg_dict.get("spark") or SparkSession.builder.getOrCreate() + arg_dict["kwargs"]["sc"] = spark.sparkContext + + # Call + {self.python_callable.__name__}(*arg_dict["args"], **arg_dict["kwargs"]) + + # Exit + exit(0) + """ + ).format( + python_callable_source=self.get_python_source(), default_arg_dict='{"args": [], "kwargs": {}}' + ) Review Comment: To enable reading input file from spark job submitted with spark-submit, it needs to be passed using `--files` option of spark-submit. However, the actual path of files passed via `--files` option is determined at runtime (under /tmp somewhere). In current `write_python_script` function, which uses [Jinja2 template file](https://github.com/apache/airflow/blob/main/airflow/utils/python_virtualenv_script.jinja2#L54-L73), `sys.argv[2]` is hardcoded. This makes it seemingly impossible to pass op_args and op_kwargs as Jinja2 context. Therefore, I crafted the execution statement directly and forced termination with `exit(0)`. https://github.com/apache/airflow/blob/e827bfbd2c79f3432411e14351c245a06410766a/airflow/utils/python_virtualenv_script.jinja2#L54-L73 -- 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