You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by je...@apache.org on 2023/03/08 01:27:56 UTC

[airflow] 02/04: Don't use importlib.metadata to get Version for speed (#29723)

This is an automated email from the ASF dual-hosted git repository.

jedcunningham pushed a commit to branch v2-5-test
in repository https://gitbox.apache.org/repos/asf/airflow.git

commit a004df2dec22b1c3ef750281ae0eb74ef958ad83
Author: Ash Berlin-Taylor <as...@apache.org>
AuthorDate: Fri Feb 24 09:45:00 2023 +0000

    Don't use importlib.metadata to get Version for speed (#29723)
    
    As discovered by @uranusjr in other PRs, loading the Metadata info at
    runtime is surprisingly expensive.
    
    Recent versions of setuptools (including the one we already say we
    depend upon in pyproject.toml) have the ability to pull the version from
    an attribute using an "ast-eval" method so this keeps the property of
    there being a single source of truth for the Airflow version, it just
    moves that places to airflow/__init__.py.
    
    You might wonder why this particular case matters at runtime? In the
    grand scheme of things it likely doesn't, except that the
    airflow/operators/python.py imports this at the top level (or did before
    this PR) which made me look in to it and discover a quick win here.
    
    (cherry picked from commit d0783744fcae40b0b6b2e208a555ea5fd9124dfb)
---
 airflow/__init__.py                                |  3 ++-
 airflow/operators/python.py                        |  5 +++--
 airflow/version.py                                 | 19 +++----------------
 dev/README_RELEASE_AIRFLOW.md                      |  2 +-
 scripts/ci/pre_commit/common_precommit_utils.py    | 13 +++++++++++++
 .../ci/pre_commit/pre_commit_update_versions.py    | 22 +++++++++-------------
 .../ci/pre_commit/pre_commit_version_heads_map.py  | 16 ++++------------
 scripts/in_container/run_migration_reference.py    |  4 ++--
 setup.cfg                                          |  2 ++
 setup.py                                           | 17 ++++++-----------
 10 files changed, 45 insertions(+), 58 deletions(-)

diff --git a/airflow/__init__.py b/airflow/__init__.py
index 41c9157000..334139b1d7 100644
--- a/airflow/__init__.py
+++ b/airflow/__init__.py
@@ -51,6 +51,8 @@ from airflow import settings
 
 __all__ = ["__version__", "login", "DAG", "PY36", "PY37", "PY38", "PY39", "PY310", "XComArg"]
 
+__version__ = "2.6.0.dev0"
+
 # Make `airflow` an namespace package, supporting installing
 # airflow.providers.* in different locations (i.e. one in site, and one in user
 # lib.)
@@ -77,7 +79,6 @@ __lazy_imports: dict[str, tuple[str, str]] = {
     "XComArg": (".models.xcom_arg", "XComArg"),
     "AirflowException": (".exceptions", "AirflowException"),
     "version": (".version", ""),
-    "__version__": (".version", "version"),
 }
 
 
diff --git a/airflow/operators/python.py b/airflow/operators/python.py
index 0a0dd34fef..41fd796507 100644
--- a/airflow/operators/python.py
+++ b/airflow/operators/python.py
@@ -41,7 +41,6 @@ from airflow.utils.context import Context, context_copy_partial, context_merge
 from airflow.utils.operator_helpers import KeywordParameters
 from airflow.utils.process_utils import execute_in_subprocess
 from airflow.utils.python_virtualenv import prepare_virtualenv, write_python_script
-from airflow.version import version as airflow_version
 
 
 def task(python_callable: Callable | None = None, multiple_outputs: bool | None = None, **kwargs):
@@ -693,9 +692,11 @@ class ExternalPythonOperator(_BasePythonVirtualenvOperator):
             return False
 
     def _get_airflow_version_from_target_env(self) -> str | None:
+        from airflow import __version__ as airflow_version
+
         try:
             result = subprocess.check_output(
-                [self.python, "-c", "from airflow import version; print(version.version)"], text=True
+                [self.python, "-c", "from airflow import __version__; print(__version__)"], text=True
             )
             target_airflow_version = result.strip()
             if target_airflow_version != airflow_version:
diff --git a/airflow/version.py b/airflow/version.py
index 3c65cebce0..38fb013244 100644
--- a/airflow/version.py
+++ b/airflow/version.py
@@ -17,20 +17,7 @@
 # under the License.
 from __future__ import annotations
 
-__all__ = ["version"]
-
-try:
-    import importlib_metadata as metadata
-except ImportError:
-    from importlib import metadata  # type: ignore[no-redef]
-
-try:
-    version = metadata.version("apache-airflow")
-except metadata.PackageNotFoundError:
-    import logging
+# Compat -- somethings access `airflow.version.version` directly
+from airflow import __version__ as version
 
-    log = logging.getLogger(__name__)
-    log.warning("Package metadata could not be found. Overriding it with version found in setup.py")
-    from setup import version
-
-del metadata
+__all__ = ["version"]
diff --git a/dev/README_RELEASE_AIRFLOW.md b/dev/README_RELEASE_AIRFLOW.md
index a98a25cc27..dd6f194522 100644
--- a/dev/README_RELEASE_AIRFLOW.md
+++ b/dev/README_RELEASE_AIRFLOW.md
@@ -246,7 +246,7 @@ The Release Candidate artifacts we vote upon should be the exact ones we vote ag
     git reset --hard origin/v${VERSION_BRANCH}-test
     ```
 
-- Set your version in `setup.py` and `airflow/api_connexion/openapi/v1.yaml` (without the RC tag).
+- Set your version in `airflow/__init__.py` and `airflow/api_connexion/openapi/v1.yaml` (without the RC tag).
 - Add supported Airflow version to `./scripts/ci/pre_commit/pre_commit_supported_versions.py` and let pre-commit do the job.
 - Replace the version in `README.md` and verify that installation instructions work fine.
 - Build the release notes:
diff --git a/scripts/ci/pre_commit/common_precommit_utils.py b/scripts/ci/pre_commit/common_precommit_utils.py
index aef6bc3dce..3dc1fceba5 100644
--- a/scripts/ci/pre_commit/common_precommit_utils.py
+++ b/scripts/ci/pre_commit/common_precommit_utils.py
@@ -16,6 +16,7 @@
 # under the License.
 from __future__ import annotations
 
+import ast
 import hashlib
 import os
 import re
@@ -24,6 +25,18 @@ from pathlib import Path
 AIRFLOW_SOURCES_ROOT = Path(__file__).parents[3].resolve()
 
 
+def read_airflow_version() -> str:
+    ast_obj = ast.parse((AIRFLOW_SOURCES_ROOT_PATH / "airflow" / "__init__.py").read_text())
+    for node in ast_obj.body:
+        if not isinstance(node, ast.Assign):
+            continue
+
+        if node.targets[0].id == "__version__":  # type: ignore[attr-defined]
+            return ast.literal_eval(node.value)
+
+    raise RuntimeError("Couldn't find __version__ in AST")
+
+
 def filter_out_providers_on_non_main_branch(files: list[str]) -> list[str]:
     """When running build on non-main branch do not take providers into account"""
     default_branch = os.environ.get("DEFAULT_BRANCH")
diff --git a/scripts/ci/pre_commit/pre_commit_update_versions.py b/scripts/ci/pre_commit/pre_commit_update_versions.py
index ee88db21fa..19cba2c648 100755
--- a/scripts/ci/pre_commit/pre_commit_update_versions.py
+++ b/scripts/ci/pre_commit/pre_commit_update_versions.py
@@ -17,23 +17,18 @@
 # under the License.
 from __future__ import annotations
 
-import glob
-import os
 import re
 import sys
-from os.path import abspath, dirname, join
+from pathlib import Path
 
-AIRFLOW_SOURCES_DIR = abspath(join(dirname(__file__), os.pardir, os.pardir, os.pardir))
+sys.path.insert(0, str(Path(__file__).parent.resolve()))  # make sure common_precommit_utils is importable
 
-sys.path.insert(0, AIRFLOW_SOURCES_DIR)
-# flake8: noqa: F401
+from common_precommit_utils import AIRFLOW_SOURCES_ROOT_PATH, read_airflow_version  # noqa: E402
 
-from setup import version  # isort:skip
 
-
-def update_version(pattern: re.Pattern, v: str, file_path: str):
+def update_version(pattern: re.Pattern, v: str, file_path: Path):
     print(f"Checking {pattern} in {file_path}")
-    with open(file_path, "r+") as f:
+    with file_path.open("r+") as f:
         file_content = f.read()
         if not pattern.search(file_content):
             raise Exception(f"Pattern {pattern!r} doesn't found in {file_path!r} file")
@@ -53,13 +48,14 @@ REPLACEMENTS = {
     r"(\(Assuming Airflow version `).*(`\))": "docs/docker-stack/README.md",
 }
 
-print(f"Current version: {version}")
 
 if __name__ == "__main__":
+    version = read_airflow_version()
+    print(f"Current version: {version}")
     for regexp, p in REPLACEMENTS.items():
         text_pattern = re.compile(regexp, flags=re.MULTILINE)
-        files = glob.glob(join(AIRFLOW_SOURCES_DIR, p), recursive=True)
+        files = list(AIRFLOW_SOURCES_ROOT_PATH.glob(p))
         if not files:
             print(f"ERROR! No files matched on {p}")
-        for file in glob.glob(join(AIRFLOW_SOURCES_DIR, p), recursive=True):
+        for file in files:
             update_version(text_pattern, version, file)
diff --git a/scripts/ci/pre_commit/pre_commit_version_heads_map.py b/scripts/ci/pre_commit/pre_commit_version_heads_map.py
index f935d6d395..5ae542b1c1 100755
--- a/scripts/ci/pre_commit/pre_commit_version_heads_map.py
+++ b/scripts/ci/pre_commit/pre_commit_version_heads_map.py
@@ -28,7 +28,9 @@ PROJECT_SOURCE_ROOT_DIR = Path(__file__).resolve().parent.parent.parent.parent
 
 DB_FILE = PROJECT_SOURCE_ROOT_DIR / "airflow" / "utils" / "db.py"
 
-SETUP_FILE = PROJECT_SOURCE_ROOT_DIR / "setup.py"
+sys.path.insert(0, str(Path(__file__).parent.resolve()))  # make sure common_precommit_utils is importable
+
+from common_precommit_utils import read_airflow_version  # noqa: E402
 
 
 def read_revision_heads_map():
@@ -45,18 +47,8 @@ def read_revision_heads_map():
     return revision_heads_map.keys()
 
 
-def read_current_airflow_version():
-
-    ast_obj = ast.parse(open(SETUP_FILE).read())
-    assignments = [a for a in ast_obj.body if isinstance(a, ast.Assign)][:10]
-
-    version = [x for x in assignments if x.targets[0].id == "version"][0]
-
-    return Version(ast.literal_eval(version.value))
-
-
 if __name__ == "__main__":
-    airflow_version = read_current_airflow_version()
+    airflow_version = Version(read_airflow_version())
     if airflow_version.is_devrelease or "b" in (airflow_version.pre or ()):
         exit(0)
     versions = read_revision_heads_map()
diff --git a/scripts/in_container/run_migration_reference.py b/scripts/in_container/run_migration_reference.py
index 0f17c45afb..43692b2c45 100755
--- a/scripts/in_container/run_migration_reference.py
+++ b/scripts/in_container/run_migration_reference.py
@@ -30,13 +30,13 @@ from typing import TYPE_CHECKING, Iterable
 from alembic.script import ScriptDirectory
 from tabulate import tabulate
 
+from airflow import __version__ as airflow_version
 from airflow.utils.db import _get_alembic_config
-from setup import version as _airflow_version
 
 if TYPE_CHECKING:
     from alembic.script import Script
 
-airflow_version = re.match(r"(\d+\.\d+\.\d+).*", _airflow_version).group(1)  # type: ignore
+airflow_version = re.match(r"(\d+\.\d+\.\d+).*", airflow_version).group(1)  # type: ignore
 project_root = Path(__file__).parents[2].resolve()
 
 
diff --git a/setup.cfg b/setup.cfg
index 34fdf014df..e55108fb92 100644
--- a/setup.cfg
+++ b/setup.cfg
@@ -21,6 +21,7 @@ summary = Programmatically author, schedule and monitor data pipelines
 author = Apache Software Foundation
 author_email = dev@airflow.apache.org
 url = https://airflow.apache.org/
+version = attr: airflow.__version__
 long_description = file: README.md
 long_description_content_type = text/markdown
 license = Apache License 2.0
@@ -43,6 +44,7 @@ classifiers =
     Framework :: Apache Airflow
 project_urls =
     Documentation=https://airflow.apache.org/docs/
+    Downloads=https://archive.apache.org/dist/airflow/
     Bug Tracker=https://github.com/apache/airflow/issues
     Source Code=https://github.com/apache/airflow
     Slack Chat=https://s.apache.org/airflow-slack
diff --git a/setup.py b/setup.py
index 74f205c0c9..5ed4bb756f 100644
--- a/setup.py
+++ b/setup.py
@@ -49,8 +49,6 @@ PY39 = sys.version_info >= (3, 9)
 
 logger = logging.getLogger(__name__)
 
-version = "2.5.1"
-
 AIRFLOW_SOURCES_ROOT = Path(__file__).parent.resolve()
 PROVIDERS_ROOT = AIRFLOW_SOURCES_ROOT / "airflow" / "providers"
 
@@ -162,7 +160,7 @@ class ListExtras(Command):
         print("\n".join(wrap(", ".join(EXTRAS_DEPENDENCIES.keys()), 100)))
 
 
-def git_version(version_: str) -> str:
+def git_version() -> str:
     """
     Return a version to identify the state of the underlying git repo. The version will
     indicate whether the head of the current git-backed working directory is tied to a
@@ -171,7 +169,6 @@ def git_version(version_: str) -> str:
     branch head. Finally, a "dirty" suffix is appended to indicate that uncommitted
     changes are present.
 
-    :param str version_: Semver version
     :return: Found Airflow version in Git repo
     """
     try:
@@ -179,7 +176,7 @@ def git_version(version_: str) -> str:
 
         try:
             repo = git.Repo(str(AIRFLOW_SOURCES_ROOT / ".git"))
-        except (git.NoSuchPathError):
+        except git.NoSuchPathError:
             logger.warning(".git directory not found: Cannot compute the git version")
             return ""
         except git.InvalidGitRepositoryError:
@@ -193,7 +190,7 @@ def git_version(version_: str) -> str:
         if repo.is_dirty():
             return f".dev0+{sha}.dirty"
         # commit is clean
-        return f".release:{version_}+{sha}"
+        return f".release:{sha}"
     return "no_git_version"
 
 
@@ -203,7 +200,7 @@ def write_version(filename: str = str(AIRFLOW_SOURCES_ROOT / "airflow" / "git_ve
 
     :param str filename: Destination file to write.
     """
-    text = f"{git_version(version)}"
+    text = git_version()
     with open(filename, "w") as file:
         file.write(text)
 
@@ -484,12 +481,12 @@ EXTRAS_DEPENDENCIES: dict[str, list[str]] = deepcopy(CORE_EXTRAS_DEPENDENCIES)
 
 
 def add_extras_for_all_providers() -> None:
-    for (provider_name, provider_dict) in PROVIDER_DEPENDENCIES.items():
+    for provider_name, provider_dict in PROVIDER_DEPENDENCIES.items():
         EXTRAS_DEPENDENCIES[provider_name] = provider_dict[DEPS]
 
 
 def add_additional_extras() -> None:
-    for (extra_name, extra_dependencies) in ADDITIONAL_EXTRAS_DEPENDENCIES.items():
+    for extra_name, extra_dependencies in ADDITIONAL_EXTRAS_DEPENDENCIES.items():
         EXTRAS_DEPENDENCIES[extra_name] = extra_dependencies
 
 
@@ -904,9 +901,7 @@ def do_setup() -> None:
     write_version()
     setup(
         distclass=AirflowDistribution,
-        version=version,
         extras_require=EXTRAS_DEPENDENCIES,
-        download_url=("https://archive.apache.org/dist/airflow/" + version),
         cmdclass={
             "extra_clean": CleanCommand,
             "compile_assets": CompileAssets,