From 18eb6b844d51601f65336e1119021402f43ab27f Mon Sep 17 00:00:00 2001 From: dirrao Date: Thu, 15 Aug 2024 11:35:24 +0530 Subject: [PATCH 1/4] python operators deprecations removal --- airflow/operators/python.py | 63 ---------------------------------- tests/operators/test_python.py | 16 ++++++--- 2 files changed, 11 insertions(+), 68 deletions(-) diff --git a/airflow/operators/python.py b/airflow/operators/python.py index 25c0b8ca68632..cf27aa004c067 100644 --- a/airflow/operators/python.py +++ b/airflow/operators/python.py @@ -77,38 +77,6 @@ def is_venv_installed() -> bool: return False -def task(python_callable: Callable | None = None, multiple_outputs: bool | None = None, **kwargs): - """ - Use :func:`airflow.decorators.task` instead, this is deprecated. - - Calls ``@task.python`` and allows users to turn a Python function into - an Airflow task. - - :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. - """ - # To maintain backwards compatibility, we import the task object into this file - # This prevents breakages in dags that use `from airflow.operators.python import task` - from airflow.decorators.python import python_task - - warnings.warn( - """airflow.operators.python.task is deprecated. Please use the following instead - - from airflow.decorators import task - @task - def my_task()""", - RemovedInAirflow3Warning, - stacklevel=2, - ) - return python_task(python_callable=python_callable, multiple_outputs=multiple_outputs, **kwargs) - - @cache def _parse_version_info(text: str) -> tuple[int, int, int, str, int]: """Parse python version info from a text.""" @@ -213,13 +181,6 @@ def __init__( show_return_value_in_logs: bool = True, **kwargs, ) -> None: - if kwargs.get("provide_context"): - warnings.warn( - "provide_context is deprecated as of 2.0 and is no longer required", - RemovedInAirflow3Warning, - stacklevel=2, - ) - kwargs.pop("provide_context", None) super().__init__(**kwargs) if not callable(python_callable): raise AirflowException("`python_callable` param must be callable") @@ -441,7 +402,6 @@ def __init__( skip_on_exit_code: int | Container[int] | None = None, env_vars: dict[str, str] | None = None, inherit_env: bool = True, - use_dill: bool = False, use_airflow_context: bool = False, **kwargs, ): @@ -462,19 +422,6 @@ def __init__( **kwargs, ) self.string_args = string_args or [] - - if use_dill: - warnings.warn( - "`use_dill` is deprecated and will be removed in a future version. " - "Please provide serializer='dill' instead.", - RemovedInAirflow3Warning, - stacklevel=3, - ) - if serializer: - raise AirflowException( - "Both 'use_dill' and 'serializer' parameters are set. Please set only one of them" - ) - serializer = "dill" serializer = serializer or "pickle" if serializer not in _SERIALIZERS: msg = ( @@ -686,9 +633,6 @@ class PythonVirtualenvOperator(_BasePythonVirtualenvOperator): environment. If set to ``True``, the virtual environment will inherit the environment variables of the parent process (``os.environ``). If set to ``False``, the virtual environment will be executed with a clean environment. - :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize - the args and result (pickle is default). This allows more complex types - but requires you to include dill in your requirements. :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable. """ @@ -717,7 +661,6 @@ def __init__( venv_cache_path: None | os.PathLike[str] = None, env_vars: dict[str, str] | None = None, inherit_env: bool = True, - use_dill: bool = False, use_airflow_context: bool = False, **kwargs, ): @@ -771,7 +714,6 @@ def __init__( skip_on_exit_code=skip_on_exit_code, env_vars=env_vars, inherit_env=inherit_env, - use_dill=use_dill, use_airflow_context=use_airflow_context, **kwargs, ) @@ -988,9 +930,6 @@ class ExternalPythonOperator(_BasePythonVirtualenvOperator): environment. If set to ``True``, the virtual environment will inherit the environment variables of the parent process (``os.environ``). If set to ``False``, the virtual environment will be executed with a clean environment. - :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize - the args and result (pickle is default). This allows more complex types - but requires you to include dill in your requirements. :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable. """ @@ -1012,7 +951,6 @@ def __init__( skip_on_exit_code: int | Container[int] | None = None, env_vars: dict[str, str] | None = None, inherit_env: bool = True, - use_dill: bool = False, use_airflow_context: bool = False, **kwargs, ): @@ -1035,7 +973,6 @@ def __init__( skip_on_exit_code=skip_on_exit_code, env_vars=env_vars, inherit_env=inherit_env, - use_dill=use_dill, use_airflow_context=use_airflow_context, **kwargs, ) diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py index 28c66511a3d38..635f3a744832c 100644 --- a/tests/operators/test_python.py +++ b/tests/operators/test_python.py @@ -292,7 +292,8 @@ def func(custom, dag): assert 1 == custom, "custom should be 1" assert dag is not None, "dag should be set" - with pytest.warns(RemovedInAirflow3Warning): + error_message = "Invalid arguments were passed to PythonOperator \\(task_id: task_test-provide-context-does-not-fail\\). Invalid arguments were:\n\\*\\*kwargs: {'provide_context': True}" + with pytest.raises(AirflowException, match=error_message): self.run_as_task(func, op_kwargs={"custom": 1}, provide_context=True) def test_context_with_conflicting_op_args(self): @@ -1156,16 +1157,21 @@ def f(): """Ensure dill is correctly installed.""" import dill # noqa: F401 - with pytest.warns(RemovedInAirflow3Warning, match="`use_dill` is deprecated and will be removed"): + with pytest.raises( + AirflowException, + match="Invalid arguments were passed to PythonVirtualenvOperator \\(task_id: task_test-add-dill-use-dill\\). Invalid arguments were:\n\\*\\*kwargs: {'use_dill': True}", + ): self.run_as_task(f, use_dill=True, system_site_packages=False) def test_ambiguous_serializer(self): def f(): pass - with pytest.warns(RemovedInAirflow3Warning, match="`use_dill` is deprecated and will be removed"): - with pytest.raises(AirflowException, match="Both 'use_dill' and 'serializer' parameters are set"): - self.run_as_task(f, use_dill=True, serializer="dill") + with pytest.raises( + AirflowException, + match="Invalid arguments were passed to PythonVirtualenvOperator \\(task_id: task_test-ambiguous-serializer\\). Invalid arguments were:\n\\*\\*kwargs: {'use_dill': True}", + ): + self.run_as_task(f, use_dill=True, serializer="dill") def test_invalid_serializer(self): def f(): From 8f03d2a8b52c6d49235b5a624869d278d1e5ef78 Mon Sep 17 00:00:00 2001 From: dirrao Date: Thu, 15 Aug 2024 11:52:23 +0530 Subject: [PATCH 2/4] python operators deprecations removal --- airflow/operators/python.py | 24 ++++++++++++++++++++++++ tests/operators/test_python.py | 13 ++++--------- 2 files changed, 28 insertions(+), 9 deletions(-) diff --git a/airflow/operators/python.py b/airflow/operators/python.py index cf27aa004c067..cc31ee0501875 100644 --- a/airflow/operators/python.py +++ b/airflow/operators/python.py @@ -402,6 +402,7 @@ def __init__( skip_on_exit_code: int | Container[int] | None = None, env_vars: dict[str, str] | None = None, inherit_env: bool = True, + use_dill: bool = False, use_airflow_context: bool = False, **kwargs, ): @@ -422,6 +423,19 @@ def __init__( **kwargs, ) self.string_args = string_args or [] + + if use_dill: + warnings.warn( + "`use_dill` is deprecated and will be removed in a future version. " + "Please provide serializer='dill' instead.", + RemovedInAirflow3Warning, + stacklevel=3, + ) + if serializer: + raise AirflowException( + "Both 'use_dill' and 'serializer' parameters are set. Please set only one of them" + ) + serializer = "dill" serializer = serializer or "pickle" if serializer not in _SERIALIZERS: msg = ( @@ -633,6 +647,9 @@ class PythonVirtualenvOperator(_BasePythonVirtualenvOperator): environment. If set to ``True``, the virtual environment will inherit the environment variables of the parent process (``os.environ``). If set to ``False``, the virtual environment will be executed with a clean environment. + :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize + the args and result (pickle is default). This allows more complex types + but requires you to include dill in your requirements. :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable. """ @@ -661,6 +678,7 @@ def __init__( venv_cache_path: None | os.PathLike[str] = None, env_vars: dict[str, str] | None = None, inherit_env: bool = True, + use_dill: bool = False, use_airflow_context: bool = False, **kwargs, ): @@ -714,6 +732,7 @@ def __init__( skip_on_exit_code=skip_on_exit_code, env_vars=env_vars, inherit_env=inherit_env, + use_dill=use_dill, use_airflow_context=use_airflow_context, **kwargs, ) @@ -930,6 +949,9 @@ class ExternalPythonOperator(_BasePythonVirtualenvOperator): environment. If set to ``True``, the virtual environment will inherit the environment variables of the parent process (``os.environ``). If set to ``False``, the virtual environment will be executed with a clean environment. + :param use_dill: Deprecated, use ``serializer`` instead. Whether to use dill to serialize + the args and result (pickle is default). This allows more complex types + but requires you to include dill in your requirements. :param use_airflow_context: Whether to provide ``get_current_context()`` to the python_callable. """ @@ -951,6 +973,7 @@ def __init__( skip_on_exit_code: int | Container[int] | None = None, env_vars: dict[str, str] | None = None, inherit_env: bool = True, + use_dill: bool = False, use_airflow_context: bool = False, **kwargs, ): @@ -973,6 +996,7 @@ def __init__( skip_on_exit_code=skip_on_exit_code, env_vars=env_vars, inherit_env=inherit_env, + use_dill=use_dill, use_airflow_context=use_airflow_context, **kwargs, ) diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py index 635f3a744832c..d302e8cab0085 100644 --- a/tests/operators/test_python.py +++ b/tests/operators/test_python.py @@ -1157,21 +1157,16 @@ def f(): """Ensure dill is correctly installed.""" import dill # noqa: F401 - with pytest.raises( - AirflowException, - match="Invalid arguments were passed to PythonVirtualenvOperator \\(task_id: task_test-add-dill-use-dill\\). Invalid arguments were:\n\\*\\*kwargs: {'use_dill': True}", - ): + with pytest.warns(RemovedInAirflow3Warning, match="`use_dill` is deprecated and will be removed"): self.run_as_task(f, use_dill=True, system_site_packages=False) def test_ambiguous_serializer(self): def f(): pass - with pytest.raises( - AirflowException, - match="Invalid arguments were passed to PythonVirtualenvOperator \\(task_id: task_test-ambiguous-serializer\\). Invalid arguments were:\n\\*\\*kwargs: {'use_dill': True}", - ): - self.run_as_task(f, use_dill=True, serializer="dill") + with pytest.warns(RemovedInAirflow3Warning, match="`use_dill` is deprecated and will be removed"): + with pytest.raises(AirflowException, match="Both 'use_dill' and 'serializer' parameters are set"): + self.run_as_task(f, use_dill=True, serializer="dill") def test_invalid_serializer(self): def f(): From e146854a37acfd14bba17e15f0168e6940e8a91e Mon Sep 17 00:00:00 2001 From: dirrao Date: Mon, 26 Aug 2024 08:28:14 +0530 Subject: [PATCH 3/4] news fragment added --- newsfragments/41493.significant.rst | 1 + 1 file changed, 1 insertion(+) create mode 100644 newsfragments/41493.significant.rst diff --git a/newsfragments/41493.significant.rst b/newsfragments/41493.significant.rst new file mode 100644 index 0000000000000..0bdd4b1c043c6 --- /dev/null +++ b/newsfragments/41493.significant.rst @@ -0,0 +1 @@ +Removed deprecated method ``task`` from ``airflow.operators.python`` operator. Please use ``python_task`` from ``airflow.decorators.python`` instead. From 168bdcf6965632b7fb744a982ce1102e9d619cbe Mon Sep 17 00:00:00 2001 From: dirrao Date: Fri, 30 Aug 2024 16:15:52 +0530 Subject: [PATCH 4/4] non string python version deprecation support removed --- airflow/operators/python.py | 7 ++----- tests/operators/test_python.py | 5 +---- 2 files changed, 3 insertions(+), 9 deletions(-) diff --git a/airflow/operators/python.py b/airflow/operators/python.py index cc31ee0501875..6656b661aecdf 100644 --- a/airflow/operators/python.py +++ b/airflow/operators/python.py @@ -693,11 +693,8 @@ def __init__( f"Sys version: {sys.version_info}. Virtual environment version: {python_version}" ) if python_version is not None and not isinstance(python_version, str): - warnings.warn( - "Passing non-string types (e.g. int or float) as python_version " - "is deprecated. Please use string value instead.", - RemovedInAirflow3Warning, - stacklevel=2, + raise AirflowException( + "Passing non-string types (e.g. int or float) as python_version not supported" ) if not is_venv_installed(): raise AirflowException("PythonVirtualenvOperator requires virtualenv, please install it.") diff --git a/tests/operators/test_python.py b/tests/operators/test_python.py index d302e8cab0085..b0d36744c0c89 100644 --- a/tests/operators/test_python.py +++ b/tests/operators/test_python.py @@ -1335,10 +1335,7 @@ def f(): return raise RuntimeError - with pytest.warns( - RemovedInAirflow3Warning, match="Passing non-string types.*python_version is deprecated" - ): - self.run_as_task(f, python_version=3, serializer=serializer, requirements=extra_requirements) + self.run_as_task(f, python_version="3", serializer=serializer, requirements=extra_requirements) def test_with_default(self): def f(a):