You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@airflow.apache.org by po...@apache.org on 2023/02/13 23:51:28 UTC

[airflow] branch main updated: Fix circular imports when airflow starts (#29494)

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

potiuk pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/airflow.git


The following commit(s) were added to refs/heads/main by this push:
     new 47b67f13da Fix circular imports when airflow starts (#29494)
47b67f13da is described below

commit 47b67f13da667de4eb69233404d4d5a33574dd58
Author: Jarek Potiuk <ja...@potiuk.com>
AuthorDate: Tue Feb 14 00:51:20 2023 +0100

    Fix circular imports when airflow starts (#29494)
    
    * Fix circular imports when airflow starts from scratch
    
    When airflow CLI is started without generated config file the CLI
    fails with circular imports. Some of the util classes imported
    during such start were importing "constants" (in fact variables)
    from the setttings and this happened before settings were fully
    initialized.
    
    This PR moves all relevant top-level imports to be local imports
    and moves initialization of the State class with settings-defined
    colors to initialization() method in order to avoid those circular
    imports.
    
    * Update airflow/models/xcom.py
    
    Co-authored-by: Tzu-ping Chung <ur...@gmail.com>
    
    
    ---------
    
    Co-authored-by: Tzu-ping Chung <ur...@gmail.com>
---
 airflow/models/xcom.py                |  5 ++++-
 airflow/settings.py                   |  5 ++++-
 airflow/utils/log/logging_mixin.py    |  4 ++--
 airflow/utils/state.py                |  3 ---
 airflow/utils/timezone.py             | 12 ++++++++++--
 tests/cli/commands/test_db_command.py |  4 ++--
 tests/models/test_taskinstance.py     |  5 +++--
 7 files changed, 25 insertions(+), 13 deletions(-)

diff --git a/airflow/models/xcom.py b/airflow/models/xcom.py
index cc76c0339c..1ecd21f405 100644
--- a/airflow/models/xcom.py
+++ b/airflow/models/xcom.py
@@ -772,7 +772,10 @@ class LazyXComAccess(collections.abc.Sequence):
             yield self._query
             return
 
-        session = settings.Session()
+        Session = getattr(settings, "Session", None)
+        if Session is None:
+            raise RuntimeError("Session must be set before!")
+        session = Session()
         try:
             yield self._query.with_session(session)
         finally:
diff --git a/airflow/settings.py b/airflow/settings.py
index ed5fe816c4..fd62391750 100644
--- a/airflow/settings.py
+++ b/airflow/settings.py
@@ -41,6 +41,7 @@ from airflow.exceptions import RemovedInAirflow3Warning
 from airflow.executors import executor_constants
 from airflow.logging_config import configure_logging
 from airflow.utils.orm_event_handlers import setup_event_handlers
+from airflow.utils.state import State
 
 if TYPE_CHECKING:
     from airflow.www.utils import UIAlert
@@ -351,7 +352,7 @@ def dispose_orm():
     global engine
     global Session
 
-    if Session is not None:
+    if Session is not None:  # type: ignore[truthy-function]
         Session.remove()
         Session = None
     if engine:
@@ -518,6 +519,8 @@ def initialize():
     import_local_settings()
     global LOGGING_CLASS_PATH
     LOGGING_CLASS_PATH = configure_logging()
+    State.state_color.update(STATE_COLORS)
+
     configure_adapters()
     # The webservers import this file from models.py with the default settings.
     configure_orm()
diff --git a/airflow/utils/log/logging_mixin.py b/airflow/utils/log/logging_mixin.py
index 79746c8492..5e650ea5be 100644
--- a/airflow/utils/log/logging_mixin.py
+++ b/airflow/utils/log/logging_mixin.py
@@ -26,8 +26,6 @@ from io import IOBase
 from logging import Handler, Logger, StreamHandler
 from typing import IO, Any, TypeVar, cast
 
-from airflow.settings import IS_K8S_EXECUTOR_POD
-
 # 7-bit C1 ANSI escape sequences
 ANSI_ESCAPE = re.compile(r"\x1B[@-_][0-?]*[ -/]*[@-~]")
 
@@ -203,6 +201,8 @@ class RedirectStdHandler(StreamHandler):
     @property
     def stream(self):
         """Returns current stream."""
+        from airflow.settings import IS_K8S_EXECUTOR_POD
+
         if IS_K8S_EXECUTOR_POD:
             return self._orig_stream
         if self._use_stderr:
diff --git a/airflow/utils/state.py b/airflow/utils/state.py
index 6558af9f2a..6f89174bd6 100644
--- a/airflow/utils/state.py
+++ b/airflow/utils/state.py
@@ -19,8 +19,6 @@ from __future__ import annotations
 
 from enum import Enum
 
-from airflow.settings import STATE_COLORS
-
 
 class TaskInstanceState(str, Enum):
     """
@@ -122,7 +120,6 @@ class State:
         TaskInstanceState.SCHEDULED: "tan",
         TaskInstanceState.DEFERRED: "mediumpurple",
     }
-    state_color.update(STATE_COLORS)  # type: ignore
 
     @classmethod
     def color(cls, state):
diff --git a/airflow/utils/timezone.py b/airflow/utils/timezone.py
index 062e7ad955..bbe38aba2e 100644
--- a/airflow/utils/timezone.py
+++ b/airflow/utils/timezone.py
@@ -24,8 +24,6 @@ import pendulum
 from dateutil.relativedelta import relativedelta
 from pendulum.datetime import DateTime
 
-from airflow.settings import TIMEZONE
-
 # UTC time zone as a tzinfo instance.
 utc = pendulum.tz.timezone("UTC")
 
@@ -104,6 +102,8 @@ def convert_to_utc(value: dt.datetime | None) -> DateTime | None:
         return value
 
     if not is_localized(value):
+        from airflow.settings import TIMEZONE
+
         value = pendulum.instance(value, TIMEZONE)
 
     return pendulum.instance(value.astimezone(utc))
@@ -133,6 +133,8 @@ def make_aware(value: dt.datetime | None, timezone: dt.tzinfo | None = None) ->
     :return: localized datetime in settings.TIMEZONE or timezone
     """
     if timezone is None:
+        from airflow.settings import TIMEZONE
+
         timezone = TIMEZONE
 
     if not value:
@@ -168,6 +170,8 @@ def make_naive(value, timezone=None):
     :return: naive datetime
     """
     if timezone is None:
+        from airflow.settings import TIMEZONE
+
         timezone = TIMEZONE
 
     # Emulate the behavior of astimezone() on Python < 3.6.
@@ -191,6 +195,8 @@ def datetime(*args, **kwargs):
     :return: datetime.datetime
     """
     if "tzinfo" not in kwargs:
+        from airflow.settings import TIMEZONE
+
         kwargs["tzinfo"] = TIMEZONE
 
     return dt.datetime(*args, **kwargs)
@@ -203,6 +209,8 @@ def parse(string: str, timezone=None) -> DateTime:
     :param string: time string
     :param timezone: the timezone
     """
+    from airflow.settings import TIMEZONE
+
     return pendulum.parse(string, tz=timezone or TIMEZONE, strict=False)  # type: ignore
 
 
diff --git a/tests/cli/commands/test_db_command.py b/tests/cli/commands/test_db_command.py
index b25e465b43..eefa88a1cc 100644
--- a/tests/cli/commands/test_db_command.py
+++ b/tests/cli/commands/test_db_command.py
@@ -285,7 +285,7 @@ class TestCLIDBClean:
         coerced to tz-aware with default timezone
         """
         timestamp = "2021-01-01 00:00:00"
-        with patch("airflow.utils.timezone.TIMEZONE", pendulum.timezone(timezone)):
+        with patch("airflow.settings.TIMEZONE", pendulum.timezone(timezone)):
             args = self.parser.parse_args(["db", "clean", "--clean-before-timestamp", f"{timestamp}", "-y"])
             db_command.cleanup_tables(args)
         run_cleanup_mock.assert_called_once_with(
@@ -304,7 +304,7 @@ class TestCLIDBClean:
         When tz included in the string then default timezone should not be used.
         """
         timestamp = "2021-01-01 00:00:00+03:00"
-        with patch("airflow.utils.timezone.TIMEZONE", pendulum.timezone(timezone)):
+        with patch("airflow.settings.TIMEZONE", pendulum.timezone(timezone)):
             args = self.parser.parse_args(["db", "clean", "--clean-before-timestamp", f"{timestamp}", "-y"])
             db_command.cleanup_tables(args)
 
diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py
index 93f5123b5d..a388b0c0d0 100644
--- a/tests/models/test_taskinstance.py
+++ b/tests/models/test_taskinstance.py
@@ -70,6 +70,7 @@ from airflow.operators.python import PythonOperator
 from airflow.sensors.base import BaseSensorOperator
 from airflow.sensors.python import PythonSensor
 from airflow.serialization.serialized_objects import SerializedBaseOperator
+from airflow.settings import TIMEZONE
 from airflow.stats import Stats
 from airflow.ti_deps.dep_context import DepContext
 from airflow.ti_deps.dependencies_deps import REQUEUEABLE_DEPS, RUNNING_DEPS
@@ -2307,13 +2308,13 @@ class TestTaskInstance:
         )
         context = ti.get_template_context()
         with pytest.deprecated_call():
-            assert context["execution_date"] == pendulum.DateTime(2021, 9, 6, tzinfo=timezone.TIMEZONE)
+            assert context["execution_date"] == pendulum.DateTime(2021, 9, 6, tzinfo=TIMEZONE)
         with pytest.deprecated_call():
             assert context["next_ds"] == "2021-09-07"
         with pytest.deprecated_call():
             assert context["next_ds_nodash"] == "20210907"
         with pytest.deprecated_call():
-            assert context["next_execution_date"] == pendulum.DateTime(2021, 9, 7, tzinfo=timezone.TIMEZONE)
+            assert context["next_execution_date"] == pendulum.DateTime(2021, 9, 7, tzinfo=TIMEZONE)
         with pytest.deprecated_call():
             assert context["prev_ds"] is None, "Does not make sense for custom timetable"
         with pytest.deprecated_call():