From 66a198d0e641b4f0feab1a274fad4217f3ca1819 Mon Sep 17 00:00:00 2001 From: Elad Kalif <45845474+eladkal@users.noreply.github.com> Date: Sun, 27 Apr 2025 09:00:56 +0300 Subject: [PATCH] Cleanup parsing_pre_import_modules setting --- airflow-core/newsfragments/49839.misc.rst | 3 ++ .../src/airflow/config_templates/config.yml | 10 ---- airflow-core/src/airflow/utils/file.py | 21 --------- airflow-core/tests/unit/dags/test_imports.py | 47 ------------------- airflow-core/tests/unit/utils/test_file.py | 19 -------- 5 files changed, 3 insertions(+), 97 deletions(-) create mode 100644 airflow-core/newsfragments/49839.misc.rst delete mode 100644 airflow-core/tests/unit/dags/test_imports.py diff --git a/airflow-core/newsfragments/49839.misc.rst b/airflow-core/newsfragments/49839.misc.rst new file mode 100644 index 0000000000000..bd29c6fc0618d --- /dev/null +++ b/airflow-core/newsfragments/49839.misc.rst @@ -0,0 +1,3 @@ +The ``parsing_pre_import_modules`` setting is removed from code base. +It has no use in Airflow 3. +If your deployment override this setting you can safely remove it. diff --git a/airflow-core/src/airflow/config_templates/config.yml b/airflow-core/src/airflow/config_templates/config.yml index a14efca725514..cbcd28bda894c 100644 --- a/airflow-core/src/airflow/config_templates/config.yml +++ b/airflow-core/src/airflow/config_templates/config.yml @@ -2199,16 +2199,6 @@ scheduler: type: integer default: "20" see_also: ":ref:`scheduler:ha:tunables`" - parsing_pre_import_modules: - description: | - The scheduler reads dag files to extract the airflow modules that are going to be used, - and imports them ahead of time to avoid having to re-do it for each parsing process. - This flag can be set to ``False`` to disable this behavior in case an airflow module needs - to be freshly imported each time (at the cost of increased DAG parsing time). - version_added: 2.6.0 - type: boolean - example: ~ - default: "True" dag_stale_not_seen_duration: description: | Time in seconds after which dags, which were not updated by Dag Processor are deactivated. diff --git a/airflow-core/src/airflow/utils/file.py b/airflow-core/src/airflow/utils/file.py index 12138043b85d9..a9ac02a7e5bf5 100644 --- a/airflow-core/src/airflow/utils/file.py +++ b/airflow-core/src/airflow/utils/file.py @@ -17,7 +17,6 @@ # under the License. from __future__ import annotations -import ast import hashlib import logging import os @@ -323,26 +322,6 @@ def might_contain_dag_via_default_heuristic(file_path: str, zip_file: zipfile.Zi return any(s in content for s in (b"dag", b"asset")) -def _find_imported_modules(module: ast.Module) -> Generator[str, None, None]: - for st in module.body: - if isinstance(st, ast.Import): - for n in st.names: - yield n.name - elif isinstance(st, ast.ImportFrom) and st.module is not None: - yield st.module - - -def iter_airflow_imports(file_path: str) -> Generator[str, None, None]: - """Find Airflow modules imported in the given file.""" - try: - parsed = ast.parse(Path(file_path).read_bytes()) - except Exception: - return - for m in _find_imported_modules(parsed): - if m.startswith("airflow."): - yield m - - def get_unique_dag_module_name(file_path: str) -> str: """Return a unique module name in the format unusual_prefix_{sha1 of module's file path}_{original module name}.""" if isinstance(file_path, str): diff --git a/airflow-core/tests/unit/dags/test_imports.py b/airflow-core/tests/unit/dags/test_imports.py deleted file mode 100644 index d46204c4726ec..0000000000000 --- a/airflow-core/tests/unit/dags/test_imports.py +++ /dev/null @@ -1,47 +0,0 @@ -# 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. - -# fmt: off - -# this file contains sample code than only needs to pass the lexer -# it is "badly" formatted on purpose to test edge cases. - -from __future__ import annotations - -# multiline import -import \ - datetime, \ - enum,time -""" -import airflow.in_comment -""" -# from import -from airflow.utils import file -# multiline airflow import -import airflow.decorators, airflow.models \ - , airflow.sensors - -if prod: # type: ignore[name-defined] - import airflow.if_branch -else: - import airflow.else_branch - -def f(): - # local import - import airflow.local_import - -# fmt: on diff --git a/airflow-core/tests/unit/utils/test_file.py b/airflow-core/tests/unit/utils/test_file.py index ab0ae5a3f62d3..9aab22a838be8 100644 --- a/airflow-core/tests/unit/utils/test_file.py +++ b/airflow-core/tests/unit/utils/test_file.py @@ -193,24 +193,6 @@ def test_might_contain_dag(self): # With safe_mode is False, the user defined callable won't be invoked assert file_utils.might_contain_dag(file_path=file_path_with_dag, safe_mode=False) - def test_get_modules(self): - file_path = os.path.join(TEST_DAGS_FOLDER, "test_imports.py") - - modules = list(file_utils.iter_airflow_imports(file_path)) - - assert len(modules) == 4 - assert "airflow.utils" in modules - assert "airflow.decorators" in modules - assert "airflow.models" in modules - assert "airflow.sensors" in modules - # this one is a local import, we don't want it. - assert "airflow.local_import" not in modules - # this one is in a comment, we don't want it - assert "airflow.in_comment" not in modules - # we don't want imports under conditions - assert "airflow.if_branch" not in modules - assert "airflow.else_branch" not in modules - def test_get_modules_from_invalid_file(self): file_path = os.path.join(TEST_DAGS_FOLDER, "README.md") # just getting a non-python file @@ -233,7 +215,6 @@ def test_list_py_file_paths(self, test_zip_path): "test_invalid_param3.py", "test_invalid_param4.py", "test_nested_dag.py", - "test_imports.py", "file_no_airflow_dag.py", # no_dag test case in test_zip folder "test.py", # no_dag test case in test_zip_module folder "__init__.py",