From e1f721df98e476a0b7379cd246671c97cbc64fb7 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 12 Nov 2024 18:02:58 +0200 Subject: [PATCH 01/52] trace and span creation with context propagation --- airflow/cli/cli_config.py | 4 + airflow/cli/commands/task_command.py | 6 + airflow/config_templates/config.yml | 10 + airflow/executors/base_executor.py | 31 ++ airflow/jobs/scheduler_job_runner.py | 55 ++-- airflow/metrics/validators.py | 3 +- airflow/models/dagrun.py | 166 ++++++++-- airflow/models/taskinstance.py | 48 +++ airflow/models/taskinstancehistory.py | 1 + airflow/serialization/pydantic/dag_run.py | 1 + .../serialization/pydantic/taskinstance.py | 1 + airflow/traces/otel_tracer.py | 175 +++++++++- airflow/traces/tracer.py | 58 ++++ airflow/utils/sqlalchemy.py | 6 +- airflow/utils/thread_safe_dict.py | 45 +++ .../logging-monitoring/traces.rst | 1 + tests/core/test_otel_tracer.py | 52 +++ tests/integration/otel/__init__.py | 17 + tests/integration/otel/dags/__init__.py | 17 + tests/integration/otel/dags/otel_test_dag.py | 104 ++++++ tests/integration/otel/test_otel.py | 298 ++++++++++++++++++ tests/integration/otel/test_utils.py | 273 ++++++++++++++++ tests/models/test_taskinstance.py | 1 + 23 files changed, 1305 insertions(+), 68 deletions(-) create mode 100644 airflow/utils/thread_safe_dict.py create mode 100644 tests/integration/otel/__init__.py create mode 100644 tests/integration/otel/dags/__init__.py create mode 100644 tests/integration/otel/dags/otel_test_dag.py create mode 100644 tests/integration/otel/test_otel.py create mode 100644 tests/integration/otel/test_utils.py diff --git a/airflow/cli/cli_config.py b/airflow/cli/cli_config.py index e93d5e25c6318..3621a7bebf4e9 100644 --- a/airflow/cli/cli_config.py +++ b/airflow/cli/cli_config.py @@ -551,6 +551,9 @@ def string_lower_type(val): action="store_true", ) ARG_RAW = Arg(("-r", "--raw"), argparse.SUPPRESS, "store_true") +ARG_CARRIER = Arg( + ("-c", "--carrier"), help="Context Carrier, containing the injected context for the task span", nargs="?" +) ARG_IGNORE_ALL_DEPENDENCIES = Arg( ("-A", "--ignore-all-dependencies"), help="Ignores all non-critical dependencies, including ignore_ti_state and ignore_task_deps", @@ -1334,6 +1337,7 @@ class GroupCommand(NamedTuple): ARG_CFG_PATH, ARG_LOCAL, ARG_RAW, + ARG_CARRIER, ARG_IGNORE_ALL_DEPENDENCIES, ARG_IGNORE_DEPENDENCIES, ARG_DEPENDS_ON_PAST, diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index e14c18399555a..30baa30ae77db 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -445,6 +445,12 @@ def task_run(args, dag: DAG | None = None) -> TaskReturnCode | None: log.info("Running %s on host %s", ti, hostname) + if args.carrier is not None: + log.info("Found args.carrier: %s. Setting the value in the ti instance.", args.carrier) + # The arg value is a dict string, and it needs to be converted back to a dict. + carrier_dict = json.loads(args.carrier) + ti.set_context_carrier(carrier_dict, with_commit=True) + if not InternalApiConfig.get_use_internal_api(): # IMPORTANT, have to re-configure ORM with the NullPool, otherwise, each "run" command may leave # behind multiple open sleeping connections while heartbeating, which could diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 4d58de3c37e28..7726134bb90cb 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -1291,6 +1291,16 @@ traces: type: string example: ~ default: "False" + otel_use_context_propagation: + description: | + If True, a separate trace with a new set of spans will be created for each dag run. + All the children spans will be created using context propagation. + This doesn't replace any of the existing spans, but it creates additional ones. + The default value is False. + version_added: 2.10.2 + type: string + example: ~ + default: "False" secrets: description: ~ options: diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index fba6d96969a13..12e0639d07c2e 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -18,6 +18,7 @@ from __future__ import annotations +import json import logging import sys from collections import defaultdict, deque @@ -32,10 +33,12 @@ from airflow.models import Log from airflow.stats import Stats from airflow.traces import NO_TRACE_ID +from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import Trace, add_span, gen_context from airflow.traces.utils import gen_span_id_from_ti_key, gen_trace_id from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import TaskInstanceState +from airflow.utils.thread_safe_dict import ThreadSafeDict PARALLELISM: int = conf.getint("core", "PARALLELISM") @@ -111,6 +114,8 @@ class BaseExecutor(LoggingMixin): :param parallelism: how many jobs should run at one time. Set to ``0`` for infinity. """ + active_spans = ThreadSafeDict() + supports_ad_hoc_ti_run: bool = False supports_sentry: bool = False @@ -125,6 +130,8 @@ class BaseExecutor(LoggingMixin): name: None | ExecutorName = None callback_sink: BaseCallbackSink | None = None + otel_use_context_propagation = conf.getboolean("traces", "otel_use_context_propagation") + def __init__(self, parallelism: int = PARALLELISM): super().__init__() self.parallelism: int = parallelism @@ -330,6 +337,30 @@ def trigger_tasks(self, open_slots: int) -> None: for _ in range(min((open_slots, len(self.queued_tasks)))): key, (command, _, queue, ti) = sorted_queue.pop(0) + if self.otel_use_context_propagation: + # If it's None, then the span for the current TaskInstanceKey hasn't been started. + if self.active_spans.get(key) is None: + parent_context = Trace.extract(ti.dag_run.context_carrier) + # Start a new span using the context from the parent. + # Attributes will be set once the task has finished so that all + # values will be available (end_time, duration, etc.). + span = Trace.start_child_span( + span_name=f"{ti.task_id}{CTX_PROP_SUFFIX}", + parent_context=parent_context, + component=f"task{CTX_PROP_SUFFIX}", + start_as_current=False, + ) + self.active_spans.set(key, span) + # Inject the current context into the carrier. + carrier = Trace.inject() + # The carrier needs to be set on the ti, but it can't happen here because db calls are expensive. + # By the time the db update has finished, another heartbeat will have started + # and the tasks will have been triggered again. + # So set the carrier as an argument to the command. + # The command execution will set it on the ti, and it will be propagated to the task itself. + command.append("--carrier") + command.append(json.dumps(carrier)) + # If a task makes it here but is still understood by the executor # to be running, it generally means that the task has been killed # externally and not yet been marked as failed. diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index ee101eaac44a1..07175eb732d1f 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -797,6 +797,13 @@ def process_executor_events( ti.pid, ) + active_ti_span = executor.active_spans.get(ti.key) + if conf.getboolean("traces", "otel_use_context_propagation") and active_ti_span is not None: + cls._set_span_attrs__process_executor_events(span=active_ti_span, state=state, ti=ti) + # End the span and remove it from the active_spans dict. + active_ti_span.end() + executor.active_spans.delete(ti.key) + with Trace.start_span_from_taskinstance(ti=ti) as span: cls._set_span_attrs__process_executor_events(span, state, ti) if conf.has_option("traces", "otel_task_log_event") and conf.getboolean( @@ -884,32 +891,32 @@ def process_executor_events( @classmethod def _set_span_attrs__process_executor_events(cls, span, state, ti): - span.set_attribute("category", "scheduler") - span.set_attribute("task_id", ti.task_id) - span.set_attribute("dag_id", ti.dag_id) - span.set_attribute("state", ti.state) + span.set_attribute("airflow.category", "scheduler") + span.set_attribute("airflow.task.task_id", ti.task_id) + span.set_attribute("airflow.task.dag_id", ti.dag_id) + span.set_attribute("airflow.task.state", ti.state) if ti.state == TaskInstanceState.FAILED: - span.set_attribute("error", True) - span.set_attribute("start_date", str(ti.start_date)) - span.set_attribute("end_date", str(ti.end_date)) - span.set_attribute("duration", ti.duration) - span.set_attribute("executor_config", str(ti.executor_config)) - span.set_attribute("execution_date", str(ti.execution_date)) - span.set_attribute("hostname", ti.hostname) - span.set_attribute("log_url", ti.log_url) - span.set_attribute("operator", str(ti.operator)) - span.set_attribute("try_number", ti.try_number) - span.set_attribute("executor_state", state) - span.set_attribute("pool", ti.pool) - span.set_attribute("queue", ti.queue) - span.set_attribute("priority_weight", ti.priority_weight) - span.set_attribute("queued_dttm", str(ti.queued_dttm)) - span.set_attribute("queued_by_job_id", ti.queued_by_job_id) - span.set_attribute("pid", ti.pid) + span.set_attribute("airflow.task.error", True) + span.set_attribute("airflow.task.start_date", str(ti.start_date)) + span.set_attribute("airflow.task.end_date", str(ti.end_date)) + span.set_attribute("airflow.task.duration", ti.duration) + span.set_attribute("airflow.task.executor_config", str(ti.executor_config)) + span.set_attribute("airflow.task.execution_date", str(ti.execution_date)) + span.set_attribute("airflow.task.hostname", ti.hostname) + span.set_attribute("airflow.task.log_url", ti.log_url) + span.set_attribute("airflow.task.operator", str(ti.operator)) + span.set_attribute("airflow.task.try_number", ti.try_number) + span.set_attribute("airflow.task.executor_state", state) + span.set_attribute("airflow.task.pool", ti.pool) + span.set_attribute("airflow.task.queue", ti.queue) + span.set_attribute("airflow.task.priority_weight", ti.priority_weight) + span.set_attribute("airflow.task.queued_dttm", str(ti.queued_dttm)) + span.set_attribute("airflow.task.queued_by_job_id", ti.queued_by_job_id) + span.set_attribute("airflow.task.pid", ti.pid) if span.is_recording(): - span.add_event(name="queued", timestamp=datetime_to_nano(ti.queued_dttm)) - span.add_event(name="started", timestamp=datetime_to_nano(ti.start_date)) - span.add_event(name="ended", timestamp=datetime_to_nano(ti.end_date)) + span.add_event(name="airflow.task.queued", timestamp=datetime_to_nano(ti.queued_dttm)) + span.add_event(name="airflow.task.started", timestamp=datetime_to_nano(ti.start_date)) + span.add_event(name="airflow.task.ended", timestamp=datetime_to_nano(ti.end_date)) def _execute(self) -> int | None: from airflow.dag_processing.manager import DagFileProcessorAgent diff --git a/airflow/metrics/validators.py b/airflow/metrics/validators.py index b47cdac1be551..d851db82b9944 100644 --- a/airflow/metrics/validators.py +++ b/airflow/metrics/validators.py @@ -82,7 +82,8 @@ class MetricNameLengthExemptionWarning(Warning): } BACK_COMPAT_METRIC_NAMES: set[Pattern[str]] = {re2.compile(name) for name in BACK_COMPAT_METRIC_NAME_PATTERNS} -OTEL_NAME_MAX_LENGTH = 63 +# According to otel spec, max length should be 255. Change if the spec gets revised. +OTEL_NAME_MAX_LENGTH = 255 DEFAULT_VALIDATOR_TYPE = "allow" diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 635cd73ccd8d7..5612139e56bfe 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -45,6 +45,7 @@ ) from sqlalchemy.exc import IntegrityError from sqlalchemy.ext.associationproxy import association_proxy +from sqlalchemy.ext.mutable import MutableDict from sqlalchemy.orm import declared_attr, joinedload, relationship, synonym, validates from sqlalchemy.sql.expression import case, false, select, true from sqlalchemy.sql.functions import coalesce @@ -67,6 +68,7 @@ from airflow.stats import Stats from airflow.ti_deps.dep_context import DepContext from airflow.ti_deps.dependencies_states import SCHEDULEABLE_STATES +from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import Trace from airflow.utils import timezone from airflow.utils.dates import datetime_to_nano @@ -74,13 +76,21 @@ from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.retries import retry_db_transaction from airflow.utils.session import NEW_SESSION, provide_session -from airflow.utils.sqlalchemy import UtcDateTime, nulls_first, tuple_in_condition, with_row_locks +from airflow.utils.sqlalchemy import ( + ExtendedJSON, + UtcDateTime, + nulls_first, + tuple_in_condition, + with_row_locks, +) from airflow.utils.state import DagRunState, State, TaskInstanceState +from airflow.utils.thread_safe_dict import ThreadSafeDict from airflow.utils.types import NOTSET, DagRunTriggeredByType, DagRunType if TYPE_CHECKING: from datetime import datetime + from opentelemetry.sdk.trace import Span from sqlalchemy.orm import Query, Session from airflow.models.dag import DAG @@ -124,6 +134,8 @@ class DagRun(Base, LoggingMixin): external trigger (i.e. manual runs). """ + active_spans = ThreadSafeDict() + __tablename__ = "dag_run" id = Column(Integer, primary_key=True) @@ -168,6 +180,9 @@ class DagRun(Base, LoggingMixin): dag_version_id = Column(UUIDType(binary=False), ForeignKey("dag_version.id", ondelete="CASCADE")) dag_version = relationship("DagVersion", back_populates="dag_runs") + # Span context carrier, used for context propagation. + context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) + # Remove this `if` after upgrading Sphinx-AutoAPI if not TYPE_CHECKING and "BUILDING_AIRFLOW_DOCS" in os.environ: dag: DAG | None @@ -223,6 +238,8 @@ class DagRun(Base, LoggingMixin): fallback=20, ) + otel_use_context_propagation = airflow_conf.getboolean("traces", "otel_use_context_propagation") + def __init__( self, dag_id: str | None = None, @@ -264,6 +281,7 @@ def __init__( self.clear_number = 0 self.triggered_by = triggered_by self.dag_version = dag_version + self.context_carrier = {} super().__init__() def __repr__(self): @@ -853,6 +871,41 @@ def is_effective_leaf(task): leaf_tis = {ti for ti in tis if ti.task_id in leaf_task_ids if ti.state != TaskInstanceState.REMOVED} return leaf_tis + @staticmethod + def _set_dagrun_span_attrs(span: Span, dag_run: DagRun): + # This is necessary to avoid an error in case of testing a paused dag. + if dag_run.queued_at is None and dag_run.start_date is not None: + dag_run.queued_at = dag_run.start_date + + if dag_run._state is DagRunState.FAILED: + span.set_attribute("airflow.dag_run.error", True) + attributes = { + "airflow.category": "DAG runs", + "airflow.dag_run.dag_id": str(dag_run.dag_id), + "airflow.dag_run.execution_date": str(dag_run.execution_date), + "airflow.dag_run.run_id": str(dag_run.run_id), + "airflow.dag_run.queued_at": str(dag_run.queued_at), + "airflow.dag_run.run_start_date": str(dag_run.start_date), + "airflow.dag_run.run_end_date": str(dag_run.end_date), + "airflow.dag_run.run_duration": str( + (dag_run.end_date - dag_run.start_date).total_seconds() + if dag_run.start_date and dag_run.end_date + else 0 + ), + "airflow.dag_run.state": str(dag_run._state), + "airflow.dag_run.external_trigger": str(dag_run.external_trigger), + "airflow.dag_run.run_type": str(dag_run.run_type), + "airflow.dag_run.data_interval_start": str(dag_run.data_interval_start), + "airflow.dag_run.data_interval_end": str(dag_run.data_interval_end), + "airflow.dag_run.dag_hash": str(dag_run.dag_hash), + "airflow.dag_run.conf": str(dag_run.conf), + } + if span.is_recording(): + span.add_event(name="airflow.dag_run.queued", timestamp=datetime_to_nano(dag_run.queued_at)) + span.add_event(name="airflow.dag_run.started", timestamp=datetime_to_nano(dag_run.start_date)) + span.add_event(name="airflow.dag_run.ended", timestamp=datetime_to_nano(dag_run.end_date)) + span.set_attributes(attributes) + @provide_session def update_state( self, session: Session = NEW_SESSION, execute_callbacks: bool = True @@ -990,6 +1043,23 @@ def recalculate(self) -> _UnfinishedStates: # finally, if the leaves aren't done, the dag is still running else: + # If there is no value in active_spans, then the span hasn't already been started. + if self.otel_use_context_propagation and (self.active_spans.get(self.run_id) is None): + span = Trace.start_root_span( + span_name=f"{self.dag_id}{CTX_PROP_SUFFIX}", + component=f"dag{CTX_PROP_SUFFIX}", + start_time=self.queued_at, + start_as_current=False, + ) + carrier = Trace.inject() + self.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) + # Set the span in a synchronized dictionary, so that the variable can be used to end the span. + self.active_spans.set(self.run_id, span) + self.log.debug( + "DagRun span has been started and the injected context_carrier is: %s", + self.context_carrier, + ) + self.set_state(DagRunState.RUNNING) if self._state == DagRunState.FAILED or self._state == DagRunState.SUCCESS: @@ -1020,35 +1090,31 @@ def recalculate(self) -> _UnfinishedStates: dagv.version if dagv else None, ) + if self.otel_use_context_propagation: + active_span = self.active_spans.get(self.run_id) + if active_span is not None: + self.log.debug( + "Found active span with span_id: %s, for dag_id: %s, run_id: %s, state: %s", + active_span.get_span_context().span_id, + self.dag_id, + self.run_id, + self.state, + ) + + self._set_dagrun_span_attrs(span=active_span, dag_run=self) + active_span.end() + # Remove the span from the dict. + self.active_spans.delete(self.run_id) + else: + self.log.debug( + "No active span has been found for dag_id: %s, run_id: %s, state: %s", + self.dag_id, + self.run_id, + self.state, + ) + with Trace.start_span_from_dagrun(dagrun=self) as span: - if self._state is DagRunState.FAILED: - span.set_attribute("error", True) - attributes = { - "category": "DAG runs", - "dag_id": str(self.dag_id), - "execution_date": str(self.execution_date), - "run_id": str(self.run_id), - "queued_at": str(self.queued_at), - "run_start_date": str(self.start_date), - "run_end_date": str(self.end_date), - "run_duration": str( - (self.end_date - self.start_date).total_seconds() - if self.start_date and self.end_date - else 0 - ), - "state": str(self._state), - "external_trigger": str(self.external_trigger), - "run_type": str(self.run_type), - "data_interval_start": str(self.data_interval_start), - "data_interval_end": str(self.data_interval_end), - "dag_version": str(dagv.version if dagv else None), - "conf": str(self.conf), - } - if span.is_recording(): - span.add_event(name="queued", timestamp=datetime_to_nano(self.queued_at)) - span.add_event(name="started", timestamp=datetime_to_nano(self.start_date)) - span.add_event(name="ended", timestamp=datetime_to_nano(self.end_date)) - span.set_attributes(attributes) + self._set_dagrun_span_attrs(span=span, dag_run=self) session.flush() @@ -1110,6 +1176,48 @@ def _filter_tis_and_exclude_removed(dag: DAG, tis: list[TI]) -> Iterable[TI]: finished_tis=finished_tis, ) + @staticmethod + @internal_api_call + def _set_context_carrier( + dag_run: DagRun, context_carrier: dict, session: Session, with_commit: bool + ) -> bool: + if not isinstance(dag_run, DagRun): + dag_run = session.scalars( + select(DagRun).where( + DagRun.dag_id == dag_run.dag_id, + DagRun.run_id == dag_run.run_id, + ) + ).one() + + if dag_run.context_carrier == context_carrier: + return False + + dag_run.log.debug("Setting dag_run context_carrier for run_id: %s", dag_run.run_id) + dag_run.context_carrier = context_carrier + + session.merge(dag_run) + + if with_commit: + session.commit() + + return True + + @provide_session + def set_context_carrier( + self, context_carrier: dict, session: Session = NEW_SESSION, with_commit: bool = False + ) -> bool: + """ + Set DagRun span context_carrier. + + :param context_carrier: dict with the injected carrier to set for the dag_run + :param session: SQLAlchemy ORM Session + :param with_commit: should the carrier be committed? + :return: has the context_carrier been changed? + """ + return self._set_context_carrier( + dag_run=self, context_carrier=context_carrier, session=session, with_commit=with_commit + ) + def notify_dagrun_state_changed(self, msg: str = ""): if self.state == DagRunState.RUNNING: get_listener_manager().hook.on_dag_run_running(dag_run=self, msg=msg) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index c05b1dd62ecaa..a9f9ad4e3d7df 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -1866,6 +1866,7 @@ class TaskInstance(Base, LoggingMixin): executor_config = Column(ExecutorConfigType(pickler=dill)) updated_at = Column(UtcDateTime, default=timezone.utcnow, onupdate=timezone.utcnow) rendered_map_index = Column(String(250)) + context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) external_executor_id = Column(StringID()) @@ -1980,6 +1981,7 @@ def __init__( self.raw = False # can be changed when calling 'run' self.test_mode = False + self.context_carrier = {} def __hash__(self): return hash((self.task_id, self.dag_id, self.run_id, self.map_index)) @@ -2374,6 +2376,49 @@ def set_state(self, state: str | None, session: Session = NEW_SESSION) -> bool: """ return self._set_state(ti=self, state=state, session=session) + @staticmethod + @internal_api_call + def _set_context_carrier( + ti: TaskInstance | TaskInstancePydantic, context_carrier, session: Session, with_commit: bool + ) -> bool: + if not isinstance(ti, TaskInstance): + ti = session.scalars( + select(TaskInstance).where( + TaskInstance.task_id == ti.task_id, + TaskInstance.dag_id == ti.dag_id, + TaskInstance.run_id == ti.run_id, + ) + ).one() + + if ti.context_carrier == context_carrier: + return False + + ti.log.debug("Setting task context_carrier for %s", ti.task_id) + ti.context_carrier = context_carrier + + session.merge(ti) + + if with_commit: + session.commit() + + return True + + @provide_session + def set_context_carrier( + self, context_carrier: dict, session: Session = NEW_SESSION, with_commit: bool = False + ) -> bool: + """ + Set TaskInstance span context_carrier. + + :param context_carrier: dict with the injected carrier to set for the TI + :param session: SQLAlchemy ORM Session + :param with_commit: should the carrier be committed? + :return: has the context_carrier been changed? + """ + return self._set_context_carrier( + ti=self, context_carrier=context_carrier, session=session, with_commit=with_commit + ) + @property def is_premature(self) -> bool: """Returns whether a task is in UP_FOR_RETRY state and its retry interval has elapsed.""" @@ -3855,6 +3900,7 @@ def __init__( key: TaskInstanceKey, run_as_user: str | None = None, priority_weight: int | None = None, + context_carrier: dict | None = None, ): self.dag_id = dag_id self.task_id = task_id @@ -3871,6 +3917,7 @@ def __init__( self.priority_weight = priority_weight self.queue = queue self.key = key + self.context_carrier = context_carrier def __repr__(self) -> str: attrs = ", ".join(f"{k}={v!r}" for k, v in self.__dict__.items()) @@ -3899,6 +3946,7 @@ def from_ti(cls, ti: TaskInstance) -> SimpleTaskInstance: key=ti.key, run_as_user=ti.run_as_user if hasattr(ti, "run_as_user") else None, priority_weight=ti.priority_weight if hasattr(ti, "priority_weight") else None, + context_carrier=ti.context_carrier if hasattr(ti, "context_carrier") else None, ) diff --git a/airflow/models/taskinstancehistory.py b/airflow/models/taskinstancehistory.py index e587cf083e3b5..f79f74eaf6535 100644 --- a/airflow/models/taskinstancehistory.py +++ b/airflow/models/taskinstancehistory.py @@ -84,6 +84,7 @@ class TaskInstanceHistory(Base): executor_config = Column(ExecutorConfigType(pickler=dill)) updated_at = Column(UtcDateTime, default=timezone.utcnow, onupdate=timezone.utcnow) rendered_map_index = Column(String(250)) + context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) external_executor_id = Column(StringID()) trigger_id = Column(Integer) diff --git a/airflow/serialization/pydantic/dag_run.py b/airflow/serialization/pydantic/dag_run.py index a0175e3749d9c..ba2aa8dc110d4 100644 --- a/airflow/serialization/pydantic/dag_run.py +++ b/airflow/serialization/pydantic/dag_run.py @@ -58,6 +58,7 @@ class DagRunPydantic(BaseModelPydantic): consumed_asset_events: List[AssetEventPydantic] # noqa: UP006 log_template_id: Optional[int] triggered_by: Optional[DagRunTriggeredByType] + context_carrier: Optional[dict] model_config = ConfigDict(from_attributes=True, arbitrary_types_allowed=True) diff --git a/airflow/serialization/pydantic/taskinstance.py b/airflow/serialization/pydantic/taskinstance.py index e8a1cfaf79a77..87fbce39c36da 100644 --- a/airflow/serialization/pydantic/taskinstance.py +++ b/airflow/serialization/pydantic/taskinstance.py @@ -123,6 +123,7 @@ class TaskInstancePydantic(BaseModelPydantic, LoggingMixin): dag_model: Optional[DagModelPydantic] raw: Optional[bool] is_trigger_log_context: Optional[bool] + context_carrier: Optional[dict] model_config = ConfigDict(from_attributes=True, arbitrary_types_allowed=True) @property diff --git a/airflow/traces/otel_tracer.py b/airflow/traces/otel_tracer.py index f85d3856ae7b3..3f9506a836c59 100644 --- a/airflow/traces/otel_tracer.py +++ b/airflow/traces/otel_tracer.py @@ -19,15 +19,18 @@ import logging import random +from typing import TYPE_CHECKING +import pendulum from opentelemetry import trace -from opentelemetry.context import create_key +from opentelemetry.context import attach, create_key from opentelemetry.exporter.otlp.proto.http.trace_exporter import OTLPSpanExporter from opentelemetry.sdk.resources import HOST_NAME, SERVICE_NAME, Resource from opentelemetry.sdk.trace import Span, Tracer as OpenTelemetryTracer, TracerProvider -from opentelemetry.sdk.trace.export import BatchSpanProcessor, ConsoleSpanExporter +from opentelemetry.sdk.trace.export import BatchSpanProcessor, ConsoleSpanExporter, SimpleSpanProcessor from opentelemetry.sdk.trace.id_generator import IdGenerator from opentelemetry.trace import Link, NonRecordingSpan, SpanContext, TraceFlags, Tracer +from opentelemetry.trace.propagation.tracecontext import TraceContextTextMapPropagator from opentelemetry.trace.span import INVALID_SPAN_ID, INVALID_TRACE_ID from airflow.configuration import conf @@ -45,10 +48,15 @@ from airflow.utils.dates import datetime_to_nano from airflow.utils.net import get_hostname +if TYPE_CHECKING: + from opentelemetry.context.context import Context + log = logging.getLogger(__name__) _NEXT_ID = create_key("next_id") +CTX_PROP_SUFFIX = "_ctx_prop" + class OtelTrace: """ @@ -57,29 +65,55 @@ class OtelTrace: When OTEL is enabled, the Trace class will be replaced by this class. """ - def __init__(self, span_exporter: OTLPSpanExporter, tag_string: str | None = None): + def __init__( + self, + span_exporter: OTLPSpanExporter, + tag_string: str | None = None, + use_simple_processor: bool = False, + ): self.span_exporter = span_exporter - self.span_processor = BatchSpanProcessor(self.span_exporter) + if use_simple_processor: + # With a BatchSpanProcessor, spans are exported at an interval. + # A task can run fast and finish before spans have enough time to get exported to the collector. + # When creating spans from inside a task, a SimpleSpanProcessor needs to be used because + # it exports the spans immediately after they are created. + self.span_processor = SimpleSpanProcessor(self.span_exporter) + else: + self.span_processor = BatchSpanProcessor(self.span_exporter) self.tag_string = tag_string self.otel_service = conf.get("traces", "otel_service") + self.resource = Resource.create( + attributes={HOST_NAME: get_hostname(), SERVICE_NAME: self.otel_service} + ) - def get_tracer( - self, component: str, trace_id: int | None = None, span_id: int | None = None - ) -> OpenTelemetryTracer | Tracer: - """Tracer that will use special AirflowOtelIdGenerator to control producing certain span and trace id.""" - resource = Resource.create(attributes={HOST_NAME: get_hostname(), SERVICE_NAME: self.otel_service}) + def get_otel_tracer_provider( + self, trace_id: int | None = None, span_id: int | None = None + ) -> TracerProvider: + """ + Tracer that will use special AirflowOtelIdGenerator to control producing certain span and trace id. + + It can be used to get a tracer and directly create spans, or for auto-instrumentation. + """ if trace_id or span_id: # in case where trace_id or span_id was given tracer_provider = TracerProvider( - resource=resource, id_generator=AirflowOtelIdGenerator(span_id=span_id, trace_id=trace_id) + resource=self.resource, + id_generator=AirflowOtelIdGenerator(span_id=span_id, trace_id=trace_id), ) else: - tracer_provider = TracerProvider(resource=resource) + tracer_provider = TracerProvider(resource=self.resource) debug = conf.getboolean("traces", "otel_debugging_on") if debug is True: log.info("[ConsoleSpanExporter] is being used") tracer_provider.add_span_processor(BatchSpanProcessor(ConsoleSpanExporter())) - tracer_provider.add_span_processor(self.span_processor) + else: + tracer_provider.add_span_processor(self.span_processor) + return tracer_provider + + def get_tracer( + self, component: str, trace_id: int | None = None, span_id: int | None = None + ) -> OpenTelemetryTracer | Tracer: + tracer_provider = self.get_otel_tracer_provider(trace_id=trace_id, span_id=span_id) tracer = tracer_provider.get_tracer(component) """ Tracer will produce a single ID value if value is provided. Note that this is one-time only, so any @@ -231,6 +265,116 @@ def start_span_from_taskinstance( ) return span + def start_root_span( + self, span_name: str, component: str | None = None, start_time=None, start_as_current: bool = True + ): + """Start a root span.""" + # If no context is passed to the new span, + # then it will try to get the context of the current active span. + # Due to that, the context parameter can't be empty. + # It needs an invalid context in order to declare the new span as root. + invalid_span_ctx = SpanContext( + trace_id=INVALID_TRACE_ID, span_id=INVALID_SPAN_ID, is_remote=True, trace_flags=TraceFlags(0x01) + ) + invalid_ctx = trace.set_span_in_context(NonRecordingSpan(invalid_span_ctx)) + + return self._new_span( + span_name=span_name, + parent_context=invalid_ctx, + component=component, + links=None, + start_time=start_time, + start_as_current=start_as_current, + ) + + def start_child_span( + self, + span_name: str, + parent_context: Context | None = None, + component: str | None = None, + links=None, + start_time=None, + start_as_current: bool = True, + ): + """Start a child span.""" + if parent_context is None: + # If no context is passed, then use the current. + parent_span_context = trace.get_current_span().get_span_context() + parent_context = trace.set_span_in_context(NonRecordingSpan(parent_span_context)) + else: + context_val = next(iter(parent_context.values())) + parent_span_context = None + if isinstance(context_val, NonRecordingSpan): + parent_span_context = context_val.get_span_context() + + _links = gen_links_from_kv_list(links) if links else [] + _links.append( + Link( + context=parent_span_context, + attributes={"meta.annotation_type": "link", "from": "parenttrace"}, + ) + ) + + return self._new_span( + span_name=span_name, + parent_context=parent_context, + component=component, + links=_links, + start_time=start_time, + start_as_current=start_as_current, + ) + + def _new_span( + self, + span_name: str, + parent_context: Context | None = None, + component: str | None = None, + links=None, + start_time=None, + start_as_current: bool = True, + ): + if component is None: + component = self.otel_service + + tracer = self.get_tracer(component=component) + + if start_time is None: + start_time = pendulum.now("UTC") + + if links is None: + links = [] + + if start_as_current: + span = tracer.start_as_current_span( + name=span_name, + context=parent_context, + links=links, + start_time=datetime_to_nano(start_time), + ) + else: + span = tracer.start_span( + name=span_name, + context=parent_context, + links=links, + start_time=datetime_to_nano(start_time), + ) + current_span_ctx = trace.set_span_in_context(NonRecordingSpan(span.get_span_context())) + # We have to manually make the span context as the active context. + # If the span needs to be injected into the carrier, then this is needed to make sure + # that the injected context will point to the span context that was just created. + attach(current_span_ctx) + return span + + def inject(self) -> dict: + """Inject the current span context into a carrier and return it.""" + carrier = {} + TraceContextTextMapPropagator().inject(carrier) + return carrier + + def extract(self, carrier: dict) -> Context: + """Extract the span context from a provided carrier.""" + return TraceContextTextMapPropagator().extract(carrier) + def gen_context(trace_id: int, span_id: int): """Generate a remote span context for given trace and span id.""" @@ -265,7 +409,7 @@ def gen_link_from_traceparent(traceparent: str): return Link(context=span_ctx, attributes={"meta.annotation_type": "link", "from": "traceparent"}) -def get_otel_tracer(cls) -> OtelTrace: +def get_otel_tracer(cls, use_simple_processor: bool | None = None) -> OtelTrace: """Get OTEL tracer from airflow configuration.""" host = conf.get("traces", "otel_host") port = conf.getint("traces", "otel_port") @@ -278,9 +422,14 @@ def get_otel_tracer(cls) -> OtelTrace: return OtelTrace( span_exporter=OTLPSpanExporter(endpoint=endpoint, headers={"Content-Type": "application/json"}), tag_string=tag_string, + use_simple_processor=use_simple_processor, ) +def get_otel_tracer_for_task(cls) -> OtelTrace: + return get_otel_tracer(cls, use_simple_processor=True) + + class AirflowOtelIdGenerator(IdGenerator): """ ID Generator for span id and trace id. diff --git a/airflow/traces/tracer.py b/airflow/traces/tracer.py index 79272d43d00bb..2cba6509d0561 100644 --- a/airflow/traces/tracer.py +++ b/airflow/traces/tracer.py @@ -181,6 +181,34 @@ def start_span_from_taskinstance( """Start a span from taskinstance.""" raise NotImplementedError() + @classmethod + def start_root_span(cls, span_name=None, component=None, start_time=None, start_as_current=True): + """Start a root span.""" + raise NotImplementedError() + + @classmethod + def start_child_span( + cls, + span_name=None, + parent_context=None, + component=None, + links=None, + start_time=None, + start_as_current=True, + ): + """Start a child span.""" + raise NotImplementedError() + + @classmethod + def inject(cls) -> dict: + """Inject the current span context into a carrier and return it.""" + raise NotImplementedError() + + @classmethod + def extract(cls, carrier) -> EmptyContext: + """Extract the span context from a provided carrier.""" + raise NotImplementedError() + class EmptyTrace: """If no Tracer is configured, EmptyTracer is used as a fallback.""" @@ -242,6 +270,36 @@ def start_span_from_taskinstance( """Start a span from taskinstance.""" return EMPTY_SPAN + @classmethod + def start_root_span( + cls, span_name=None, component=None, start_time=None, start_as_current=True + ) -> EmptySpan: + """Start a root span.""" + return EMPTY_SPAN + + @classmethod + def start_child_span( + cls, + span_name=None, + parent_context=None, + component=None, + links=None, + start_time=None, + start_as_current=True, + ) -> EmptySpan: + """Start a child span.""" + return EMPTY_SPAN + + @classmethod + def inject(cls): + """Inject the current span context into a carrier and return it.""" + return {} + + @classmethod + def extract(cls, carrier) -> EmptyContext: + """Extract the span context from a provided carrier.""" + raise EMPTY_CTX + class _Trace(type): factory: Callable diff --git a/airflow/utils/sqlalchemy.py b/airflow/utils/sqlalchemy.py index c81ecf037a62e..bf5e3791200ae 100644 --- a/airflow/utils/sqlalchemy.py +++ b/airflow/utils/sqlalchemy.py @@ -27,6 +27,7 @@ from packaging import version from sqlalchemy import TIMESTAMP, PickleType, event, nullsfirst, tuple_ from sqlalchemy.dialects import mysql +from sqlalchemy.dialects.postgresql import JSONB from sqlalchemy.types import JSON, Text, TypeDecorator from airflow.configuration import conf @@ -110,7 +111,10 @@ class ExtendedJSON(TypeDecorator): cache_ok = True def load_dialect_impl(self, dialect) -> TypeEngine: - return dialect.type_descriptor(JSON) + if dialect.name == "postgresql": + return dialect.type_descriptor(JSONB) + else: + return dialect.type_descriptor(JSON) def process_bind_param(self, value, dialect): from airflow.serialization.serialized_objects import BaseSerialization diff --git a/airflow/utils/thread_safe_dict.py b/airflow/utils/thread_safe_dict.py new file mode 100644 index 0000000000000..d6581258d3e87 --- /dev/null +++ b/airflow/utils/thread_safe_dict.py @@ -0,0 +1,45 @@ +# 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 threading + + +class ThreadSafeDict: + """Dictionary that uses a lock during operations, to ensure thread safety.""" + + def __init__(self): + self.sync_dict = {} + self.thread_lock = threading.Lock() + + def set(self, key, value): + with self.thread_lock: + self.sync_dict[key] = value + + def get(self, key): + with self.thread_lock: + return self.sync_dict.get(key) + + def delete(self, key): + with self.thread_lock: + if key in self.sync_dict: + del self.sync_dict[key] + + def get_all(self): + with self.thread_lock: + # Return a copy to avoid exposing the internal dictionary. + return self.sync_dict.copy() diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/traces.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/traces.rst index 93fe6e87b40bf..b1de367f48dba 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/traces.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/traces.rst @@ -42,6 +42,7 @@ Add the following lines to your configuration file e.g. ``airflow.cfg`` otel_application = airflow otel_ssl_active = False otel_task_log_event = True + otel_use_context_propagation = True Enable Https ----------------- diff --git a/tests/core/test_otel_tracer.py b/tests/core/test_otel_tracer.py index 666e78e2dc84f..6ab7a568a2e4b 100644 --- a/tests/core/test_otel_tracer.py +++ b/tests/core/test_otel_tracer.py @@ -155,3 +155,55 @@ def test_traskinstance_tracer(self, conf_a, exporter): log.info(span2) assert span1["context"]["trace_id"] == f"0x{utils.gen_trace_id(ti.dag_run)}" assert span1["context"]["span_id"] == f"0x{utils.gen_span_id(ti)}" + + @patch("opentelemetry.sdk.trace.export.ConsoleSpanExporter") + @patch("airflow.traces.otel_tracer.conf") + def test_context_propagation(self, conf_a, exporter): + # necessary to speed up the span to be emitted + with env_vars({"OTEL_BSP_SCHEDULE_DELAY": "1"}): + log = logging.getLogger("TestOtelTrace.test_context_propagation") + log.setLevel(logging.DEBUG) + conf_a.get.return_value = "abc" + conf_a.getint.return_value = 123 + # this will enable debug to set - which outputs the result to console + conf_a.getboolean.return_value = True + + # mocking console exporter with in mem exporter for better assertion + in_mem_exporter = InMemorySpanExporter() + exporter.return_value = in_mem_exporter + + # Method that represents another service which is + # - getting the carrier + # - extracting the context + # - using the context to create a new span + # The new span should be associated with the span from the injected context carrier. + def _task_func(otel_tr: Trace, carrier: dict): + parent_context = otel_tr.extract(carrier) + + with otel_tr.start_child_span(span_name="sub_span", parent_context=parent_context) as span: + span.set_attribute("attr2", "val2") + json_span = json.loads(span.to_json()) + return json_span + + tracer = otel_tracer.get_otel_tracer(Trace) + + root_span = tracer.start_root_span(span_name="root_span", start_as_current=False) + # The context is available, it can be injected into the carrier. + context_carrier = tracer.inject() + + # Some function that uses the carrier to create a new span. + json_span2 = _task_func(otel_tr=tracer, carrier=context_carrier) + + json_span1 = json.loads(root_span.to_json()) + # Manually end the span. + root_span.end() + + log.info(json_span1) + log.info(json_span2) + # Verify that span1 is a root span. + assert json_span1["parent_id"] is None + # Check span2 parent_id to verify that it's a child of span1. + assert json_span2["parent_id"] == json_span1["context"]["span_id"] + # The trace_id and the span_id are randomly generated by the otel sdk. + # Both spans should belong to the same trace. + assert json_span1["context"]["trace_id"] == json_span2["context"]["trace_id"] diff --git a/tests/integration/otel/__init__.py b/tests/integration/otel/__init__.py new file mode 100644 index 0000000000000..217e5db960782 --- /dev/null +++ b/tests/integration/otel/__init__.py @@ -0,0 +1,17 @@ +# +# 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. diff --git a/tests/integration/otel/dags/__init__.py b/tests/integration/otel/dags/__init__.py new file mode 100644 index 0000000000000..217e5db960782 --- /dev/null +++ b/tests/integration/otel/dags/__init__.py @@ -0,0 +1,17 @@ +# +# 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. diff --git a/tests/integration/otel/dags/otel_test_dag.py b/tests/integration/otel/dags/otel_test_dag.py new file mode 100644 index 0000000000000..326b7c11c7f9f --- /dev/null +++ b/tests/integration/otel/dags/otel_test_dag.py @@ -0,0 +1,104 @@ +# 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 logging +from datetime import datetime + +from opentelemetry import trace + +from airflow import DAG +from airflow.operators.python import PythonOperator +from airflow.traces import otel_tracer +from airflow.traces.otel_tracer import CTX_PROP_SUFFIX +from airflow.traces.tracer import Trace + +logger = logging.getLogger("airflow.test_dag") + +args = { + "owner": "airflow", + "start_date": datetime(2024, 9, 1), + "retries": 0, +} + +# DAG definition. +with DAG( + "otel_test_dag", + default_args=args, + schedule=None, + catchup=False, +) as dag: + # Tasks. + def task1_func(**dag_context): + logger.info("Starting Task_1.") + + ti = dag_context["ti"] + context_carrier = ti.context_carrier + + otel_task_tracer = otel_tracer.get_otel_tracer_for_task(Trace) + tracer_provider = otel_task_tracer.get_otel_tracer_provider() + + if context_carrier is not None: + logger.info("Found ti.context_carrier: %s.", context_carrier) + logger.info("Extracting the span context from the context_carrier.") + parent_context = Trace.extract(context_carrier) + with otel_task_tracer.start_child_span( + span_name=f"{ti.task_id}_sub_span1{CTX_PROP_SUFFIX}", + parent_context=parent_context, + component=f"dag{CTX_PROP_SUFFIX}", + ) as s1: + s1.set_attribute("attr1", "val1") + logger.info("From task sub_span1.") + + with otel_task_tracer.start_child_span(f"{ti.task_id}_sub_span2{CTX_PROP_SUFFIX}") as s2: + s2.set_attribute("attr2", "val2") + logger.info("From task sub_span2.") + + tracer = trace.get_tracer("trace_test.tracer", tracer_provider=tracer_provider) + with tracer.start_as_current_span(name=f"{ti.task_id}_sub_span3{CTX_PROP_SUFFIX}") as s3: + s3.set_attribute("attr3", "val3") + logger.info("From task sub_span3.") + + with otel_task_tracer.start_child_span( + span_name=f"{ti.task_id}_sub_span4{CTX_PROP_SUFFIX}", + parent_context=parent_context, + component=f"dag{CTX_PROP_SUFFIX}", + ) as s4: + s4.set_attribute("attr4", "val4") + logger.info("From task sub_span4.") + + logger.info("Task_1 finished.") + + def task2_func(): + logger.info("Starting Task_2.") + for i in range(3): + logger.info("Task_2, iteration '%d'.", i) + logger.info("Task_2 finished.") + + # Task operators. + t1 = PythonOperator( + task_id="task_1", + python_callable=task1_func, + ) + + t2 = PythonOperator( + task_id="task_2", + python_callable=task2_func, + ) + + # Dependencies. + t1 >> t2 diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py new file mode 100644 index 0000000000000..50d4ec16329f3 --- /dev/null +++ b/tests/integration/otel/test_otel.py @@ -0,0 +1,298 @@ +# 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 logging +import os +import subprocess +import time + +import pendulum +import pytest + +from airflow.executors import executor_loader +from airflow.executors.executor_utils import ExecutorName +from airflow.models import DagBag, DagRun +from airflow.models.serialized_dag import SerializedDagModel +from airflow.traces.otel_tracer import CTX_PROP_SUFFIX +from airflow.utils.session import create_session +from airflow.utils.state import State + +from tests.integration.otel.test_utils import ( + assert_parent_children_spans, + assert_parent_children_spans_for_non_root, + assert_span_name_belongs_to_root_span, + assert_span_not_in_children_spans, + dump_airflow_metadata_db, + extract_spans_from_output, + get_parent_child_dict, +) + +log = logging.getLogger("test_otel.TestOtelIntegration") + + +@pytest.mark.integration("redis") +@pytest.mark.backend("postgres") +class TestOtelIntegration: + test_dir = os.path.dirname(os.path.abspath(__file__)) + dag_folder = os.path.join(test_dir, "dags") + + @classmethod + def setup_class(cls): + os.environ["AIRFLOW__TRACES__OTEL_ON"] = "True" + os.environ["AIRFLOW__TRACES__OTEL_HOST"] = "localhost" + os.environ["AIRFLOW__TRACES__OTEL_PORT"] = "4318" + os.environ["AIRFLOW__TRACES__OTEL_DEBUGGING_ON"] = "True" + os.environ["AIRFLOW__TRACES__OTEL_TASK_LOG_EVENT"] = "True" + os.environ["AIRFLOW__TRACES__OTEL_USE_CONTEXT_PROPAGATION"] = "True" + + os.environ["AIRFLOW__SCHEDULER__STANDALONE_DAG_PROCESSOR"] = "False" + os.environ["AIRFLOW__SCHEDULER__PROCESSOR_POLL_INTERVAL"] = "2" + + os.environ["AIRFLOW__CORE__DAGS_FOLDER"] = f"{cls.dag_folder}" + + os.environ["AIRFLOW__CORE__LOAD_EXAMPLES"] = "False" + os.environ["AIRFLOW__CORE__UNIT_TEST_MODE"] = "False" + + @pytest.fixture + def celery_worker_env_vars(self, monkeypatch): + os.environ["AIRFLOW__CORE__EXECUTOR"] = "CeleryExecutor" + executor_name = ExecutorName( + module_path="airflow.providers.celery.executors.celery_executor.CeleryExecutor", + alias="CeleryExecutor", + ) + monkeypatch.setattr(executor_loader, "_alias_to_executors", {"CeleryExecutor": executor_name}) + + def test_dag_spans_with_context_propagation(self, monkeypatch, celery_worker_env_vars, capfd, session): + """ + Test that a DAG runs successfully and exports the correct spans. + Integration with a scheduler, a celery worker, a postgres db and a redis broker. + """ + # Uncomment to enable debug mode and get span and db dumps on the output. + # log.setLevel(logging.DEBUG) + + celery_command_args = [ + "celery", + "--app", + "airflow.providers.celery.executors.celery_executor.app", + "worker", + "--concurrency", + "1", + "--loglevel", + "INFO", + ] + + scheduler_command_args = [ + "airflow", + "scheduler", + ] + + celery_worker_process = None + scheduler_process = None + try: + # Start the processes here and not as fixtures, so that the test can capture their output. + celery_worker_process = subprocess.Popen( + celery_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) + + scheduler_process = subprocess.Popen( + scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) + + # Wait to ensure both processes have started. + time.sleep(10) + + execution_date = pendulum.now("UTC") + + # Load DAGs from the dag directory. + dag_bag = DagBag(dag_folder=self.dag_folder, include_examples=False) + + dag_id = "otel_test_dag" + dag = dag_bag.get_dag(dag_id) + + assert dag is not None, f"DAG with ID {dag_id} not found." + + with create_session() as session: + # Sync the DAG to the database. + dag.sync_to_db(session=session) + # Manually serialize the dag and write it to the db to avoid a db error. + SerializedDagModel.write_dag(dag, session=session) + session.commit() + + unpause_command = ["airflow", "dags", "unpause", dag_id] + + # Unpause the dag using the cli. + subprocess.run(unpause_command, check=True, env=os.environ.copy()) + + run_id = f"manual__{execution_date.isoformat()}" + + trigger_command = [ + "airflow", + "dags", + "trigger", + dag_id, + "--run-id", + run_id, + "--exec-date", + execution_date.isoformat(), + ] + + # Trigger the dag using the cli. + subprocess.run(trigger_command, check=True, env=os.environ.copy()) + + # Wait timeout for the DAG run to complete. + max_wait_time = 60 # seconds + start_time = time.time() + + dag_run_state = None + + while time.time() - start_time < max_wait_time: + with create_session() as session: + dag_run = ( + session.query(DagRun) + .filter( + DagRun.dag_id == dag_id, + DagRun.run_id == run_id, + ) + .first() + ) + + if dag_run is None: + time.sleep(5) + continue + + dag_run_state = dag_run.state + log.info("DAG Run state: %s.", dag_run_state) + + if dag_run_state in [State.SUCCESS, State.FAILED]: + break + + time.sleep(5) + + if logging.root.level == logging.DEBUG: + with create_session() as session: + dump_airflow_metadata_db(session) + + assert ( + dag_run_state == State.SUCCESS + ), f"Dag run did not complete successfully. Final state: {dag_run_state}." + finally: + # Terminate the processes. + celery_worker_process.terminate() + celery_worker_process.wait() + + scheduler_process.terminate() + scheduler_process.wait() + + out, err = capfd.readouterr() + log.debug("out-start --\n%s\n-- out-end", out) + log.debug("err-start --\n%s\n-- err-end", err) + + # Dag run should have succeeded. Test the spans in the output. + # Get a list of lines from the captured output. + output_lines = out.splitlines() + + # Filter the output, create a json obj for each span and then store them into dictionaries. + # One dictionary with only the root spans, and one with all the captured spans (root and otherwise). + root_span_dict, span_dict = extract_spans_from_output(output_lines) + # Generate a dictionary with parent child relationships. + # This is done by comparing the span_id of each root span with the parent_id of each non-root span. + parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) + + dag_span_name = str(dag_id + CTX_PROP_SUFFIX) + assert_span_name_belongs_to_root_span( + root_span_dict=root_span_dict, span_name=dag_span_name, should_succeed=True + ) + + non_existent_dag_span_name = str(dag_id + CTX_PROP_SUFFIX + "fail") + assert_span_name_belongs_to_root_span( + root_span_dict=root_span_dict, span_name=non_existent_dag_span_name, should_succeed=False + ) + + dag_children_span_names = [] + task_instance_ids = dag.task_ids + + for task_id in task_instance_ids: + dag_children_span_names.append(f"{task_id}{CTX_PROP_SUFFIX}") + + first_task_id = task_instance_ids[0] + + assert_parent_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + parent_name=dag_span_name, + children_names=dag_children_span_names, + ) + + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_span_name, + child_name=first_task_id, + span_exists=True, + ) + + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_span_name, + child_name=f"{first_task_id}_fail", + span_exists=False, + ) + + # Any spans generated under a task, are children of the task span. + # The span hierarchy for dag 'test_dag' is + # dag span + # |_ task_1 span + # |_ sub_span_1 + # |_ sub_span_2 + # |_ sub_span_3 + # |_ sub_span_4 + # |_ task_2 span + + first_task_children_span_names = [ + f"{first_task_id}_sub_span1{CTX_PROP_SUFFIX}", + f"{first_task_id}_sub_span4{CTX_PROP_SUFFIX}", + ] + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{first_task_id}{CTX_PROP_SUFFIX}", + children_names=first_task_children_span_names, + ) + + # Single element list. + sub_span1_children_span_names = [f"{first_task_id}_sub_span2{CTX_PROP_SUFFIX}"] + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{first_task_id}_sub_span1{CTX_PROP_SUFFIX}", + children_names=sub_span1_children_span_names, + ) + + sub_span2_children_span_names = [f"{first_task_id}_sub_span3{CTX_PROP_SUFFIX}"] + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{first_task_id}_sub_span2{CTX_PROP_SUFFIX}", + children_names=sub_span2_children_span_names, + ) diff --git a/tests/integration/otel/test_utils.py b/tests/integration/otel/test_utils.py new file mode 100644 index 0000000000000..f79da9a7a2ffd --- /dev/null +++ b/tests/integration/otel/test_utils.py @@ -0,0 +1,273 @@ +# 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 json +import logging +import pprint + +from sqlalchemy import inspect + +from airflow.models import Base + +log = logging.getLogger("tests.otel.test_utils") + + +def dump_airflow_metadata_db(session): + inspector = inspect(session.bind) + all_tables = inspector.get_table_names() + + # dump with the entire db + db_dump = {} + + log.debug("\n-----START_airflow_db_dump-----\n") + + for table_name in all_tables: + log.debug("\nDumping table: %s", table_name) + table = Base.metadata.tables.get(table_name) + if table is not None: + query = session.query(table) + results = [dict(row) for row in query.all()] + db_dump[table_name] = results + # Pretty-print the table contents + if table_name == "connection": + filtered_results = [row for row in results if row.get("conn_id") == "airflow_db"] + pprint.pprint({table_name: filtered_results}, width=120) + else: + pprint.pprint({table_name: results}, width=120) + else: + log.debug("Table %s not found in metadata.", table_name) + + log.debug("\nAirflow metadata database dump complete.") + log.debug("\n-----END_airflow_db_dump-----\n") + + +# Example span from the ConsoleSpanExporter +# { +# "name": "perform_heartbeat", +# "context": { +# "trace_id": "0xa18781ea597c3d07c85e95fd3a6d7d40", +# "span_id": "0x8ae7bb13ec5b28ba", +# "trace_state": "[]" +# }, +# "kind": "SpanKind.INTERNAL", +# "parent_id": "0x17ac77a4a840758d", +# "start_time": "2024-10-30T16:19:33.947155Z", +# "end_time": "2024-10-30T16:19:33.947192Z", +# "status": { +# "status_code": "UNSET" +# }, +# "attributes": {}, +# "events": [], +# "links": [], +# "resource": { +# "attributes": { +# "telemetry.sdk.language": "python", +# "telemetry.sdk.name": "opentelemetry", +# "telemetry.sdk.version": "1.27.0", +# "host.name": "host.local", +# "service.name": "Airflow" +# }, +# "schema_url": "" +# } +# } + + +def extract_spans_from_output(output_lines): + span_dict = {} + root_span_dict = {} + total_lines = len(output_lines) + index = 0 + + while index < total_lines: + line = output_lines[index].strip() + # The start and the end of the json object, don't have any indentation. + # We can use that to identify them. + if line.startswith("{") and line == "{": # Json start. + # Get all the lines and append them until we reach the end. + json_lines = [line] + index += 1 + while index < total_lines: + line = output_lines[index] + json_lines.append(line) + if line.strip().startswith("}") and line == "}": # Json end. + # Since, this is the end of the object, break the loop. + break + index += 1 + # Create a formatted json string and then convert the string to a python dict. + json_str = "\n".join(json_lines) + try: + span = json.loads(json_str) + span_id = span["context"]["span_id"] + span_dict[span_id] = span + + if span["parent_id"] is None: + # This is a root span, add it to the root_span_map as well. + root_span_id = span["context"]["span_id"] + root_span_dict[root_span_id] = span + + except json.JSONDecodeError as e: + log.error("Failed to parse JSON span: %s", e) + log.error("Failed JSON string:") + log.error(json_str) + else: + index += 1 + + return root_span_dict, span_dict + + +def get_id_for_a_given_name(span_dict: dict, span_name: str): + for span_id, span in span_dict.items(): + if span["name"] == span_name: + return span_id + return None + + +def get_parent_child_dict(root_span_dict, span_dict): + """Create a dictionary with parent-child span relationships.""" + parent_child_dict = {} + for root_span_id, root_span in root_span_dict.items(): + # Compare each 'root_span_id' with each 'parent_id' from the span_dict. + # If there is a match, then the span in the span_dict, is a child. + # For every root span, create a list of child spans. + child_span_list = [] + for span_id, span in span_dict.items(): + if root_span_id == span_id: + # It's the same span, skip. + continue + # If the parent_id matches the root_span_id and if the trace_id is the same. + if ( + span["parent_id"] == root_span_id + and root_span["context"]["trace_id"] == span["context"]["trace_id"] + ): + child_span_list.append(span) + parent_child_dict[root_span_id] = child_span_list + return parent_child_dict + + +def get_child_list_for_non_root(span_dict: dict, span_name: str): + """ + Get a list of children spans for a parent span that isn't also a root span. + e.g. a task span with sub-spans, is a parent span but not a root span. + """ + parent_span_id = get_id_for_a_given_name(span_dict=span_dict, span_name=span_name) + parent_span = span_dict.get(parent_span_id) + + child_span_list = [] + for span_id, span in span_dict.items(): + if span_id == parent_span_id: + # It's the same span, skip. + continue + if ( + span["parent_id"] == parent_span_id + and span["context"]["trace_id"] == parent_span["context"]["trace_id"] + ): + child_span_list.append(span) + + return child_span_list + + +def assert_parent_name_and_get_id(root_span_dict: dict, span_name: str): + parent_id = get_id_for_a_given_name(root_span_dict, span_name) + + assert parent_id is not None, f"Parent span '{span_name}' wasn't found." + + return parent_id + + +def assert_span_name_belongs_to_root_span(root_span_dict: dict, span_name: str, should_succeed: bool): + """Check that a given span name belongs to a root span.""" + log.info("Checking that '%s' is a root span.", span_name) + # Check if any root span has the specified span_name + name_exists = any(root_span.get("name", None) == span_name for root_span in root_span_dict.values()) + + # Assert based on the should_succeed flag + if should_succeed: + assert name_exists, f"Expected span '{span_name}' to belong to a root span, but it does not." + log.info("Span '%s' belongs to a root span, as expected.", span_name) + else: + assert not name_exists, f"Expected span '{span_name}' not to belong to a root span, but it does." + log.info("Span '%s' doesn't belong to a root span, as expected.", span_name) + + +def assert_parent_children_spans( + parent_child_dict: dict, root_span_dict: dict, parent_name: str, children_names: list[str] +): + """Check that all spans in a given list are children of a given root span name.""" + log.info("Checking that spans '%s' are children of root span '%s'.", children_names, parent_name) + # Iterate the root_span_dict, to get the span_id for the parent_name. + parent_id = assert_parent_name_and_get_id(root_span_dict=root_span_dict, span_name=parent_name) + + # Use the root span_id to get the children ids. + child_span_list = parent_child_dict[parent_id] + + # For each children id, get the entry from the span_dict. + names_from_dict = [] + for child_span in child_span_list: + name = child_span["name"] + names_from_dict.append(name) + + # Assert that all given children names match the names from the dictionary. + for name in children_names: + assert ( + name in names_from_dict + ), f"Span name '{name}' wasn't found in children span names. It's not a child of span '{parent_name}'." + + +def assert_parent_children_spans_for_non_root(span_dict: dict, parent_name: str, children_names: list[str]): + """Check that all spans in a given list are children of a given non-root span name.""" + log.info("Checking that spans '%s' are children of span '%s'.", children_names, parent_name) + child_span_list = get_child_list_for_non_root(span_dict=span_dict, span_name=parent_name) + + # For each children id, get the entry from the span_dict. + names_from_dict = [] + for child_span in child_span_list: + name = child_span["name"] + names_from_dict.append(name) + + # Assert that all given children names match the names from the dictionary. + for name in children_names: + assert ( + name in names_from_dict + ), f"Span name '{name}' wasn't found in children span names. It's not a child of span '{parent_name}'." + + +def assert_span_not_in_children_spans( + parent_child_dict: dict, + root_span_dict: dict, + span_dict: dict, + parent_name: str, + child_name: str, + span_exists: bool, +): + """Check that a span for a given name, doesn't belong to the children of a given root span name.""" + log.info("Checking that span '%s' is not a child of span '%s'.", child_name, parent_name) + # Iterate the root_span_dict, to get the span_id for the parent_name. + parent_id = assert_parent_name_and_get_id(root_span_dict=root_span_dict, span_name=parent_name) + + # Use the root span_id to get the children ids. + child_span_id_list = parent_child_dict[parent_id] + + child_id = get_id_for_a_given_name(span_dict=span_dict, span_name=child_name) + + if span_exists: + assert child_id is not None, f"Span '{child_name}' should exist but it doesn't." + assert ( + child_id not in child_span_id_list + ), f"Span '{child_name}' shouldn't be a child of span '{parent_name}', but it is." + else: + assert child_id is None, f"Span '{child_name}' shouldn't exist but it does." diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index 8f15005f43d13..f928960ee6c67 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -4047,6 +4047,7 @@ def test_refresh_from_db(self, create_task_instance): "updated_at": None, "task_display_name": "Test Refresh from DB Task", "dag_version_id": None, + "context_carrier": {}, } # Make sure we aren't missing any new value in our expected_values list. expected_keys = {f"task_instance.{key}" for key in expected_values} From fdb92df91f7c50ee187c16bea5ae9ab549339ae0 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 12 Nov 2024 18:59:35 +0200 Subject: [PATCH 02/52] cleanup --- airflow/config_templates/config.yml | 2 +- airflow/models/dagrun.py | 12 ++++-------- airflow/traces/otel_tracer.py | 12 +++++++++--- tests/integration/otel/dags/otel_test_dag.py | 2 +- 4 files changed, 15 insertions(+), 13 deletions(-) diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 7726134bb90cb..35fbb2a60125d 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -1297,7 +1297,7 @@ traces: All the children spans will be created using context propagation. This doesn't replace any of the existing spans, but it creates additional ones. The default value is False. - version_added: 2.10.2 + version_added: 2.10.3 type: string example: ~ default: "False" diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 5612139e56bfe..5345d68659bce 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -872,11 +872,7 @@ def is_effective_leaf(task): return leaf_tis @staticmethod - def _set_dagrun_span_attrs(span: Span, dag_run: DagRun): - # This is necessary to avoid an error in case of testing a paused dag. - if dag_run.queued_at is None and dag_run.start_date is not None: - dag_run.queued_at = dag_run.start_date - + def _set_dagrun_span_attrs(span: Span, dag_run: DagRun, dagv: DagVersion): if dag_run._state is DagRunState.FAILED: span.set_attribute("airflow.dag_run.error", True) attributes = { @@ -897,7 +893,7 @@ def _set_dagrun_span_attrs(span: Span, dag_run: DagRun): "airflow.dag_run.run_type": str(dag_run.run_type), "airflow.dag_run.data_interval_start": str(dag_run.data_interval_start), "airflow.dag_run.data_interval_end": str(dag_run.data_interval_end), - "airflow.dag_run.dag_hash": str(dag_run.dag_hash), + "airflow.dag_version.version": str(dagv.version if dagv else None), "airflow.dag_run.conf": str(dag_run.conf), } if span.is_recording(): @@ -1101,7 +1097,7 @@ def recalculate(self) -> _UnfinishedStates: self.state, ) - self._set_dagrun_span_attrs(span=active_span, dag_run=self) + self._set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) active_span.end() # Remove the span from the dict. self.active_spans.delete(self.run_id) @@ -1114,7 +1110,7 @@ def recalculate(self) -> _UnfinishedStates: ) with Trace.start_span_from_dagrun(dagrun=self) as span: - self._set_dagrun_span_attrs(span=span, dag_run=self) + self._set_dagrun_span_attrs(span=span, dag_run=self, dagv=dagv) session.flush() diff --git a/airflow/traces/otel_tracer.py b/airflow/traces/otel_tracer.py index 3f9506a836c59..c0293ba7a5b89 100644 --- a/airflow/traces/otel_tracer.py +++ b/airflow/traces/otel_tracer.py @@ -72,7 +72,8 @@ def __init__( use_simple_processor: bool = False, ): self.span_exporter = span_exporter - if use_simple_processor: + self.use_simple_processor = use_simple_processor + if self.use_simple_processor: # With a BatchSpanProcessor, spans are exported at an interval. # A task can run fast and finish before spans have enough time to get exported to the collector. # When creating spans from inside a task, a SimpleSpanProcessor needs to be used because @@ -105,9 +106,14 @@ def get_otel_tracer_provider( debug = conf.getboolean("traces", "otel_debugging_on") if debug is True: log.info("[ConsoleSpanExporter] is being used") - tracer_provider.add_span_processor(BatchSpanProcessor(ConsoleSpanExporter())) + if self.use_simple_processor: + span_processor_for_tracer_prov = SimpleSpanProcessor(ConsoleSpanExporter()) + else: + span_processor_for_tracer_prov = BatchSpanProcessor(ConsoleSpanExporter()) else: - tracer_provider.add_span_processor(self.span_processor) + span_processor_for_tracer_prov = self.span_processor + + tracer_provider.add_span_processor(span_processor_for_tracer_prov) return tracer_provider def get_tracer( diff --git a/tests/integration/otel/dags/otel_test_dag.py b/tests/integration/otel/dags/otel_test_dag.py index 326b7c11c7f9f..36349ab863e4c 100644 --- a/tests/integration/otel/dags/otel_test_dag.py +++ b/tests/integration/otel/dags/otel_test_dag.py @@ -22,7 +22,7 @@ from opentelemetry import trace from airflow import DAG -from airflow.operators.python import PythonOperator +from airflow.providers.standard.operators.python import PythonOperator from airflow.traces import otel_tracer from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import Trace From 4b3c93e31adc179e9870f00cf5619eeb4030631f Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Thu, 14 Nov 2024 14:29:15 +0200 Subject: [PATCH 03/52] revert the metrics name length change --- airflow/metrics/validators.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airflow/metrics/validators.py b/airflow/metrics/validators.py index d851db82b9944..b47cdac1be551 100644 --- a/airflow/metrics/validators.py +++ b/airflow/metrics/validators.py @@ -82,8 +82,7 @@ class MetricNameLengthExemptionWarning(Warning): } BACK_COMPAT_METRIC_NAMES: set[Pattern[str]] = {re2.compile(name) for name in BACK_COMPAT_METRIC_NAME_PATTERNS} -# According to otel spec, max length should be 255. Change if the spec gets revised. -OTEL_NAME_MAX_LENGTH = 255 +OTEL_NAME_MAX_LENGTH = 63 DEFAULT_VALIDATOR_TYPE = "allow" From 939511b7e2b233e59cee7910678f2e1c2c7dc9c4 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Fri, 15 Nov 2024 17:45:50 +0200 Subject: [PATCH 04/52] cleanup --- airflow/traces/otel_tracer.py | 19 ++++++++++++++++--- airflow/traces/tracer.py | 2 +- 2 files changed, 17 insertions(+), 4 deletions(-) diff --git a/airflow/traces/otel_tracer.py b/airflow/traces/otel_tracer.py index c0293ba7a5b89..7d6889c77eda2 100644 --- a/airflow/traces/otel_tracer.py +++ b/airflow/traces/otel_tracer.py @@ -272,7 +272,12 @@ def start_span_from_taskinstance( return span def start_root_span( - self, span_name: str, component: str | None = None, start_time=None, start_as_current: bool = True + self, + span_name: str, + component: str | None = None, + links=None, + start_time=None, + start_as_current: bool = True, ): """Start a root span.""" # If no context is passed to the new span, @@ -284,11 +289,16 @@ def start_root_span( ) invalid_ctx = trace.set_span_in_context(NonRecordingSpan(invalid_span_ctx)) + if links is None: + _links = [] + else: + _links = links + return self._new_span( span_name=span_name, parent_context=invalid_ctx, component=component, - links=None, + links=_links, start_time=start_time, start_as_current=start_as_current, ) @@ -313,7 +323,10 @@ def start_child_span( if isinstance(context_val, NonRecordingSpan): parent_span_context = context_val.get_span_context() - _links = gen_links_from_kv_list(links) if links else [] + if links is None: + _links = [] + else: + _links = links _links.append( Link( context=parent_span_context, diff --git a/airflow/traces/tracer.py b/airflow/traces/tracer.py index 2cba6509d0561..7a533dae7a3e9 100644 --- a/airflow/traces/tracer.py +++ b/airflow/traces/tracer.py @@ -298,7 +298,7 @@ def inject(cls): @classmethod def extract(cls, carrier) -> EmptyContext: """Extract the span context from a provided carrier.""" - raise EMPTY_CTX + return EMPTY_CTX class _Trace(type): From bae8bca04e89c58ca229d9e6b597abadeb541121 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sat, 16 Nov 2024 14:09:40 +0200 Subject: [PATCH 05/52] move the active span dictionaries to the scheduler class --- airflow/executors/base_executor.py | 10 +++- airflow/jobs/scheduler_job_runner.py | 19 ++++++- airflow/models/dagrun.py | 82 +++++++++++++++++++--------- 3 files changed, 81 insertions(+), 30 deletions(-) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index 12e0639d07c2e..b8d82606756ff 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -38,7 +38,6 @@ from airflow.traces.utils import gen_span_id_from_ti_key, gen_trace_id from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import TaskInstanceState -from airflow.utils.thread_safe_dict import ThreadSafeDict PARALLELISM: int = conf.getint("core", "PARALLELISM") @@ -52,6 +51,7 @@ from airflow.executors.executor_utils import ExecutorName from airflow.models.taskinstance import TaskInstance from airflow.models.taskinstancekey import TaskInstanceKey + from airflow.utils.thread_safe_dict import ThreadSafeDict # Command to execute - list of strings # the first element is always "airflow". @@ -114,7 +114,7 @@ class BaseExecutor(LoggingMixin): :param parallelism: how many jobs should run at one time. Set to ``0`` for infinity. """ - active_spans = ThreadSafeDict() + active_spans = None supports_ad_hoc_ti_run: bool = False supports_sentry: bool = False @@ -153,6 +153,10 @@ def __init__(self, parallelism: int = PARALLELISM): def __repr__(self): return f"{self.__class__.__name__}(parallelism={self.parallelism})" + @classmethod + def set_active_spans(cls, active_spans: ThreadSafeDict): + cls.active_spans = active_spans + def start(self): # pragma: no cover """Executors may need to get things started.""" @@ -339,7 +343,7 @@ def trigger_tasks(self, open_slots: int) -> None: if self.otel_use_context_propagation: # If it's None, then the span for the current TaskInstanceKey hasn't been started. - if self.active_spans.get(key) is None: + if self.active_spans is not None and self.active_spans.get(key) is None: parent_context = Trace.extract(ti.dag_run.context_carrier) # Start a new span using the context from the parent. # Attributes will be set once the task has finished so that all diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 07175eb732d1f..19226c24ec64c 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -77,6 +77,7 @@ with_row_locks, ) from airflow.utils.state import DagRunState, JobState, State, TaskInstanceState +from airflow.utils.thread_safe_dict import ThreadSafeDict from airflow.utils.types import DagRunTriggeredByType, DagRunType if TYPE_CHECKING: @@ -161,6 +162,11 @@ class SchedulerJobRunner(BaseJobRunner, LoggingMixin): job_type = "SchedulerJob" + # Dict entries: dag_run.run_id - span + active_dagrun_spans = ThreadSafeDict() + # Dict entries: ti.key - span + active_ti_spans = ThreadSafeDict() + def __init__( self, job: Job, @@ -797,12 +803,12 @@ def process_executor_events( ti.pid, ) - active_ti_span = executor.active_spans.get(ti.key) + active_ti_span = cls.active_ti_spans.get(ti.key) if conf.getboolean("traces", "otel_use_context_propagation") and active_ti_span is not None: cls._set_span_attrs__process_executor_events(span=active_ti_span, state=state, ti=ti) # End the span and remove it from the active_spans dict. active_ti_span.end() - executor.active_spans.delete(ti.key) + cls.active_ti_spans.delete(ti.key) with Trace.start_span_from_taskinstance(ti=ti) as span: cls._set_span_attrs__process_executor_events(span, state, ti) @@ -1102,6 +1108,15 @@ def _run_scheduler_loop(self) -> None: self.log.debug("Waiting for processors to finish since we're using sqlite") self.processor_agent.wait_until_finished() + # This is passing a reference to the dictionary, making it shared. + # Any changes made by a dag_run instance, will also be reflected to the dictionary of this class. + DagRun.set_active_spans(active_spans=self.active_dagrun_spans) + + # local import due to type_checking. + from airflow.executors.base_executor import BaseExecutor + + BaseExecutor.set_active_spans(active_spans=self.active_ti_spans) + with create_session() as session: # This will schedule for as many executors as possible. num_queued_tis = self._do_scheduling(session) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 5345d68659bce..6d46d4e761d45 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -20,7 +20,18 @@ import itertools import os from collections import defaultdict -from typing import TYPE_CHECKING, Any, Callable, Iterable, Iterator, NamedTuple, Sequence, TypeVar, overload +from typing import ( + TYPE_CHECKING, + Any, + Callable, + Iterable, + Iterator, + NamedTuple, + Sequence, + TypeVar, + Union, + overload, +) import re2 from sqlalchemy import ( @@ -69,7 +80,7 @@ from airflow.ti_deps.dep_context import DepContext from airflow.ti_deps.dependencies_states import SCHEDULEABLE_STATES from airflow.traces.otel_tracer import CTX_PROP_SUFFIX -from airflow.traces.tracer import Trace +from airflow.traces.tracer import EmptySpan, Trace from airflow.utils import timezone from airflow.utils.dates import datetime_to_nano from airflow.utils.helpers import chunks, is_container, prune_dict @@ -309,6 +320,10 @@ def stats_tags(self) -> dict[str, str]: def logical_date(self) -> datetime: return self.execution_date + @classmethod + def set_active_spans(cls, active_spans: ThreadSafeDict): + cls.active_spans = active_spans + def get_state(self): return self._state @@ -872,10 +887,22 @@ def is_effective_leaf(task): return leaf_tis @staticmethod - def _set_dagrun_span_attrs(span: Span, dag_run: DagRun, dagv: DagVersion): + def _set_dagrun_span_attrs(span: Span | EmptySpan, dag_run: DagRun, dagv: DagVersion): if dag_run._state is DagRunState.FAILED: span.set_attribute("airflow.dag_run.error", True) - attributes = { + + AttributeValue = Union[ + str, + bool, + int, + float, + Sequence[str], + Sequence[bool], + Sequence[int], + Sequence[float], + ] + + attributes: dict[str, AttributeValue] = { "airflow.category": "DAG runs", "airflow.dag_run.dag_id": str(dag_run.dag_id), "airflow.dag_run.execution_date": str(dag_run.execution_date), @@ -1040,7 +1067,11 @@ def recalculate(self) -> _UnfinishedStates: # finally, if the leaves aren't done, the dag is still running else: # If there is no value in active_spans, then the span hasn't already been started. - if self.otel_use_context_propagation and (self.active_spans.get(self.run_id) is None): + if ( + self.otel_use_context_propagation + and self.active_spans is not None + and (self.active_spans.get(self.run_id) is None) + ): span = Trace.start_root_span( span_name=f"{self.dag_id}{CTX_PROP_SUFFIX}", component=f"dag{CTX_PROP_SUFFIX}", @@ -1087,27 +1118,28 @@ def recalculate(self) -> _UnfinishedStates: ) if self.otel_use_context_propagation: - active_span = self.active_spans.get(self.run_id) - if active_span is not None: - self.log.debug( - "Found active span with span_id: %s, for dag_id: %s, run_id: %s, state: %s", - active_span.get_span_context().span_id, - self.dag_id, - self.run_id, - self.state, - ) + if self.active_spans is not None: + active_span = self.active_spans.get(self.run_id) + if active_span is not None: + self.log.debug( + "Found active span with span_id: %s, for dag_id: %s, run_id: %s, state: %s", + active_span.get_span_context().span_id, + self.dag_id, + self.run_id, + self.state, + ) - self._set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) - active_span.end() - # Remove the span from the dict. - self.active_spans.delete(self.run_id) - else: - self.log.debug( - "No active span has been found for dag_id: %s, run_id: %s, state: %s", - self.dag_id, - self.run_id, - self.state, - ) + self._set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) + active_span.end() + # Remove the span from the dict. + self.active_spans.delete(self.run_id) + else: + self.log.debug( + "No active span has been found for dag_id: %s, run_id: %s, state: %s", + self.dag_id, + self.run_id, + self.state, + ) with Trace.start_span_from_dagrun(dagrun=self) as span: self._set_dagrun_span_attrs(span=span, dag_run=self, dagv=dagv) From c841f17ce8331da70025f3e3b84c49750e75b54f Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sat, 16 Nov 2024 14:12:11 +0200 Subject: [PATCH 06/52] fix variable name + comment --- airflow/models/dagrun.py | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 6d46d4e761d45..67e08036a16bc 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -891,7 +891,7 @@ def _set_dagrun_span_attrs(span: Span | EmptySpan, dag_run: DagRun, dagv: DagVer if dag_run._state is DagRunState.FAILED: span.set_attribute("airflow.dag_run.error", True) - AttributeValue = Union[ + attribute_value_type = Union[ str, bool, int, @@ -902,7 +902,8 @@ def _set_dagrun_span_attrs(span: Span | EmptySpan, dag_run: DagRun, dagv: DagVer Sequence[float], ] - attributes: dict[str, AttributeValue] = { + # Explicitly set the value type to Union[...] to avoid a mypy error. + attributes: dict[str, attribute_value_type] = { "airflow.category": "DAG runs", "airflow.dag_run.dag_id": str(dag_run.dag_id), "airflow.dag_run.execution_date": str(dag_run.execution_date), From 0266f27364765ce2014c292eab98eebaf2c12d2e Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 26 Nov 2024 15:22:17 +0200 Subject: [PATCH 07/52] initial impl: handling scheduler ha --- airflow/cli/commands/task_command.py | 4 +- airflow/executors/base_executor.py | 4 +- airflow/jobs/scheduler_job_runner.py | 120 +++- airflow/models/dagrun.py | 169 ++++- airflow/models/taskinstance.py | 55 ++ airflow/models/taskinstancehistory.py | 2 + airflow/serialization/pydantic/dag_run.py | 1 + .../serialization/pydantic/taskinstance.py | 1 + airflow/traces/tracer.py | 23 + airflow/utils/span_status.py | 33 + airflow/utils/thread_safe_dict.py | 4 + .../ci/docker-compose/integration-otel.yml | 1 + tests/integration/otel/dags/otel_test_dag.py | 2 +- .../otel/dags/otel_test_dag_with_pause.py | 157 ++++ tests/integration/otel/test_otel.py | 672 ++++++++++++++++-- 15 files changed, 1154 insertions(+), 94 deletions(-) create mode 100644 airflow/utils/span_status.py create mode 100644 tests/integration/otel/dags/otel_test_dag_with_pause.py diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 30baa30ae77db..4f00e585ad795 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -66,6 +66,7 @@ from airflow.utils.net import get_hostname from airflow.utils.providers_configuration_loader import providers_configuration_loaded from airflow.utils.session import NEW_SESSION, create_session, provide_session +from airflow.utils.span_status import SpanStatus from airflow.utils.state import DagRunState from airflow.utils.task_instance_session import set_current_task_instance_session from airflow.utils.types import DagRunTriggeredByType @@ -449,7 +450,8 @@ def task_run(args, dag: DAG | None = None) -> TaskReturnCode | None: log.info("Found args.carrier: %s. Setting the value in the ti instance.", args.carrier) # The arg value is a dict string, and it needs to be converted back to a dict. carrier_dict = json.loads(args.carrier) - ti.set_context_carrier(carrier_dict, with_commit=True) + ti.set_context_carrier(context_carrier=carrier_dict, with_commit=True) + ti.set_span_status(status=SpanStatus.ACTIVE, with_commit=True) if not InternalApiConfig.get_use_internal_api(): # IMPORTANT, have to re-configure ORM with the NullPool, otherwise, each "run" command may leave diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index b8d82606756ff..7b4e65d5bb6e7 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -38,6 +38,7 @@ from airflow.traces.utils import gen_span_id_from_ti_key, gen_trace_id from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.state import TaskInstanceState +from airflow.utils.thread_safe_dict import ThreadSafeDict PARALLELISM: int = conf.getint("core", "PARALLELISM") @@ -51,7 +52,6 @@ from airflow.executors.executor_utils import ExecutorName from airflow.models.taskinstance import TaskInstance from airflow.models.taskinstancekey import TaskInstanceKey - from airflow.utils.thread_safe_dict import ThreadSafeDict # Command to execute - list of strings # the first element is always "airflow". @@ -114,7 +114,7 @@ class BaseExecutor(LoggingMixin): :param parallelism: how many jobs should run at one time. Set to ``0`` for infinity. """ - active_spans = None + active_spans = ThreadSafeDict() supports_ad_hoc_ti_run: bool = False supports_sentry: bool = False diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 19226c24ec64c..de400599d1d6e 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -70,6 +70,7 @@ from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.retries import MAX_DB_RETRIES, retry_db_transaction, run_with_db_retries from airflow.utils.session import NEW_SESSION, create_session, provide_session +from airflow.utils.span_status import SpanStatus from airflow.utils.sqlalchemy import ( is_lock_not_available_error, prohibit_commit, @@ -222,6 +223,8 @@ def register_signals(self) -> None: def _exit_gracefully(self, signum: int, frame: FrameType | None) -> None: """Clean up processor_agent to avoid leaving orphan processes.""" + self._cleanup_active_spans_before_process_exit() + if not _is_parent_process(): # Only the parent process should perform the cleanup. return @@ -803,12 +806,28 @@ def process_executor_events( ti.pid, ) + # Each scheduler processes its own executor events. + # If the executor of the scheduler has run the task, + # then only that scheduler will mark the task as finished + # unless the process dies and the task has to be adopted by another scheduler. + # There is no point in notifying another scheduler that the task span has to be ended. + # But because the span might not be ended immediately, + # the task end time must be set as the span end time. active_ti_span = cls.active_ti_spans.get(ti.key) - if conf.getboolean("traces", "otel_use_context_propagation") and active_ti_span is not None: - cls._set_span_attrs__process_executor_events(span=active_ti_span, state=state, ti=ti) - # End the span and remove it from the active_spans dict. - active_ti_span.end() - cls.active_ti_spans.delete(ti.key) + if conf.getboolean("traces", "otel_use_context_propagation"): + if active_ti_span is not None: + cls._set_span_attrs__process_executor_events(span=active_ti_span, state=state, ti=ti) + # End the span and remove it from the active_ti_spans dict. + active_ti_span.end(end_time=datetime_to_nano(ti.end_date)) + cls.active_ti_spans.delete(ti.key) + ti.set_span_status(status=SpanStatus.ENDED, session=session) + else: + # TODO: check if this is needed in case the task is adopted or + # finished not by the executor that started it. + if ti.span_status == SpanStatus.ACTIVE: + # Another scheduler has started the span. + # Update the SpanStatus to let the process know that it must end it. + ti.set_span_status(status=SpanStatus.SHOULD_END, session=session) with Trace.start_span_from_taskinstance(ti=ti) as span: cls._set_span_attrs__process_executor_events(span, state, ti) @@ -1029,6 +1048,85 @@ def _update_dag_run_state_for_paused_dags(self, session: Session = NEW_SESSION) except Exception as e: # should not fail the scheduler self.log.exception("Failed to update dag run state for paused dags due to %s", e) + @provide_session + def _cleanup_active_spans_before_process_exit(self, session: Session = NEW_SESSION): + # No need to do a commit for every update. The annotation will commit all of them once at the end. + for run_id, span in self.active_dagrun_spans.get_all().items(): + span.end() + dag_run: DagRun = session.scalars(select(DagRun).where(DagRun.run_id == run_id)).one() + if dag_run.state in State.finished_dr_states: + dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + else: + dag_run.set_span_status( + status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False + ) + initial_dag_run_context = Trace.extract(dag_run.context_carrier) + with Trace.start_child_span( + span_name="current_scheduler_exited", parent_context=initial_dag_run_context + ) as s: + s.set_attribute("trace_status", "needs continuance") + + for key, span in self.active_ti_spans.get_all().items(): + span.end() + # Can't compare the key directly because the try_number or the map_index might not be the same. + ti: TaskInstance = session.scalars( + select(TaskInstance).where( + TaskInstance.dag_id == key.dag_id, + TaskInstance.task_id == key.task_id, + TaskInstance.run_id == key.run_id, + ) + ).one() + if ti.state in State.finished: + ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + else: + ti.set_span_status(status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False) + + self.active_dagrun_spans.clear() + self.active_ti_spans.clear() + + @provide_session + def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): + # The scheduler that starts a dag_run or a task is also the one that starts the spans. + # Each scheduler should end the spans that it has started. + # + # Otel spans are designed so that only the process that starts them, + # has full control over their lifecycle. + # This also means that the process that started them, is the only one that can end them. + # + # If another scheduler has finished processing a dag_run or a task and there is a reference + # on the active_spans dictionary, then that means that the current scheduler started + # the span, and therefore must end it. + dag_runs_should_end: list[DagRun] = session.scalars( + select(DagRun).where(DagRun.span_status == SpanStatus.SHOULD_END) + ).all() + tis_should_end: list[TaskInstance] = session.scalars( + select(TaskInstance).where(TaskInstance.span_status == SpanStatus.SHOULD_END) + ).all() + + for dag_run in dag_runs_should_end: + active_dagrun_span = self.active_dagrun_spans.get(dag_run.run_id) + if active_dagrun_span is not None: + if dag_run.state in State.finished_dr_states: + dagv = session.scalar(select(DagVersion).where(DagVersion.id == dag_run.dag_version_id)) + DagRun.set_dagrun_span_attrs(span=active_dagrun_span, dag_run=dag_run, dagv=dagv) + + active_dagrun_span.end(end_time=datetime_to_nano(dag_run.end_date)) + else: + active_dagrun_span.end() + self.active_dagrun_spans.delete(dag_run.run_id) + dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + + for ti in tis_should_end: + active_ti_span = self.active_ti_spans.get(ti.key) + if active_ti_span is not None: + if ti.state in State.finished: + self._set_span_attrs__process_executor_events(span=active_ti_span, state=ti.state, ti=ti) + active_ti_span.end(end_time=datetime_to_nano(ti.end_date)) + else: + active_ti_span.end() + self.active_ti_spans.delete(ti.key) + ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + def _run_scheduler_loop(self) -> None: """ Harvest DAG parsing results, queue tasks, and perform executor heartbeat; the actual scheduler loop. @@ -1108,9 +1206,15 @@ def _run_scheduler_loop(self) -> None: self.log.debug("Waiting for processors to finish since we're using sqlite") self.processor_agent.wait_until_finished() - # This is passing a reference to the dictionary, making it shared. - # Any changes made by a dag_run instance, will also be reflected to the dictionary of this class. - DagRun.set_active_spans(active_spans=self.active_dagrun_spans) + # This is using a new session. It's not done under the session block below + # because the changes need to be committed and the block runs expunge_all before the commit. + self._end_spans_of_externally_ended_ops() + + # Pass a reference to the dictionary. + # Any changes made by a dag_run instance, will be reflected to the dictionaries of this class. + DagRun.set_active_spans( + active_dagrun_spans=self.active_dagrun_spans, active_ti_spans=self.active_ti_spans + ) # local import due to type_checking. from airflow.executors.base_executor import BaseExecutor diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 67e08036a16bc..47a607d1e24c7 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -87,6 +87,7 @@ from airflow.utils.log.logging_mixin import LoggingMixin from airflow.utils.retries import retry_db_transaction from airflow.utils.session import NEW_SESSION, provide_session +from airflow.utils.span_status import SpanStatus from airflow.utils.sqlalchemy import ( ExtendedJSON, UtcDateTime, @@ -145,7 +146,8 @@ class DagRun(Base, LoggingMixin): external trigger (i.e. manual runs). """ - active_spans = ThreadSafeDict() + active_dagrun_spans = ThreadSafeDict() + active_ti_spans = ThreadSafeDict() __tablename__ = "dag_run" @@ -193,6 +195,7 @@ class DagRun(Base, LoggingMixin): # Span context carrier, used for context propagation. context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) + span_status = Column(String(50), default=SpanStatus.NOT_STARTED) # Remove this `if` after upgrading Sphinx-AutoAPI if not TYPE_CHECKING and "BUILDING_AIRFLOW_DOCS" in os.environ: @@ -321,8 +324,9 @@ def logical_date(self) -> datetime: return self.execution_date @classmethod - def set_active_spans(cls, active_spans: ThreadSafeDict): - cls.active_spans = active_spans + def set_active_spans(cls, active_dagrun_spans: ThreadSafeDict, active_ti_spans: ThreadSafeDict): + cls.active_dagrun_spans = active_dagrun_spans + cls.active_ti_spans = active_ti_spans def get_state(self): return self._state @@ -887,7 +891,7 @@ def is_effective_leaf(task): return leaf_tis @staticmethod - def _set_dagrun_span_attrs(span: Span | EmptySpan, dag_run: DagRun, dagv: DagVersion): + def set_dagrun_span_attrs(span: Span | EmptySpan, dag_run: DagRun, dagv: DagVersion): if dag_run._state is DagRunState.FAILED: span.set_attribute("airflow.dag_run.error", True) @@ -945,6 +949,7 @@ def update_state( """ # Callback to execute in case of Task Failures callback: DagCallbackRequest | None = None + print(f"x: dag_state: {self._state} | span_status: {self.span_status}") class _UnfinishedStates(NamedTuple): tis: Sequence[TI] @@ -1067,26 +1072,79 @@ def recalculate(self) -> _UnfinishedStates: # finally, if the leaves aren't done, the dag is still running else: - # If there is no value in active_spans, then the span hasn't already been started. + # If there is no value in active_dagrun_spans, then the span hasn't already been started. if ( self.otel_use_context_propagation - and self.active_spans is not None - and (self.active_spans.get(self.run_id) is None) + and self.active_dagrun_spans is not None + and self.active_dagrun_spans.get(self.run_id) is None ): - span = Trace.start_root_span( - span_name=f"{self.dag_id}{CTX_PROP_SUFFIX}", - component=f"dag{CTX_PROP_SUFFIX}", - start_time=self.queued_at, - start_as_current=False, - ) - carrier = Trace.inject() - self.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) - # Set the span in a synchronized dictionary, so that the variable can be used to end the span. - self.active_spans.set(self.run_id, span) - self.log.debug( - "DagRun span has been started and the injected context_carrier is: %s", - self.context_carrier, - ) + if ( + self.span_status == SpanStatus.NOT_STARTED + or self.span_status == SpanStatus.NEEDS_CONTINUANCE + ): + span = None + continue_ti_spans = False + if self.span_status == SpanStatus.NOT_STARTED: + span = Trace.start_root_span( + span_name=f"{self.dag_id}{CTX_PROP_SUFFIX}", + component=f"dag{CTX_PROP_SUFFIX}", + start_time=self.queued_at, + start_as_current=False, + ) + elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: + # Use the existing context_carrier to set the initial dag_run span as the parent. + parent_context = Trace.extract(self.context_carrier) + with Trace.start_child_span( + span_name="new_scheduler", parent_context=parent_context + ) as s: + s.set_attribute("trace_status", "continued") + + span = Trace.start_child_span( + span_name=f"{self.dag_id}_continued{CTX_PROP_SUFFIX}", + parent_context=parent_context, + component=f"dag{CTX_PROP_SUFFIX}", + # No start time + start_as_current=False, + ) + # After this span is started, the context_carrier will be replaced by the new one. + # New task span will use this span as the parent. + continue_ti_spans = True + carrier = Trace.inject() + self.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) + self.set_span_status(status=SpanStatus.ACTIVE, session=session, with_commit=False) + # Set the span in a synchronized dictionary, so that the variable can be used to end the span. + self.active_dagrun_spans.set(self.run_id, span) + self.log.debug( + "DagRun span has been started and the injected context_carrier is: %s", + self.context_carrier, + ) + # Start TI spans that also need continuance. + if continue_ti_spans: + new_dagrun_context = Trace.extract(self.context_carrier) + for ti in tis: + if ti.span_status == SpanStatus.NEEDS_CONTINUANCE: + ti_span = Trace.start_child_span( + span_name=f"{ti.task_id}_continued{CTX_PROP_SUFFIX}", + parent_context=new_dagrun_context, + start_as_current=False, + ) + ti_carrier = Trace.inject() + ti.set_context_carrier( + context_carrier=ti_carrier, session=session, with_commit=False + ) + ti.set_span_status( + status=SpanStatus.ACTIVE, session=session, with_commit=False + ) + self.active_ti_spans.set(ti.key, ti_span) + else: + self.log.info( + "Found span_status '%s', while updating state for dag_run '%s'", + self.span_status, + self.run_id, + ) + + for ti in schedulable_tis: + ti.dag_run = self self.set_state(DagRunState.RUNNING) @@ -1119,8 +1177,8 @@ def recalculate(self) -> _UnfinishedStates: ) if self.otel_use_context_propagation: - if self.active_spans is not None: - active_span = self.active_spans.get(self.run_id) + if self.active_dagrun_spans is not None: + active_span = self.active_dagrun_spans.get(self.run_id) if active_span is not None: self.log.debug( "Found active span with span_id: %s, for dag_id: %s, run_id: %s, state: %s", @@ -1130,20 +1188,29 @@ def recalculate(self) -> _UnfinishedStates: self.state, ) - self._set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) + self.set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) active_span.end() # Remove the span from the dict. - self.active_spans.delete(self.run_id) + self.active_dagrun_spans.delete(self.run_id) + self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: - self.log.debug( - "No active span has been found for dag_id: %s, run_id: %s, state: %s", - self.dag_id, - self.run_id, - self.state, - ) + if self.span_status == SpanStatus.ACTIVE: + # Another scheduler has started the span. + # Update the DB SpanStatus to notify the owner to end it. + self.set_span_status( + status=SpanStatus.SHOULD_END, session=session, with_commit=False + ) + else: + self.log.debug( + "No active span has been found for dag_id: %s, run_id: %s, state: %s", + self.dag_id, + self.run_id, + self.state, + ) with Trace.start_span_from_dagrun(dagrun=self) as span: - self._set_dagrun_span_attrs(span=span, dag_run=self, dagv=dagv) + if span is not None: # To avoid a static-code check error. + self.set_dagrun_span_attrs(span=span, dag_run=self, dagv=dagv) session.flush() @@ -1247,6 +1314,44 @@ def set_context_carrier( dag_run=self, context_carrier=context_carrier, session=session, with_commit=with_commit ) + @staticmethod + @internal_api_call + def _set_span_status(dag_run: DagRun, status: SpanStatus, session: Session, with_commit: bool) -> bool: + if not isinstance(dag_run, DagRun): + dag_run = session.scalars( + select(DagRun).where( + DagRun.dag_id == dag_run.dag_id, + DagRun.run_id == dag_run.run_id, + ) + ).one() + + if dag_run.span_status == status: + return False + + dag_run.log.debug("Setting dag_run span_status for run_id: %s", dag_run.run_id) + dag_run.span_status = status + + session.merge(dag_run) + + if with_commit: + session.commit() + + return True + + @provide_session + def set_span_status( + self, status: SpanStatus, session: Session = NEW_SESSION, with_commit: bool = False + ) -> bool: + """ + Set DagRun span_status. + + :param status: dict with the injected carrier to set for the dag_run + :param session: SQLAlchemy ORM Session + :param with_commit: should the status be committed? + :return: has the span_status been changed? + """ + return self._set_span_status(dag_run=self, status=status, session=session, with_commit=with_commit) + def notify_dagrun_state_changed(self, msg: str = ""): if self.state == DagRunState.RUNNING: get_listener_manager().hook.on_dag_run_running(dag_run=self, msg=msg) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index a9f9ad4e3d7df..5c0818a7b8087 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -129,6 +129,7 @@ from airflow.utils.platform import getuser from airflow.utils.retries import run_with_db_retries from airflow.utils.session import NEW_SESSION, create_session, provide_session +from airflow.utils.span_status import SpanStatus from airflow.utils.sqlalchemy import ( ExecutorConfigType, ExtendedJSON, @@ -829,6 +830,7 @@ def _set_ti_attrs(target, source, include_dag_run=False): target.next_method = source.next_method target.next_kwargs = source.next_kwargs target.dag_version_id = source.dag_version_id + target.context_carrier = source.context_carrier if include_dag_run: target.execution_date = source.execution_date @@ -850,6 +852,7 @@ def _set_ti_attrs(target, source, include_dag_run=False): target.dag_run.dag_version_id = source.dag_run.dag_version_id target.dag_run.updated_at = source.dag_run.updated_at target.dag_run.log_template_id = source.dag_run.log_template_id + target.dag_run.context_carrier = source.dag_run.context_carrier def _refresh_from_db( @@ -890,6 +893,14 @@ def _refresh_from_db( # to also update dag_run information as it might not be available. We cannot always do it in # case ti is TaskInstance, because it might be detached/ not loaded yet and dag_run might # not be available. + if not include_dag_run: + inspector = inspect(ti) + # Check if the ti is detached or the dag_run isn't loaded. + if not inspector.detached and "dag_run" not in inspector.unloaded: + # It's best to include the dag_run whenever possible, + # in case there are changes to the span context_carrier. + include_dag_run = True + _set_ti_attrs(task_instance, ti, include_dag_run=include_dag_run) else: task_instance.state = None @@ -1867,6 +1878,7 @@ class TaskInstance(Base, LoggingMixin): updated_at = Column(UtcDateTime, default=timezone.utcnow, onupdate=timezone.utcnow) rendered_map_index = Column(String(250)) context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) + span_status = Column(String(50), default=SpanStatus.NOT_STARTED) external_executor_id = Column(StringID()) @@ -2419,6 +2431,47 @@ def set_context_carrier( ti=self, context_carrier=context_carrier, session=session, with_commit=with_commit ) + @staticmethod + @internal_api_call + def _set_span_status( + ti: TaskInstance | TaskInstancePydantic, status: SpanStatus, session: Session, with_commit: bool + ) -> bool: + if not isinstance(ti, TaskInstance): + ti = session.scalars( + select(TaskInstance).where( + TaskInstance.task_id == ti.task_id, + TaskInstance.dag_id == ti.dag_id, + TaskInstance.run_id == ti.run_id, + ) + ).one() + + if ti.span_status == status: + return False + + ti.log.debug("Setting task span_status for %s", ti.task_id) + ti.span_status = status + + session.merge(ti) + + if with_commit: + session.commit() + + return True + + @provide_session + def set_span_status( + self, status: SpanStatus, session: Session = NEW_SESSION, with_commit: bool = False + ) -> bool: + """ + Set TaskInstance span_status. + + :param status: dict with the injected carrier to set for the dag_run + :param session: SQLAlchemy ORM Session + :param with_commit: should the status be committed? + :return: has the span_status been changed? + """ + return self._set_span_status(ti=self, status=status, session=session, with_commit=with_commit) + @property def is_premature(self) -> bool: """Returns whether a task is in UP_FOR_RETRY state and its retry interval has elapsed.""" @@ -3901,6 +3954,7 @@ def __init__( run_as_user: str | None = None, priority_weight: int | None = None, context_carrier: dict | None = None, + span_status: str | None = None, ): self.dag_id = dag_id self.task_id = task_id @@ -3918,6 +3972,7 @@ def __init__( self.queue = queue self.key = key self.context_carrier = context_carrier + self.span_status = span_status def __repr__(self) -> str: attrs = ", ".join(f"{k}={v!r}" for k, v in self.__dict__.items()) diff --git a/airflow/models/taskinstancehistory.py b/airflow/models/taskinstancehistory.py index f79f74eaf6535..a8a6fb5f32c4b 100644 --- a/airflow/models/taskinstancehistory.py +++ b/airflow/models/taskinstancehistory.py @@ -38,6 +38,7 @@ from airflow.models.base import Base, StringID from airflow.utils import timezone from airflow.utils.session import NEW_SESSION, provide_session +from airflow.utils.span_status import SpanStatus from airflow.utils.sqlalchemy import ( ExecutorConfigType, ExtendedJSON, @@ -85,6 +86,7 @@ class TaskInstanceHistory(Base): updated_at = Column(UtcDateTime, default=timezone.utcnow, onupdate=timezone.utcnow) rendered_map_index = Column(String(250)) context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) + span_status = Column(String(50), default=SpanStatus.NOT_STARTED) external_executor_id = Column(StringID()) trigger_id = Column(Integer) diff --git a/airflow/serialization/pydantic/dag_run.py b/airflow/serialization/pydantic/dag_run.py index ba2aa8dc110d4..c11db18ed2ab4 100644 --- a/airflow/serialization/pydantic/dag_run.py +++ b/airflow/serialization/pydantic/dag_run.py @@ -59,6 +59,7 @@ class DagRunPydantic(BaseModelPydantic): log_template_id: Optional[int] triggered_by: Optional[DagRunTriggeredByType] context_carrier: Optional[dict] + span_status: Optional[str] model_config = ConfigDict(from_attributes=True, arbitrary_types_allowed=True) diff --git a/airflow/serialization/pydantic/taskinstance.py b/airflow/serialization/pydantic/taskinstance.py index 87fbce39c36da..5133651f708b8 100644 --- a/airflow/serialization/pydantic/taskinstance.py +++ b/airflow/serialization/pydantic/taskinstance.py @@ -124,6 +124,7 @@ class TaskInstancePydantic(BaseModelPydantic, LoggingMixin): raw: Optional[bool] is_trigger_log_context: Optional[bool] context_carrier: Optional[dict] + span_status: Optional[str] model_config = ConfigDict(from_attributes=True, arbitrary_types_allowed=True) @property diff --git a/airflow/traces/tracer.py b/airflow/traces/tracer.py index 7a533dae7a3e9..e726b2fdd2307 100644 --- a/airflow/traces/tracer.py +++ b/airflow/traces/tracer.py @@ -22,6 +22,8 @@ import socket from typing import TYPE_CHECKING, Any, Callable +from sqlalchemy.util.compat import contextmanager + from airflow.configuration import conf from airflow.typing_compat import Protocol @@ -324,6 +326,27 @@ def __init__(cls, *args, **kwargs) -> None: else: cls.__class__.factory = EmptyTrace + @classmethod + @contextmanager + def tracer_with_params(cls, *args, **kwargs): + """Context manager to temporarily set parameters for the tracer factory.""" + initial_factory = cls.factory + initial_instance = cls.instance + try: + if conf.has_option("traces", "otel_on") and conf.getboolean("traces", "otel_on"): + from airflow.traces import otel_tracer + + cls.factory = lambda: otel_tracer.get_otel_tracer(*args, **kwargs) + else: + cls.factory = EmptyTrace + # Reset the instance to ensure the new parameters are used + cls.instance = None + yield + finally: + # Restore the factory and the instance, to their initial values. + cls.factory = initial_factory + cls.instance = initial_instance + @classmethod def get_constant_tags(cls) -> str | None: """Get constant tags to add to all traces.""" diff --git a/airflow/utils/span_status.py b/airflow/utils/span_status.py new file mode 100644 index 0000000000000..4d018e72aac87 --- /dev/null +++ b/airflow/utils/span_status.py @@ -0,0 +1,33 @@ +# +# 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 + +from enum import Enum + + +class SpanStatus(str, Enum): + """All possible statuses for a span.""" + + NOT_STARTED = "not_started" + ACTIVE = "active" + ENDED = "ended" + SHOULD_END = "should_end" + NEEDS_CONTINUANCE = "needs_continuance" + + def __str__(self) -> str: + return self.value diff --git a/airflow/utils/thread_safe_dict.py b/airflow/utils/thread_safe_dict.py index d6581258d3e87..ecd1aed202c62 100644 --- a/airflow/utils/thread_safe_dict.py +++ b/airflow/utils/thread_safe_dict.py @@ -39,6 +39,10 @@ def delete(self, key): if key in self.sync_dict: del self.sync_dict[key] + def clear(self): + with self.thread_lock: + self.sync_dict.clear() + def get_all(self): with self.thread_lock: # Return a copy to avoid exposing the internal dictionary. diff --git a/scripts/ci/docker-compose/integration-otel.yml b/scripts/ci/docker-compose/integration-otel.yml index 7a635c17c7d22..d2cd759ad7f59 100644 --- a/scripts/ci/docker-compose/integration-otel.yml +++ b/scripts/ci/docker-compose/integration-otel.yml @@ -67,6 +67,7 @@ services: airflow: environment: + - INTEGRATION_OTEL=true - AIRFLOW__METRICS__OTEL_ON=True - AIRFLOW__METRICS__OTEL_HOST=breeze-otel-collector - AIRFLOW__METRICS__OTEL_PORT=4318 diff --git a/tests/integration/otel/dags/otel_test_dag.py b/tests/integration/otel/dags/otel_test_dag.py index 36349ab863e4c..4af68fc799ea9 100644 --- a/tests/integration/otel/dags/otel_test_dag.py +++ b/tests/integration/otel/dags/otel_test_dag.py @@ -27,7 +27,7 @@ from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import Trace -logger = logging.getLogger("airflow.test_dag") +logger = logging.getLogger("airflow.otel_test_dag") args = { "owner": "airflow", diff --git a/tests/integration/otel/dags/otel_test_dag_with_pause.py b/tests/integration/otel/dags/otel_test_dag_with_pause.py new file mode 100644 index 0000000000000..fc3a0205f00c5 --- /dev/null +++ b/tests/integration/otel/dags/otel_test_dag_with_pause.py @@ -0,0 +1,157 @@ +# 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 logging +import os +from datetime import datetime + +from opentelemetry import trace +from sqlalchemy import select + +from airflow import DAG +from airflow.models import TaskInstance +from airflow.providers.standard.operators.python import PythonOperator +from airflow.traces import otel_tracer +from airflow.traces.otel_tracer import CTX_PROP_SUFFIX +from airflow.traces.tracer import Trace +from airflow.utils.session import create_session + +logger = logging.getLogger("airflow.otel_test_dag_with_pause") + +args = { + "owner": "airflow", + "start_date": datetime(2024, 9, 1), + "retries": 0, +} + +# DAG definition. +with DAG( + "otel_test_dag_with_pause", + default_args=args, + schedule=None, + catchup=False, +) as dag: + # Tasks. + def task1_func(**dag_context): + logger.info("Starting Task_1.") + + ti = dag_context["ti"] + context_carrier = ti.context_carrier + + dag_folder = os.path.dirname(os.path.abspath(__file__)) + control_file = os.path.join(dag_folder, "dag_control.txt") + + # Create the file and write 'pause' to it. + with open(control_file, "w") as file: + file.write("pause") + + # Pause execution until the word 'pause' is replaced on the file. + while True: + # If there is an exception, then writing to the file failed. Let it exit. + file_contents = None + with open(control_file) as file: + file_contents = file.read() + + if "pause" in file_contents: + logger.info("Task has been paused.") + continue + else: + logger.info("Resuming task execution.") + # Break the loop and finish with the task execution. + break + + otel_task_tracer = otel_tracer.get_otel_tracer_for_task(Trace) + tracer_provider = otel_task_tracer.get_otel_tracer_provider() + + if context_carrier is not None: + logger.info("Found ti.context_carrier: %s.", context_carrier) + logger.info("Extracting the span context from the context_carrier.") + + # If the task takes too long to execute, then the ti should be read from the db + # to make sure that the initial context_carrier is the same. + with create_session() as session: + session_ti: TaskInstance = session.scalars( + select(TaskInstance).where( + TaskInstance.task_id == ti.task_id, + TaskInstance.run_id == ti.run_id, + ) + ).one() + context_carrier = session_ti.context_carrier + + parent_context = Trace.extract(context_carrier) + with otel_task_tracer.start_child_span( + span_name=f"{ti.task_id}_sub_span1{CTX_PROP_SUFFIX}", + parent_context=parent_context, + component=f"dag{CTX_PROP_SUFFIX}", + ) as s1: + s1.set_attribute("attr1", "val1") + logger.info("From task sub_span1.") + + with otel_task_tracer.start_child_span(f"{ti.task_id}_sub_span2{CTX_PROP_SUFFIX}") as s2: + s2.set_attribute("attr2", "val2") + logger.info("From task sub_span2.") + + tracer = trace.get_tracer("trace_test.tracer", tracer_provider=tracer_provider) + with tracer.start_as_current_span(name=f"{ti.task_id}_sub_span3{CTX_PROP_SUFFIX}") as s3: + s3.set_attribute("attr3", "val3") + logger.info("From task sub_span3.") + + with create_session() as session: + session_ti: TaskInstance = session.scalars( + select(TaskInstance).where( + TaskInstance.task_id == ti.task_id, + TaskInstance.run_id == ti.run_id, + ) + ).one() + context_carrier = session_ti.context_carrier + parent_context = Trace.extract(context_carrier) + + with otel_task_tracer.start_child_span( + span_name=f"{ti.task_id}_sub_span4{CTX_PROP_SUFFIX}", + parent_context=parent_context, + component=f"dag{CTX_PROP_SUFFIX}", + ) as s4: + s4.set_attribute("attr4", "val4") + logger.info("From task sub_span4.") + + # Cleanup the control file. + if os.path.exists(control_file): + os.remove(control_file) + print("Control file has been cleaned up.") + + logger.info("Task_1 finished.") + + def task2_func(): + logger.info("Starting Task_2.") + for i in range(3): + logger.info("Task_2, iteration '%d'.", i) + logger.info("Task_2 finished.") + + # Task operators. + t1 = PythonOperator( + task_id="task_1", + python_callable=task1_func, + ) + + t2 = PythonOperator( + task_id="task_2", + python_callable=task2_func, + ) + + # Dependencies. + t1 >> t2 diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index 50d4ec16329f3..16b5db2df0a79 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -18,6 +18,7 @@ import logging import os +import signal import subprocess import time @@ -26,10 +27,12 @@ from airflow.executors import executor_loader from airflow.executors.executor_utils import ExecutorName -from airflow.models import DagBag, DagRun +from airflow.models import DAG, DagBag, DagRun from airflow.models.serialized_dag import SerializedDagModel +from airflow.models.taskinstance import TaskInstance from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.utils.session import create_session +from airflow.utils.span_status import SpanStatus from airflow.utils.state import State from tests.integration.otel.test_utils import ( @@ -48,16 +51,34 @@ @pytest.mark.integration("redis") @pytest.mark.backend("postgres") class TestOtelIntegration: + """ + This test is using a ConsoleSpanExporter so that it can capture + the spans from the stdout and run assertions on them. + + It can also be used with otel and jaeger for manual testing. + To export the spans to otel and visualize them with jaeger, + - start breeze with '--integration otel' + - run on the shell 'export use_otel=true' + - run the test + - check 'http://localhost:36686/' + + To get span dumps on the stdout, run 'export log_level=debug'. + """ + test_dir = os.path.dirname(os.path.abspath(__file__)) dag_folder = os.path.join(test_dir, "dags") + use_otel = os.getenv("use_otel", default="false") + log_level = os.getenv("log_level", default="none") + @classmethod def setup_class(cls): os.environ["AIRFLOW__TRACES__OTEL_ON"] = "True" - os.environ["AIRFLOW__TRACES__OTEL_HOST"] = "localhost" + os.environ["AIRFLOW__TRACES__OTEL_HOST"] = "breeze-otel-collector" os.environ["AIRFLOW__TRACES__OTEL_PORT"] = "4318" - os.environ["AIRFLOW__TRACES__OTEL_DEBUGGING_ON"] = "True" - os.environ["AIRFLOW__TRACES__OTEL_TASK_LOG_EVENT"] = "True" + if cls.use_otel != "true": + os.environ["AIRFLOW__TRACES__OTEL_DEBUGGING_ON"] = "True" + os.environ["AIRFLOW__TRACES__OTEL_USE_CONTEXT_PROPAGATION"] = "True" os.environ["AIRFLOW__SCHEDULER__STANDALONE_DAG_PROCESSOR"] = "False" @@ -82,8 +103,8 @@ def test_dag_spans_with_context_propagation(self, monkeypatch, celery_worker_env Test that a DAG runs successfully and exports the correct spans. Integration with a scheduler, a celery worker, a postgres db and a redis broker. """ - # Uncomment to enable debug mode and get span and db dumps on the output. - # log.setLevel(logging.DEBUG) + if self.log_level == "debug": + log.setLevel(logging.DEBUG) celery_command_args = [ "celery", @@ -161,7 +182,7 @@ def test_dag_spans_with_context_propagation(self, monkeypatch, celery_worker_env subprocess.run(trigger_command, check=True, env=os.environ.copy()) # Wait timeout for the DAG run to complete. - max_wait_time = 60 # seconds + max_wait_time = 90 # seconds start_time = time.time() dag_run_state = None @@ -208,9 +229,553 @@ def test_dag_spans_with_context_propagation(self, monkeypatch, celery_worker_env log.debug("out-start --\n%s\n-- out-end", out) log.debug("err-start --\n%s\n-- err-end", err) - # Dag run should have succeeded. Test the spans in the output. + if self.use_otel != "true": + # Dag run should have succeeded. Test the spans from the output. + self.check_spans_without_continuance(output=out, dag=dag) + + def test_dag_spans_with_different_schedulers_processing_the_same_dag( + self, monkeypatch, celery_worker_env_vars, capfd, session + ): + """ + Similar to test_dag_spans_with_context_propagation but this test will start with one scheduler, + and during the dag execution, it will stop the process and start a new one. + + A txt file will be used for signaling the test and the dag in order to make sure that + the 1st scheduler is stopped while the first task is executing and that + the 2nd scheduler picks up the task and dag processing. + The steps will be + - The dag starts running, creates the file with a signal word and waits until the word is changed. + - The test checks if the file exist, stops the scheduler, starts a new scheduler and updates the file. + - The dag gets the update and continues until the task is finished. + At this point, the second scheduler should handle the rest of the dag processing. + """ + if self.log_level == "debug": + log.setLevel(logging.DEBUG) + + celery_command_args = [ + "celery", + "--app", + "airflow.providers.celery.executors.celery_executor.app", + "worker", + "--concurrency", + "1", + "--loglevel", + "INFO", + ] + + scheduler_command_args = [ + "airflow", + "scheduler", + ] + + celery_worker_process = None + scheduler_process_1 = None + scheduler_process_2 = None + try: + # Start the processes here and not as fixtures, so that the test can capture their output. + celery_worker_process = subprocess.Popen( + celery_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) + + scheduler_process_1 = subprocess.Popen( + scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) + + # Wait to ensure both processes have started. + time.sleep(10) + + execution_date = pendulum.now("UTC") + + # Load DAGs from the dag directory. + dag_bag = DagBag(dag_folder=self.dag_folder, include_examples=False) + + dag_id = "otel_test_dag" + dag = dag_bag.get_dag(dag_id) + + assert dag is not None, f"DAG with ID {dag_id} not found." + + with create_session() as session: + # Sync the DAG to the database. + dag.sync_to_db(session=session) + # Manually serialize the dag and write it to the db to avoid a db error. + SerializedDagModel.write_dag(dag, session=session) + session.commit() + + unpause_command = ["airflow", "dags", "unpause", dag_id] + + # Unpause the dag using the cli. + subprocess.run(unpause_command, check=True, env=os.environ.copy()) + + run_id = f"manual__{execution_date.isoformat()}" + + trigger_command = [ + "airflow", + "dags", + "trigger", + dag_id, + "--run-id", + run_id, + "--exec-date", + execution_date.isoformat(), + ] + + # Trigger the dag using the cli. + subprocess.run(trigger_command, check=True, env=os.environ.copy()) + + with create_session() as session: + tis: list[TaskInstance] = dag.get_task_instances(session=session) + + task_1 = tis[0] + + while True: + with create_session() as session: + ti = ( + session.query(TaskInstance) + .filter( + TaskInstance.task_id == task_1.task_id, + TaskInstance.run_id == task_1.run_id, + ) + .first() + ) + + if ti is None: + continue + + # Wait until the task has been finished. + if ti.state in State.finished: + break + + with capfd.disabled(): + # When we pause the scheduler1 thread, capfd keeps trying to read the + # file descriptors for the process and ends up freezing the test. + # There won't be any exported spans from the following code, + # so not capturing the output, doesn't make a difference. + scheduler_process_1.send_signal(signal.SIGSTOP) + + scheduler_process_2 = subprocess.Popen( + scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) + + with create_session() as session: + dag_run = ( + session.query(DagRun) + .filter( + DagRun.dag_id == dag_id, + DagRun.run_id == run_id, + ) + .first() + ) + + assert ( + dag_run.state == State.RUNNING + ), f"Dag Run state isn't RUNNING. State: {dag_run.state}" + assert ( + dag_run.span_status == SpanStatus.ACTIVE + ), f"Dag Run span_status isn't ACTIVE. Span_status: {dag_run.span_status}" + + # Wait for scheduler2 to be up and running. + time.sleep(10) + + # Wait timeout for the DAG run to complete. + max_wait_time = 120 # seconds + start_time = time.time() + + dag_run_state = None + dag_run_span_status = None + + while time.time() - start_time < max_wait_time: + with create_session() as session: + dag_run = ( + session.query(DagRun) + .filter( + DagRun.dag_id == dag_id, + DagRun.run_id == run_id, + ) + .first() + ) + + if dag_run is None: + time.sleep(5) + continue + + dag_run_state = dag_run.state + # log.info("DAG Run state: %s.", dag_run_state) + + dag_run_span_status = dag_run.span_status + # log.info("DAG Run span status: %s.", dag_run_span_status) + + if dag_run_state in [State.SUCCESS, State.FAILED]: + break + + time.sleep(5) + + if logging.root.level == logging.DEBUG: + with create_session() as session: + dump_airflow_metadata_db(session) + + assert ( + dag_run_state == State.SUCCESS + ), f"Dag run did not complete successfully. Final state: {dag_run_state}." + assert dag_run_span_status == SpanStatus.SHOULD_END, ( + f"Dag run has been finished by scheduler2 but span status isn't {SpanStatus.SHOULD_END}." + f"Actual status: {dag_run_span_status}." + ) + + scheduler_process_1.send_signal(signal.SIGCONT) + scheduler_process_2.terminate() + + # Wait for the scheduler to start again and continue running. + time.sleep(10) + + max_wait_time = 30 # seconds + start_time = time.time() + + dag_run_span_status = None + + while time.time() - start_time < max_wait_time: + with create_session() as session: + dag_run = ( + session.query(DagRun) + .filter( + DagRun.dag_id == dag_id, + DagRun.run_id == run_id, + ) + .first() + ) + + if dag_run is None: + time.sleep(5) + continue + + dag_run_span_status = dag_run.span_status + log.info("DAG Run span status: %s.", dag_run_span_status) + + if dag_run_span_status == SpanStatus.ENDED: + break + assert dag_run_span_status == SpanStatus.ENDED, ( + f"Scheduler1 should have ended the span and updated status to {SpanStatus.ENDED} but didn't." + f"Actual status: {dag_run_span_status}." + ) + + finally: + # Terminate the processes. + celery_worker_process.terminate() + celery_worker_process.wait() + + scheduler_process_1.terminate() + scheduler_process_1.wait() + + # scheduler_process_2.terminate() + scheduler_process_2.wait() + + out, err = capfd.readouterr() + log.debug("out-start --\n%s\n-- out-end", out) + log.debug("err-start --\n%s\n-- err-end", err) + + if self.use_otel != "true": + # Dag run should have succeeded. Test the spans in the output. + self.check_spans_without_continuance(output=out, dag=dag) + + def test_dag_spans_with_scheduler_ha_and_stopped_scheduler( + self, monkeypatch, celery_worker_env_vars, capfd, session + ): + """ + Similar to test_dag_spans_with_context_propagation but this test will start with one scheduler, + and during the dag execution, it will stop the process and start a new one. + + A txt file will be used for signaling the test and the dag in order to make sure that + the 1st scheduler is stopped while the first task is executing and that + the 2nd scheduler picks up the task and dag processing. + The steps will be + - The dag starts running, creates the file with a signal word and waits until the word is changed. + - The test checks if the file exist, stops the scheduler, starts a new scheduler and updates the file. + - The dag gets the update and continues until the task is finished. + At this point, the second scheduler should handle the rest of the dag processing. + """ + if self.log_level == "debug": + log.setLevel(logging.DEBUG) + + celery_command_args = [ + "celery", + "--app", + "airflow.providers.celery.executors.celery_executor.app", + "worker", + "--concurrency", + "1", + "--loglevel", + "INFO", + ] + + scheduler_command_args = [ + "airflow", + "scheduler", + ] + + celery_worker_process = None + scheduler_process_1 = None + scheduler_process_2 = None + try: + # Start the processes here and not as fixtures, so that the test can capture their output. + celery_worker_process = subprocess.Popen( + celery_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) + + scheduler_process_1 = subprocess.Popen( + scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) + + # Wait to ensure both processes have started. + time.sleep(10) + + execution_date = pendulum.now("UTC") + + # Load DAGs from the dag directory. + dag_bag = DagBag(dag_folder=self.dag_folder, include_examples=False) + + dag_id = "otel_test_dag_with_pause" + dag = dag_bag.get_dag(dag_id) + + assert dag is not None, f"DAG with ID {dag_id} not found." + + with create_session() as session: + # Sync the DAG to the database. + dag.sync_to_db(session=session) + # Manually serialize the dag and write it to the db to avoid a db error. + SerializedDagModel.write_dag(dag, session=session) + session.commit() + + unpause_command = ["airflow", "dags", "unpause", dag_id] + + # Unpause the dag using the cli. + subprocess.run(unpause_command, check=True, env=os.environ.copy()) + + run_id = f"manual__{execution_date.isoformat()}" + + trigger_command = [ + "airflow", + "dags", + "trigger", + dag_id, + "--run-id", + run_id, + "--exec-date", + execution_date.isoformat(), + ] + + # Trigger the dag using the cli. + subprocess.run(trigger_command, check=True, env=os.environ.copy()) + + # Control file path. + control_file = os.path.join(self.dag_folder, "dag_control.txt") + + while True: + try: + with open(control_file) as file: + # If it reaches inside the block, then the file exists and the test can read it. + break + except FileNotFoundError: + print("Control file not found. Waiting...") + time.sleep(1) + continue + + # Since, we are past the loop, then the file exists and the dag has been paused. + # Terminate scheduler1 and start scheduler2. + scheduler_process_1.terminate() + + scheduler_process_2 = subprocess.Popen( + scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) + + # Wait for scheduler2 to be up and running. + time.sleep(10) + + # Rewrite the file to unpause the dag. + with open(control_file, "w") as file: + file.write("continue") + + # Wait timeout for the DAG run to complete. + max_wait_time = 120 # seconds + start_time = time.time() + + dag_run_state = None + + while time.time() - start_time < max_wait_time: + with create_session() as session: + dag_run = ( + session.query(DagRun) + .filter( + DagRun.dag_id == dag_id, + DagRun.run_id == run_id, + ) + .first() + ) + + if dag_run is None: + time.sleep(5) + continue + + dag_run_state = dag_run.state + log.info("DAG Run state: %s.", dag_run_state) + + if dag_run_state in [State.SUCCESS, State.FAILED]: + break + + time.sleep(5) + + if logging.root.level == logging.DEBUG: + with create_session() as session: + dump_airflow_metadata_db(session) + + assert ( + dag_run_state == State.SUCCESS + ), f"Dag run did not complete successfully. Final state: {dag_run_state}." + finally: + # Terminate the processes. + celery_worker_process.terminate() + celery_worker_process.wait() + + scheduler_process_1.wait() + + scheduler_process_2.terminate() + scheduler_process_2.wait() + + out, err = capfd.readouterr() + log.debug("out-start --\n%s\n-- out-end", out) + log.debug("err-start --\n%s\n-- err-end", err) + + if self.use_otel != "true": + # Dag run should have succeeded. Test the spans in the output. + # Get a list of lines from the captured output. + output_lines = out.splitlines() + + # Filter the output, create a json obj for each span and then store them into dictionaries. + # One dictionary with only the root spans, and one with all the captured spans (root and otherwise). + root_span_dict, span_dict = extract_spans_from_output(output_lines) + # Generate a dictionary with parent child relationships. + # This is done by comparing the span_id of each root span with the parent_id of each non-root span. + parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) + + # The span hierarchy for dag 'otel_test_dag_with_pause' is + # dag span + # |_ task_1 span + # |_ scheduler_exited span + # |_ new_scheduler span + # |_ dag span (continued) + # |_ task_1 span (continued) + # |_ sub_span_1 + # |_ sub_span_2 + # |_ sub_span_3 + # |_ sub_span_4 + # |_ task_2 span + + task_instance_ids = dag.task_ids + task1_id = task_instance_ids[0] + task2_id = task_instance_ids[1] + + dag_root_span_name = f"{dag_id}{CTX_PROP_SUFFIX}" + + dag_root_span_children_names = [ + f"{task1_id}{CTX_PROP_SUFFIX}", + "current_scheduler_exited", + "new_scheduler", + f"{dag_id}_continued{CTX_PROP_SUFFIX}", + ] + + dag_continued_span_children_names = [ + f"{task1_id}_continued{CTX_PROP_SUFFIX}", + f"{task2_id}{CTX_PROP_SUFFIX}", + ] + + task1_span_children_names = [ + f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + f"{task1_id}_sub_span4{CTX_PROP_SUFFIX}", + ] + + # Single element lists. + task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}"] + task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3{CTX_PROP_SUFFIX}"] + + assert_span_name_belongs_to_root_span( + root_span_dict=root_span_dict, span_name=dag_root_span_name, should_succeed=True + ) + + # Check direct children of the root span. + assert_parent_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + parent_name=dag_root_span_name, + children_names=dag_root_span_children_names, + ) + + # Use a span name that exists, but it's not a direct child. + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_root_span_name, + child_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", + span_exists=True, + ) + + # Use a span name that doesn't exist at all. + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_root_span_name, + child_name=f"{task1_id}_non_existent", + span_exists=False, + ) + + # Check children of the continued dag span. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{dag_id}_continued{CTX_PROP_SUFFIX}", + children_names=dag_continued_span_children_names, + ) + + # Check children of the continued task1 span. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", + children_names=task1_span_children_names, + ) + + # Check children of task1 sub span1. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + children_names=task1_sub_span1_children_span_names, + ) + + # Check children of task1 sub span2. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", + children_names=task1_sub_span2_children_span_names, + ) + + def check_spans_without_continuance(self, output: str, dag: DAG): # Get a list of lines from the captured output. - output_lines = out.splitlines() + output_lines = output.splitlines() # Filter the output, create a json obj for each span and then store them into dictionaries. # One dictionary with only the root spans, and one with all the captured spans (root and otherwise). @@ -219,80 +784,87 @@ def test_dag_spans_with_context_propagation(self, monkeypatch, celery_worker_env # This is done by comparing the span_id of each root span with the parent_id of each non-root span. parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) - dag_span_name = str(dag_id + CTX_PROP_SUFFIX) - assert_span_name_belongs_to_root_span( - root_span_dict=root_span_dict, span_name=dag_span_name, should_succeed=True - ) + # Any spans generated under a task, are children of the task span. + # The span hierarchy for dag 'otel_test_dag' is + # dag span + # |_ task_1 span + # |_ sub_span_1 + # |_ sub_span_2 + # |_ sub_span_3 + # |_ sub_span_4 + # |_ task_2 span - non_existent_dag_span_name = str(dag_id + CTX_PROP_SUFFIX + "fail") - assert_span_name_belongs_to_root_span( - root_span_dict=root_span_dict, span_name=non_existent_dag_span_name, should_succeed=False - ) + dag_id = dag.dag_id - dag_children_span_names = [] task_instance_ids = dag.task_ids + task1_id = task_instance_ids[0] + task2_id = task_instance_ids[1] + + dag_root_span_name = f"{dag_id}{CTX_PROP_SUFFIX}" - for task_id in task_instance_ids: - dag_children_span_names.append(f"{task_id}{CTX_PROP_SUFFIX}") + dag_root_span_children_names = [ + f"{task1_id}{CTX_PROP_SUFFIX}", + f"{task2_id}{CTX_PROP_SUFFIX}", + ] - first_task_id = task_instance_ids[0] + task1_span_children_names = [ + f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + f"{task1_id}_sub_span4{CTX_PROP_SUFFIX}", + ] + + # Single element lists. + task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}"] + task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3{CTX_PROP_SUFFIX}"] + + assert_span_name_belongs_to_root_span( + root_span_dict=root_span_dict, span_name=dag_root_span_name, should_succeed=True + ) + # Check direct children of the root span. assert_parent_children_spans( parent_child_dict=parent_child_dict, root_span_dict=root_span_dict, - parent_name=dag_span_name, - children_names=dag_children_span_names, + parent_name=dag_root_span_name, + children_names=dag_root_span_children_names, ) + # Use a span name that exists, but it's not a direct child. assert_span_not_in_children_spans( parent_child_dict=parent_child_dict, root_span_dict=root_span_dict, span_dict=span_dict, - parent_name=dag_span_name, - child_name=first_task_id, + parent_name=dag_root_span_name, + child_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", span_exists=True, ) + # Use a span name that doesn't exist at all. assert_span_not_in_children_spans( parent_child_dict=parent_child_dict, root_span_dict=root_span_dict, span_dict=span_dict, - parent_name=dag_span_name, - child_name=f"{first_task_id}_fail", + parent_name=dag_root_span_name, + child_name=f"{task1_id}_non_existent", span_exists=False, ) - # Any spans generated under a task, are children of the task span. - # The span hierarchy for dag 'test_dag' is - # dag span - # |_ task_1 span - # |_ sub_span_1 - # |_ sub_span_2 - # |_ sub_span_3 - # |_ sub_span_4 - # |_ task_2 span - - first_task_children_span_names = [ - f"{first_task_id}_sub_span1{CTX_PROP_SUFFIX}", - f"{first_task_id}_sub_span4{CTX_PROP_SUFFIX}", - ] + # Check children of the task1 span. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{first_task_id}{CTX_PROP_SUFFIX}", - children_names=first_task_children_span_names, + parent_name=f"{task1_id}{CTX_PROP_SUFFIX}", + children_names=task1_span_children_names, ) - # Single element list. - sub_span1_children_span_names = [f"{first_task_id}_sub_span2{CTX_PROP_SUFFIX}"] + # Check children of task1 sub span1. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{first_task_id}_sub_span1{CTX_PROP_SUFFIX}", - children_names=sub_span1_children_span_names, + parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + children_names=task1_sub_span1_children_span_names, ) - sub_span2_children_span_names = [f"{first_task_id}_sub_span3{CTX_PROP_SUFFIX}"] + # Check children of task1 sub span2. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{first_task_id}_sub_span2{CTX_PROP_SUFFIX}", - children_names=sub_span2_children_span_names, + parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", + children_names=task1_sub_span2_children_span_names, ) From 9af9c6b9ee2d0b12f9b6e56fdc1916179bddb253 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 26 Nov 2024 16:55:30 +0200 Subject: [PATCH 08/52] fix span timings --- airflow/executors/base_executor.py | 1 + airflow/models/dagrun.py | 4 ++-- airflow/utils/dates.py | 10 +++++++++- 3 files changed, 12 insertions(+), 3 deletions(-) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index 7b4e65d5bb6e7..e696f3a7ea27c 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -352,6 +352,7 @@ def trigger_tasks(self, open_slots: int) -> None: span_name=f"{ti.task_id}{CTX_PROP_SUFFIX}", parent_context=parent_context, component=f"task{CTX_PROP_SUFFIX}", + start_time=ti.queued_dttm, start_as_current=False, ) self.active_spans.set(key, span) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 47a607d1e24c7..00e28df0f7837 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -1088,7 +1088,7 @@ def recalculate(self) -> _UnfinishedStates: span = Trace.start_root_span( span_name=f"{self.dag_id}{CTX_PROP_SUFFIX}", component=f"dag{CTX_PROP_SUFFIX}", - start_time=self.queued_at, + start_time=self.queued_at, # This is later converted to nano. start_as_current=False, ) elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: @@ -1189,7 +1189,7 @@ def recalculate(self) -> _UnfinishedStates: ) self.set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) - active_span.end() + active_span.end(end_time=datetime_to_nano(self.end_date)) # Remove the span from the dict. self.active_dagrun_spans.delete(self.run_id) self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) diff --git a/airflow/utils/dates.py b/airflow/utils/dates.py index 70d9ee8345025..422c662f93640 100644 --- a/airflow/utils/dates.py +++ b/airflow/utils/dates.py @@ -17,6 +17,8 @@ # under the License. from __future__ import annotations +import calendar + cron_presets: dict[str, str] = { "@hourly": "0 * * * *", "@daily": "0 0 * * *", @@ -30,5 +32,11 @@ def datetime_to_nano(datetime) -> int | None: """Convert datetime to nanoseconds.""" if datetime: - return int(datetime.timestamp() * 1000000000) + if datetime.tzinfo is None: + # There is no timezone info, handle it the same as UTC. + timestamp = calendar.timegm(datetime.timetuple()) + datetime.microsecond / 1e6 + else: + # The datetime is timezone-aware. Use timestamp directly. + timestamp = datetime.timestamp() + return int(timestamp * 1e9) return None From 526d50e2f07f053a1d4cdd3d10d1da91959a4659 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 26 Nov 2024 18:22:58 +0200 Subject: [PATCH 09/52] refactor test_otel.py --- tests/integration/otel/test_otel.py | 1046 ++++++++++++++------------- 1 file changed, 534 insertions(+), 512 deletions(-) diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index 16b5db2df0a79..48bb62c7af5b3 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -48,6 +48,280 @@ log = logging.getLogger("test_otel.TestOtelIntegration") +def unpause_trigger_dag_and_get_run_id(dag_id: str) -> str: + unpause_command = ["airflow", "dags", "unpause", dag_id] + + # Unpause the dag using the cli. + subprocess.run(unpause_command, check=True, env=os.environ.copy()) + + execution_date = pendulum.now("UTC") + run_id = f"manual__{execution_date.isoformat()}" + + trigger_command = [ + "airflow", + "dags", + "trigger", + dag_id, + "--run-id", + run_id, + "--exec-date", + execution_date.isoformat(), + ] + + # Trigger the dag using the cli. + subprocess.run(trigger_command, check=True, env=os.environ.copy()) + + return run_id + + +def wait_for_dag_run_and_check_span_status(dag_id: str, run_id: str, max_wait_time: int, span_status: str): + # max_wait_time, is the timeout for the DAG run to complete. The value is in seconds. + start_time = time.time() + + while time.time() - start_time < max_wait_time: + with create_session() as session: + dag_run = ( + session.query(DagRun) + .filter( + DagRun.dag_id == dag_id, + DagRun.run_id == run_id, + ) + .first() + ) + + if dag_run is None: + time.sleep(5) + continue + + dag_run_state = dag_run.state + log.info("DAG Run state: %s.", dag_run_state) + + dag_run_span_status = dag_run.span_status + log.info("DAG Run span status: %s.", dag_run_span_status) + + if dag_run_state in [State.SUCCESS, State.FAILED]: + break + + assert ( + dag_run_state == State.SUCCESS + ), f"Dag run did not complete successfully. Final state: {dag_run_state}." + + assert ( + dag_run_span_status == span_status + ), f"Dag run span status isn't {span_status} as expected.Actual status: {dag_run_span_status}." + + +def check_spans_with_continuance(output: str, dag: DAG): + # Get a list of lines from the captured output. + output_lines = output.splitlines() + + # Filter the output, create a json obj for each span and then store them into dictionaries. + # One dictionary with only the root spans, and one with all the captured spans (root and otherwise). + root_span_dict, span_dict = extract_spans_from_output(output_lines) + # Generate a dictionary with parent child relationships. + # This is done by comparing the span_id of each root span with the parent_id of each non-root span. + parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) + + # The span hierarchy for dag 'otel_test_dag_with_pause' is + # dag span + # |_ task_1 span + # |_ scheduler_exited span + # |_ new_scheduler span + # |_ dag span (continued) + # |_ task_1 span (continued) + # |_ sub_span_1 + # |_ sub_span_2 + # |_ sub_span_3 + # |_ sub_span_4 + # |_ task_2 span + + dag_id = dag.dag_id + + task_instance_ids = dag.task_ids + task1_id = task_instance_ids[0] + task2_id = task_instance_ids[1] + + dag_root_span_name = f"{dag_id}{CTX_PROP_SUFFIX}" + + dag_root_span_children_names = [ + f"{task1_id}{CTX_PROP_SUFFIX}", + "current_scheduler_exited", + "new_scheduler", + f"{dag_id}_continued{CTX_PROP_SUFFIX}", + ] + + dag_continued_span_children_names = [ + f"{task1_id}_continued{CTX_PROP_SUFFIX}", + f"{task2_id}{CTX_PROP_SUFFIX}", + ] + + task1_span_children_names = [ + f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + f"{task1_id}_sub_span4{CTX_PROP_SUFFIX}", + ] + + # Single element lists. + task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}"] + task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3{CTX_PROP_SUFFIX}"] + + assert_span_name_belongs_to_root_span( + root_span_dict=root_span_dict, span_name=dag_root_span_name, should_succeed=True + ) + + # Check direct children of the root span. + assert_parent_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + parent_name=dag_root_span_name, + children_names=dag_root_span_children_names, + ) + + # Use a span name that exists, but it's not a direct child. + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_root_span_name, + child_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", + span_exists=True, + ) + + # Use a span name that doesn't exist at all. + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_root_span_name, + child_name=f"{task1_id}_non_existent", + span_exists=False, + ) + + # Check children of the continued dag span. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{dag_id}_continued{CTX_PROP_SUFFIX}", + children_names=dag_continued_span_children_names, + ) + + # Check children of the continued task1 span. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", + children_names=task1_span_children_names, + ) + + # Check children of task1 sub span1. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + children_names=task1_sub_span1_children_span_names, + ) + + # Check children of task1 sub span2. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", + children_names=task1_sub_span2_children_span_names, + ) + + +def check_spans_without_continuance(output: str, dag: DAG): + # Get a list of lines from the captured output. + output_lines = output.splitlines() + + # Filter the output, create a json obj for each span and then store them into dictionaries. + # One dictionary with only the root spans, and one with all the captured spans (root and otherwise). + root_span_dict, span_dict = extract_spans_from_output(output_lines) + # Generate a dictionary with parent child relationships. + # This is done by comparing the span_id of each root span with the parent_id of each non-root span. + parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) + + # Any spans generated under a task, are children of the task span. + # The span hierarchy for dag 'otel_test_dag' is + # dag span + # |_ task_1 span + # |_ sub_span_1 + # |_ sub_span_2 + # |_ sub_span_3 + # |_ sub_span_4 + # |_ task_2 span + + dag_id = dag.dag_id + + task_instance_ids = dag.task_ids + task1_id = task_instance_ids[0] + task2_id = task_instance_ids[1] + + dag_root_span_name = f"{dag_id}{CTX_PROP_SUFFIX}" + + dag_root_span_children_names = [ + f"{task1_id}{CTX_PROP_SUFFIX}", + f"{task2_id}{CTX_PROP_SUFFIX}", + ] + + task1_span_children_names = [ + f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + f"{task1_id}_sub_span4{CTX_PROP_SUFFIX}", + ] + + # Single element lists. + task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}"] + task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3{CTX_PROP_SUFFIX}"] + + assert_span_name_belongs_to_root_span( + root_span_dict=root_span_dict, span_name=dag_root_span_name, should_succeed=True + ) + + # Check direct children of the root span. + assert_parent_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + parent_name=dag_root_span_name, + children_names=dag_root_span_children_names, + ) + + # Use a span name that exists, but it's not a direct child. + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_root_span_name, + child_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + span_exists=True, + ) + + # Use a span name that doesn't exist at all. + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_root_span_name, + child_name=f"{task1_id}_non_existent", + span_exists=False, + ) + + # Check children of the task1 span. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}{CTX_PROP_SUFFIX}", + children_names=task1_span_children_names, + ) + + # Check children of task1 sub span1. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + children_names=task1_sub_span1_children_span_names, + ) + + # Check children of task1 sub span2. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", + children_names=task1_sub_span2_children_span_names, + ) + + @pytest.mark.integration("redis") @pytest.mark.backend("postgres") class TestOtelIntegration: @@ -71,6 +345,22 @@ class TestOtelIntegration: use_otel = os.getenv("use_otel", default="false") log_level = os.getenv("log_level", default="none") + celery_command_args = [ + "celery", + "--app", + "airflow.providers.celery.executors.celery_executor.app", + "worker", + "--concurrency", + "1", + "--loglevel", + "INFO", + ] + + scheduler_command_args = [ + "airflow", + "scheduler", + ] + @classmethod def setup_class(cls): os.environ["AIRFLOW__TRACES__OTEL_ON"] = "True" @@ -89,6 +379,9 @@ def setup_class(cls): os.environ["AIRFLOW__CORE__LOAD_EXAMPLES"] = "False" os.environ["AIRFLOW__CORE__UNIT_TEST_MODE"] = "False" + if cls.log_level == "debug": + log.setLevel(logging.DEBUG) + @pytest.fixture def celery_worker_env_vars(self, monkeypatch): os.environ["AIRFLOW__CORE__EXECUTOR"] = "CeleryExecutor" @@ -98,96 +391,115 @@ def celery_worker_env_vars(self, monkeypatch): ) monkeypatch.setattr(executor_loader, "_alias_to_executors", {"CeleryExecutor": executor_name}) - def test_dag_spans_with_context_propagation(self, monkeypatch, celery_worker_env_vars, capfd, session): + def test_same_scheduler_processing_the_entire_dag( + self, monkeypatch, celery_worker_env_vars, capfd, session + ): """ Test that a DAG runs successfully and exports the correct spans. Integration with a scheduler, a celery worker, a postgres db and a redis broker. """ - if self.log_level == "debug": - log.setLevel(logging.DEBUG) - - celery_command_args = [ - "celery", - "--app", - "airflow.providers.celery.executors.celery_executor.app", - "worker", - "--concurrency", - "1", - "--loglevel", - "INFO", - ] - - scheduler_command_args = [ - "airflow", - "scheduler", - ] - celery_worker_process = None scheduler_process = None try: - # Start the processes here and not as fixtures, so that the test can capture their output. - celery_worker_process = subprocess.Popen( - celery_command_args, - env=os.environ.copy(), - stdout=None, - stderr=None, - ) + # Start the processes here and not as fixtures or in a common setup, + # so that the test can capture their output. + celery_worker_process, scheduler_process = self.start_worker_and_scheduler1() - scheduler_process = subprocess.Popen( - scheduler_command_args, - env=os.environ.copy(), - stdout=None, - stderr=None, + dag_id = "otel_test_dag" + dag = self.serialize_and_get_dag(dag_id=dag_id) + + run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) + + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=90, span_status=SpanStatus.ENDED ) - # Wait to ensure both processes have started. - time.sleep(10) + if logging.root.level == logging.DEBUG: + with create_session() as session: + dump_airflow_metadata_db(session) + finally: + # Terminate the processes. + celery_worker_process.terminate() + celery_worker_process.wait() + + scheduler_process.terminate() + scheduler_process.wait() + + out, err = capfd.readouterr() + log.debug("out-start --\n%s\n-- out-end", out) + log.debug("err-start --\n%s\n-- err-end", err) + + if self.use_otel != "true": + # Dag run should have succeeded. Test the spans from the output. + check_spans_without_continuance(output=out, dag=dag) + + def test_another_scheduler_executing_the_second_task_and_the_rest_of_the_dag( + self, monkeypatch, celery_worker_env_vars, capfd, session + ): + """ + Similar to test_dag_spans_with_context_propagation but this test will start with one scheduler, + and during the dag execution, it will stop the process and start a new one. - execution_date = pendulum.now("UTC") + A txt file will be used for signaling the test and the dag in order to make sure that + the 1st scheduler is stopped while the first task is executing and that + the 2nd scheduler picks up the task and dag processing. + The steps will be + - The dag starts running, creates the file with a signal word and waits until the word is changed. + - The test checks if the file exist, stops the scheduler, starts a new scheduler and updates the file. + - The dag gets the update and continues until the task is finished. + At this point, the second scheduler should handle the rest of the dag processing. + """ - # Load DAGs from the dag directory. - dag_bag = DagBag(dag_folder=self.dag_folder, include_examples=False) + celery_worker_process = None + scheduler_process_1 = None + scheduler_process_2 = None + try: + # Start the processes here and not as fixtures or in a common setup, + # so that the test can capture their output. + celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() dag_id = "otel_test_dag" - dag = dag_bag.get_dag(dag_id) + dag = self.serialize_and_get_dag(dag_id=dag_id) - assert dag is not None, f"DAG with ID {dag_id} not found." + run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) with create_session() as session: - # Sync the DAG to the database. - dag.sync_to_db(session=session) - # Manually serialize the dag and write it to the db to avoid a db error. - SerializedDagModel.write_dag(dag, session=session) - session.commit() - - unpause_command = ["airflow", "dags", "unpause", dag_id] + tis: list[TaskInstance] = dag.get_task_instances(session=session) - # Unpause the dag using the cli. - subprocess.run(unpause_command, check=True, env=os.environ.copy()) + task_1 = tis[0] - run_id = f"manual__{execution_date.isoformat()}" + while True: + with create_session() as session: + ti = ( + session.query(TaskInstance) + .filter( + TaskInstance.task_id == task_1.task_id, + TaskInstance.run_id == task_1.run_id, + ) + .first() + ) - trigger_command = [ - "airflow", - "dags", - "trigger", - dag_id, - "--run-id", - run_id, - "--exec-date", - execution_date.isoformat(), - ] + if ti is None: + continue - # Trigger the dag using the cli. - subprocess.run(trigger_command, check=True, env=os.environ.copy()) + # Wait until the task has been finished. + if ti.state in State.finished: + break - # Wait timeout for the DAG run to complete. - max_wait_time = 90 # seconds - start_time = time.time() + with capfd.disabled(): + # When we pause the scheduler1 thread, capfd keeps trying to read the + # file descriptors for the process and ends up freezing the test. + # There won't be any exported spans from the following code, + # so not capturing the output, doesn't make a difference. + scheduler_process_1.send_signal(signal.SIGSTOP) - dag_run_state = None + scheduler_process_2 = subprocess.Popen( + self.scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) - while time.time() - start_time < max_wait_time: with create_session() as session: dag_run = ( session.query(DagRun) @@ -198,42 +510,54 @@ def test_dag_spans_with_context_propagation(self, monkeypatch, celery_worker_env .first() ) - if dag_run is None: - time.sleep(5) - continue + assert ( + dag_run.state == State.RUNNING + ), f"Dag Run state isn't RUNNING. State: {dag_run.state}" + assert ( + dag_run.span_status == SpanStatus.ACTIVE + ), f"Dag Run span_status isn't ACTIVE. Span_status: {dag_run.span_status}" - dag_run_state = dag_run.state - log.info("DAG Run state: %s.", dag_run_state) + # Wait for scheduler2 to be up and running. + time.sleep(10) - if dag_run_state in [State.SUCCESS, State.FAILED]: - break + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.SHOULD_END + ) - time.sleep(5) + if logging.root.level == logging.DEBUG: + with create_session() as session: + dump_airflow_metadata_db(session) - if logging.root.level == logging.DEBUG: - with create_session() as session: - dump_airflow_metadata_db(session) + scheduler_process_1.send_signal(signal.SIGCONT) + scheduler_process_2.terminate() + + # Wait for the scheduler to start again and continue running. + time.sleep(10) + + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=30, span_status=SpanStatus.ENDED + ) - assert ( - dag_run_state == State.SUCCESS - ), f"Dag run did not complete successfully. Final state: {dag_run_state}." finally: # Terminate the processes. celery_worker_process.terminate() celery_worker_process.wait() - scheduler_process.terminate() - scheduler_process.wait() + scheduler_process_1.terminate() + scheduler_process_1.wait() + + # scheduler_process_2.terminate() + scheduler_process_2.wait() out, err = capfd.readouterr() log.debug("out-start --\n%s\n-- out-end", out) log.debug("err-start --\n%s\n-- err-end", err) if self.use_otel != "true": - # Dag run should have succeeded. Test the spans from the output. - self.check_spans_without_continuance(output=out, dag=dag) + # Dag run should have succeeded. Test the spans in the output. + check_spans_without_continuance(output=out, dag=dag) - def test_dag_spans_with_different_schedulers_processing_the_same_dag( + def test_scheduler_change_in_the_middle_of_first_task_execution_until_the_end( self, monkeypatch, celery_worker_env_vars, capfd, session ): """ @@ -249,84 +573,19 @@ def test_dag_spans_with_different_schedulers_processing_the_same_dag( - The dag gets the update and continues until the task is finished. At this point, the second scheduler should handle the rest of the dag processing. """ - if self.log_level == "debug": - log.setLevel(logging.DEBUG) - - celery_command_args = [ - "celery", - "--app", - "airflow.providers.celery.executors.celery_executor.app", - "worker", - "--concurrency", - "1", - "--loglevel", - "INFO", - ] - - scheduler_command_args = [ - "airflow", - "scheduler", - ] celery_worker_process = None scheduler_process_1 = None scheduler_process_2 = None try: - # Start the processes here and not as fixtures, so that the test can capture their output. - celery_worker_process = subprocess.Popen( - celery_command_args, - env=os.environ.copy(), - stdout=None, - stderr=None, - ) - - scheduler_process_1 = subprocess.Popen( - scheduler_command_args, - env=os.environ.copy(), - stdout=None, - stderr=None, - ) - - # Wait to ensure both processes have started. - time.sleep(10) - - execution_date = pendulum.now("UTC") - - # Load DAGs from the dag directory. - dag_bag = DagBag(dag_folder=self.dag_folder, include_examples=False) + # Start the processes here and not as fixtures or in a common setup, + # so that the test can capture their output. + celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() dag_id = "otel_test_dag" - dag = dag_bag.get_dag(dag_id) - - assert dag is not None, f"DAG with ID {dag_id} not found." - - with create_session() as session: - # Sync the DAG to the database. - dag.sync_to_db(session=session) - # Manually serialize the dag and write it to the db to avoid a db error. - SerializedDagModel.write_dag(dag, session=session) - session.commit() - - unpause_command = ["airflow", "dags", "unpause", dag_id] + dag = self.serialize_and_get_dag(dag_id=dag_id) - # Unpause the dag using the cli. - subprocess.run(unpause_command, check=True, env=os.environ.copy()) - - run_id = f"manual__{execution_date.isoformat()}" - - trigger_command = [ - "airflow", - "dags", - "trigger", - dag_id, - "--run-id", - run_id, - "--exec-date", - execution_date.isoformat(), - ] - - # Trigger the dag using the cli. - subprocess.run(trigger_command, check=True, env=os.environ.copy()) + run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) with create_session() as session: tis: list[TaskInstance] = dag.get_task_instances(session=session) @@ -359,7 +618,7 @@ def test_dag_spans_with_different_schedulers_processing_the_same_dag( scheduler_process_1.send_signal(signal.SIGSTOP) scheduler_process_2 = subprocess.Popen( - scheduler_command_args, + self.scheduler_command_args, env=os.environ.copy(), stdout=None, stderr=None, @@ -385,87 +644,23 @@ def test_dag_spans_with_different_schedulers_processing_the_same_dag( # Wait for scheduler2 to be up and running. time.sleep(10) - # Wait timeout for the DAG run to complete. - max_wait_time = 120 # seconds - start_time = time.time() - - dag_run_state = None - dag_run_span_status = None - - while time.time() - start_time < max_wait_time: - with create_session() as session: - dag_run = ( - session.query(DagRun) - .filter( - DagRun.dag_id == dag_id, - DagRun.run_id == run_id, - ) - .first() - ) - - if dag_run is None: - time.sleep(5) - continue - - dag_run_state = dag_run.state - # log.info("DAG Run state: %s.", dag_run_state) - - dag_run_span_status = dag_run.span_status - # log.info("DAG Run span status: %s.", dag_run_span_status) - - if dag_run_state in [State.SUCCESS, State.FAILED]: - break - - time.sleep(5) + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.SHOULD_END + ) if logging.root.level == logging.DEBUG: with create_session() as session: dump_airflow_metadata_db(session) - assert ( - dag_run_state == State.SUCCESS - ), f"Dag run did not complete successfully. Final state: {dag_run_state}." - assert dag_run_span_status == SpanStatus.SHOULD_END, ( - f"Dag run has been finished by scheduler2 but span status isn't {SpanStatus.SHOULD_END}." - f"Actual status: {dag_run_span_status}." - ) - scheduler_process_1.send_signal(signal.SIGCONT) scheduler_process_2.terminate() # Wait for the scheduler to start again and continue running. time.sleep(10) - max_wait_time = 30 # seconds - start_time = time.time() - - dag_run_span_status = None - - while time.time() - start_time < max_wait_time: - with create_session() as session: - dag_run = ( - session.query(DagRun) - .filter( - DagRun.dag_id == dag_id, - DagRun.run_id == run_id, - ) - .first() - ) - - if dag_run is None: - time.sleep(5) - continue - - dag_run_span_status = dag_run.span_status - log.info("DAG Run span status: %s.", dag_run_span_status) - - if dag_run_span_status == SpanStatus.ENDED: - break - assert dag_run_span_status == SpanStatus.ENDED, ( - f"Scheduler1 should have ended the span and updated status to {SpanStatus.ENDED} but didn't." - f"Actual status: {dag_run_span_status}." + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=30, span_status=SpanStatus.ENDED ) - finally: # Terminate the processes. celery_worker_process.terminate() @@ -483,9 +678,9 @@ def test_dag_spans_with_different_schedulers_processing_the_same_dag( if self.use_otel != "true": # Dag run should have succeeded. Test the spans in the output. - self.check_spans_without_continuance(output=out, dag=dag) + check_spans_without_continuance(output=out, dag=dag) - def test_dag_spans_with_scheduler_ha_and_stopped_scheduler( + def test_scheduler_exits_gracefully_in_the_middle_of_the_first_task( self, monkeypatch, celery_worker_env_vars, capfd, session ): """ @@ -501,84 +696,106 @@ def test_dag_spans_with_scheduler_ha_and_stopped_scheduler( - The dag gets the update and continues until the task is finished. At this point, the second scheduler should handle the rest of the dag processing. """ - if self.log_level == "debug": - log.setLevel(logging.DEBUG) - - celery_command_args = [ - "celery", - "--app", - "airflow.providers.celery.executors.celery_executor.app", - "worker", - "--concurrency", - "1", - "--loglevel", - "INFO", - ] - - scheduler_command_args = [ - "airflow", - "scheduler", - ] celery_worker_process = None scheduler_process_1 = None scheduler_process_2 = None try: - # Start the processes here and not as fixtures, so that the test can capture their output. - celery_worker_process = subprocess.Popen( - celery_command_args, - env=os.environ.copy(), - stdout=None, - stderr=None, - ) + # Start the processes here and not as fixtures or in a common setup, + # so that the test can capture their output. + celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() + + dag_id = "otel_test_dag_with_pause" + dag = self.serialize_and_get_dag(dag_id=dag_id) + + run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) + + # Control file path. + control_file = os.path.join(self.dag_folder, "dag_control.txt") + + while True: + try: + with open(control_file) as file: + # If it reaches inside the block, then the file exists and the test can read it. + break + except FileNotFoundError: + print("Control file not found. Waiting...") + time.sleep(1) + continue - scheduler_process_1 = subprocess.Popen( - scheduler_command_args, + # Since, we are past the loop, then the file exists and the dag has been paused. + # Terminate scheduler1 and start scheduler2. + scheduler_process_1.terminate() + + scheduler_process_2 = subprocess.Popen( + self.scheduler_command_args, env=os.environ.copy(), stdout=None, stderr=None, ) - # Wait to ensure both processes have started. + # Wait for scheduler2 to be up and running. time.sleep(10) - execution_date = pendulum.now("UTC") + # Rewrite the file to unpause the dag. + with open(control_file, "w") as file: + file.write("continue") - # Load DAGs from the dag directory. - dag_bag = DagBag(dag_folder=self.dag_folder, include_examples=False) + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.ENDED + ) - dag_id = "otel_test_dag_with_pause" - dag = dag_bag.get_dag(dag_id) + if logging.root.level == logging.DEBUG: + with create_session() as session: + dump_airflow_metadata_db(session) - assert dag is not None, f"DAG with ID {dag_id} not found." + finally: + # Terminate the processes. + celery_worker_process.terminate() + celery_worker_process.wait() - with create_session() as session: - # Sync the DAG to the database. - dag.sync_to_db(session=session) - # Manually serialize the dag and write it to the db to avoid a db error. - SerializedDagModel.write_dag(dag, session=session) - session.commit() + scheduler_process_1.wait() + + scheduler_process_2.terminate() + scheduler_process_2.wait() + + out, err = capfd.readouterr() + log.debug("out-start --\n%s\n-- out-end", out) + log.debug("err-start --\n%s\n-- err-end", err) - unpause_command = ["airflow", "dags", "unpause", dag_id] + if self.use_otel != "true": + # Dag run should have succeeded. Test the spans in the output. + check_spans_with_continuance(output=out, dag=dag) - # Unpause the dag using the cli. - subprocess.run(unpause_command, check=True, env=os.environ.copy()) + def test_scheduler_exits_forcefully_in_the_middle_of_the_first_task( + self, monkeypatch, celery_worker_env_vars, capfd, session + ): + """ + This test will start with one scheduler, and during the dag execution, + it will stop the process and start a new one. + + A txt file will be used for signaling the test and the dag in order to make sure that + the 1st scheduler is stopped while the first task is executing and that + the 2nd scheduler picks up the task and dag processing. + The steps will be + - The dag starts running, creates the file with a signal word and waits until the word is changed. + - The test checks if the file exist, stops the scheduler, starts a new scheduler and updates the file. + - The dag gets the update and continues until the task is finished. + At this point, the second scheduler should handle the rest of the dag processing. + """ - run_id = f"manual__{execution_date.isoformat()}" + celery_worker_process = None + scheduler_process_1 = None + scheduler_process_2 = None + try: + # Start the processes here and not as fixtures or in a common setup, + # so that the test can capture their output. + celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() - trigger_command = [ - "airflow", - "dags", - "trigger", - dag_id, - "--run-id", - run_id, - "--exec-date", - execution_date.isoformat(), - ] + dag_id = "otel_test_dag_with_pause" + dag = self.serialize_and_get_dag(dag_id=dag_id) - # Trigger the dag using the cli. - subprocess.run(trigger_command, check=True, env=os.environ.copy()) + run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) # Control file path. control_file = os.path.join(self.dag_folder, "dag_control.txt") @@ -598,7 +815,7 @@ def test_dag_spans_with_scheduler_ha_and_stopped_scheduler( scheduler_process_1.terminate() scheduler_process_2 = subprocess.Popen( - scheduler_command_args, + self.scheduler_command_args, env=os.environ.copy(), stdout=None, stderr=None, @@ -611,42 +828,14 @@ def test_dag_spans_with_scheduler_ha_and_stopped_scheduler( with open(control_file, "w") as file: file.write("continue") - # Wait timeout for the DAG run to complete. - max_wait_time = 120 # seconds - start_time = time.time() - - dag_run_state = None - - while time.time() - start_time < max_wait_time: - with create_session() as session: - dag_run = ( - session.query(DagRun) - .filter( - DagRun.dag_id == dag_id, - DagRun.run_id == run_id, - ) - .first() - ) - - if dag_run is None: - time.sleep(5) - continue - - dag_run_state = dag_run.state - log.info("DAG Run state: %s.", dag_run_state) - - if dag_run_state in [State.SUCCESS, State.FAILED]: - break - - time.sleep(5) + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.ENDED + ) if logging.root.level == logging.DEBUG: with create_session() as session: dump_airflow_metadata_db(session) - assert ( - dag_run_state == State.SUCCESS - ), f"Dag run did not complete successfully. Final state: {dag_run_state}." finally: # Terminate the processes. celery_worker_process.terminate() @@ -663,208 +852,41 @@ def test_dag_spans_with_scheduler_ha_and_stopped_scheduler( if self.use_otel != "true": # Dag run should have succeeded. Test the spans in the output. - # Get a list of lines from the captured output. - output_lines = out.splitlines() - - # Filter the output, create a json obj for each span and then store them into dictionaries. - # One dictionary with only the root spans, and one with all the captured spans (root and otherwise). - root_span_dict, span_dict = extract_spans_from_output(output_lines) - # Generate a dictionary with parent child relationships. - # This is done by comparing the span_id of each root span with the parent_id of each non-root span. - parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) - - # The span hierarchy for dag 'otel_test_dag_with_pause' is - # dag span - # |_ task_1 span - # |_ scheduler_exited span - # |_ new_scheduler span - # |_ dag span (continued) - # |_ task_1 span (continued) - # |_ sub_span_1 - # |_ sub_span_2 - # |_ sub_span_3 - # |_ sub_span_4 - # |_ task_2 span - - task_instance_ids = dag.task_ids - task1_id = task_instance_ids[0] - task2_id = task_instance_ids[1] - - dag_root_span_name = f"{dag_id}{CTX_PROP_SUFFIX}" - - dag_root_span_children_names = [ - f"{task1_id}{CTX_PROP_SUFFIX}", - "current_scheduler_exited", - "new_scheduler", - f"{dag_id}_continued{CTX_PROP_SUFFIX}", - ] - - dag_continued_span_children_names = [ - f"{task1_id}_continued{CTX_PROP_SUFFIX}", - f"{task2_id}{CTX_PROP_SUFFIX}", - ] - - task1_span_children_names = [ - f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", - f"{task1_id}_sub_span4{CTX_PROP_SUFFIX}", - ] - - # Single element lists. - task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}"] - task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3{CTX_PROP_SUFFIX}"] - - assert_span_name_belongs_to_root_span( - root_span_dict=root_span_dict, span_name=dag_root_span_name, should_succeed=True - ) - - # Check direct children of the root span. - assert_parent_children_spans( - parent_child_dict=parent_child_dict, - root_span_dict=root_span_dict, - parent_name=dag_root_span_name, - children_names=dag_root_span_children_names, - ) - - # Use a span name that exists, but it's not a direct child. - assert_span_not_in_children_spans( - parent_child_dict=parent_child_dict, - root_span_dict=root_span_dict, - span_dict=span_dict, - parent_name=dag_root_span_name, - child_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", - span_exists=True, - ) - - # Use a span name that doesn't exist at all. - assert_span_not_in_children_spans( - parent_child_dict=parent_child_dict, - root_span_dict=root_span_dict, - span_dict=span_dict, - parent_name=dag_root_span_name, - child_name=f"{task1_id}_non_existent", - span_exists=False, - ) - - # Check children of the continued dag span. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{dag_id}_continued{CTX_PROP_SUFFIX}", - children_names=dag_continued_span_children_names, - ) - - # Check children of the continued task1 span. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", - children_names=task1_span_children_names, - ) - - # Check children of task1 sub span1. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", - children_names=task1_sub_span1_children_span_names, - ) - - # Check children of task1 sub span2. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", - children_names=task1_sub_span2_children_span_names, - ) - - def check_spans_without_continuance(self, output: str, dag: DAG): - # Get a list of lines from the captured output. - output_lines = output.splitlines() - - # Filter the output, create a json obj for each span and then store them into dictionaries. - # One dictionary with only the root spans, and one with all the captured spans (root and otherwise). - root_span_dict, span_dict = extract_spans_from_output(output_lines) - # Generate a dictionary with parent child relationships. - # This is done by comparing the span_id of each root span with the parent_id of each non-root span. - parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) - - # Any spans generated under a task, are children of the task span. - # The span hierarchy for dag 'otel_test_dag' is - # dag span - # |_ task_1 span - # |_ sub_span_1 - # |_ sub_span_2 - # |_ sub_span_3 - # |_ sub_span_4 - # |_ task_2 span - - dag_id = dag.dag_id - - task_instance_ids = dag.task_ids - task1_id = task_instance_ids[0] - task2_id = task_instance_ids[1] - - dag_root_span_name = f"{dag_id}{CTX_PROP_SUFFIX}" - - dag_root_span_children_names = [ - f"{task1_id}{CTX_PROP_SUFFIX}", - f"{task2_id}{CTX_PROP_SUFFIX}", - ] - - task1_span_children_names = [ - f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", - f"{task1_id}_sub_span4{CTX_PROP_SUFFIX}", - ] - - # Single element lists. - task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}"] - task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3{CTX_PROP_SUFFIX}"] - - assert_span_name_belongs_to_root_span( - root_span_dict=root_span_dict, span_name=dag_root_span_name, should_succeed=True + check_spans_with_continuance(output=out, dag=dag) + + def start_worker_and_scheduler1(self): + celery_worker_process = subprocess.Popen( + self.celery_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, ) - # Check direct children of the root span. - assert_parent_children_spans( - parent_child_dict=parent_child_dict, - root_span_dict=root_span_dict, - parent_name=dag_root_span_name, - children_names=dag_root_span_children_names, + scheduler_process = subprocess.Popen( + self.scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, ) - # Use a span name that exists, but it's not a direct child. - assert_span_not_in_children_spans( - parent_child_dict=parent_child_dict, - root_span_dict=root_span_dict, - span_dict=span_dict, - parent_name=dag_root_span_name, - child_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", - span_exists=True, - ) + # Wait to ensure both processes have started. + time.sleep(10) - # Use a span name that doesn't exist at all. - assert_span_not_in_children_spans( - parent_child_dict=parent_child_dict, - root_span_dict=root_span_dict, - span_dict=span_dict, - parent_name=dag_root_span_name, - child_name=f"{task1_id}_non_existent", - span_exists=False, - ) + return celery_worker_process, scheduler_process - # Check children of the task1 span. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}{CTX_PROP_SUFFIX}", - children_names=task1_span_children_names, - ) + def serialize_and_get_dag(self, dag_id: str) -> DAG: + # Load DAGs from the dag directory. + dag_bag = DagBag(dag_folder=self.dag_folder, include_examples=False) - # Check children of task1 sub span1. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", - children_names=task1_sub_span1_children_span_names, - ) + dag = dag_bag.get_dag(dag_id) - # Check children of task1 sub span2. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", - children_names=task1_sub_span2_children_span_names, - ) + assert dag is not None, f"DAG with ID {dag_id} not found." + + with create_session() as session: + # Sync the DAG to the database. + dag.sync_to_db(session=session) + # Manually serialize the dag and write it to the db to avoid a db error. + SerializedDagModel.write_dag(dag, session=session) + session.commit() + + return dag From 07076ee81adb41052418290b502b5fae56abfbaf Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Mon, 2 Dec 2024 18:23:53 +0200 Subject: [PATCH 10/52] handle scheduler forceful exit --- airflow/jobs/scheduler_job_runner.py | 97 +++- airflow/models/dagrun.py | 58 ++- airflow/models/taskinstance.py | 12 +- airflow/serialization/pydantic/dag_run.py | 1 + tests/integration/otel/test_otel.py | 592 ++++++++++++++-------- tests/integration/otel/test_utils.py | 569 +++++++++++++++++++-- 6 files changed, 1071 insertions(+), 258 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index de400599d1d6e..71138c383aeb0 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -63,6 +63,7 @@ from airflow.ti_deps.dependencies_states import EXECUTION_STATES from airflow.timetables.simple import AssetTriggeredTimetable from airflow.traces import utils as trace_utils +from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import Trace, add_span from airflow.utils import timezone from airflow.utils.dates import datetime_to_nano @@ -1052,11 +1053,12 @@ def _update_dag_run_state_for_paused_dags(self, session: Session = NEW_SESSION) def _cleanup_active_spans_before_process_exit(self, session: Session = NEW_SESSION): # No need to do a commit for every update. The annotation will commit all of them once at the end. for run_id, span in self.active_dagrun_spans.get_all().items(): - span.end() dag_run: DagRun = session.scalars(select(DagRun).where(DagRun.run_id == run_id)).one() if dag_run.state in State.finished_dr_states: + span.end(end_time=datetime_to_nano(dag_run.end_date)) dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: + span.end() dag_run.set_span_status( status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False ) @@ -1067,7 +1069,6 @@ def _cleanup_active_spans_before_process_exit(self, session: Session = NEW_SESSI s.set_attribute("trace_status", "needs continuance") for key, span in self.active_ti_spans.get_all().items(): - span.end() # Can't compare the key directly because the try_number or the map_index might not be the same. ti: TaskInstance = session.scalars( select(TaskInstance).where( @@ -1077,8 +1078,10 @@ def _cleanup_active_spans_before_process_exit(self, session: Session = NEW_SESSI ) ).one() if ti.state in State.finished: + span.end(end_time=datetime_to_nano(ti.end_date)) ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: + span.end() ti.set_span_status(status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False) self.active_dagrun_spans.clear() @@ -1127,6 +1130,83 @@ def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): self.active_ti_spans.delete(ti.key) ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + @provide_session + def _recreate_dead_scheduler_spans_if_needed(self, dag_run: DagRun, session: Session = NEW_SESSION): + scheduler_health_timeout = conf.getint("scheduler", "scheduler_health_check_threshold") + + # There are two scenarios: + # 1. scheduler is unhealthy but managed to update span_status + # 2. scheduler is unhealthy and didn't manage to make any updates + # Check the span_status first, in case the 2nd db query can be avoided (scenario 1). + + # Get the latest values from the db. + dr: DagRun = session.scalars( + select(DagRun).where( + DagRun.run_id == dag_run.run_id, + DagRun.dag_id == dag_run.dag_id, + ) + ).one() + + # If the dag_run is scheduled by a different scheduler, and it's still running and the span is active, + # then check the Job table to determine if the initial scheduler is still healthy. + if ( + dr.scheduled_by_job_id != self.job.id + and dr.state in State.unfinished_dr_states + and dr.span_status == SpanStatus.ACTIVE + ): + job: Job = session.scalars( + select(Job).where( + Job.id == dr.scheduled_by_job_id, + Job.job_type == "SchedulerJob", + ) + ).one() + + # If the time passed since the last heartbeat is less than the timeout. + is_healthy = scheduler_health_timeout > (timezone.utcnow() - job.latest_heartbeat).total_seconds() + + if not is_healthy: + # Start a new span for the dag_run. + dr_span = Trace.start_root_span( + span_name=f"{dr.dag_id}_recreated{CTX_PROP_SUFFIX}", + component="dag", + start_time=dr.queued_at, + start_as_current=False, + ) + carrier = Trace.inject() + # Update the context_carrier and leave the SpanStatus as ACTIVE. + dr.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) + self.active_dagrun_spans.set(dr.run_id, dr_span) + + tis = dr.get_task_instances(session=session) + + # At this point, any tis will have been adopted by the current scheduler. + # If the span_status is ACTIVE but there isn't an entry on the active spans, + # then it was started by the unhealthy scheduler. + tis_needing_spans = [ + ti + for ti in tis + if ti.span_status == SpanStatus.ACTIVE and self.active_ti_spans.get(ti.key) is None + ] + + dr_context = Trace.extract(dr.context_carrier) + for ti in tis_needing_spans: + ti_span = Trace.start_child_span( + span_name=f"{ti.task_id}_recreated{CTX_PROP_SUFFIX}", + parent_context=dr_context, + start_time=ti.queued_dttm, + start_as_current=False, + ) + ti_carrier = Trace.inject() + ti.set_context_carrier(context_carrier=ti_carrier, session=session, with_commit=False) + + if ti.state in State.finished: + ti_span.end(end_time=datetime_to_nano(ti.end_date)) + ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + else: + # TODO: revisit this. Will the current scheduler end the span? + ti.set_span_status(status=SpanStatus.ACTIVE, session=session, with_commit=False) + self.active_ti_spans.set(ti.key, ti_span) + def _run_scheduler_loop(self) -> None: """ Harvest DAG parsing results, queue tasks, and perform executor heartbeat; the actual scheduler loop. @@ -1850,6 +1930,19 @@ def _schedule_dag_run( "The DAG disappeared before verifying integrity: %s. Skipping.", dag_run.dag_id ) return callback + + if ( + dag_run.scheduled_by_job_id is not None + and dag_run.set_scheduled_by_job_id != self.job.id + and self.active_dagrun_spans.get(dag_run.run_id) is None + ): + # If the dag_run has been previously scheduled by another job and there is no active span, + # then check if the job is still healthy. + # If it's not healthy, then recreate the spans. + self._recreate_dead_scheduler_spans_if_needed(dag_run, session) + + dag_run.set_scheduled_by_job_id(job_id=self.job.id, session=session, with_commit=False) + # TODO[HA]: Rename update_state -> schedule_dag_run, ?? something else? schedulable_tis, callback_to_run = dag_run.update_state(session=session, execute_callbacks=False) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 00e28df0f7837..f991ce5507507 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -193,6 +193,7 @@ class DagRun(Base, LoggingMixin): dag_version_id = Column(UUIDType(binary=False), ForeignKey("dag_version.id", ondelete="CASCADE")) dag_version = relationship("DagVersion", back_populates="dag_runs") + scheduled_by_job_id = Column(Integer) # Span context carrier, used for context propagation. context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) span_status = Column(String(50), default=SpanStatus.NOT_STARTED) @@ -295,6 +296,7 @@ def __init__( self.clear_number = 0 self.triggered_by = triggered_by self.dag_version = dag_version + self.scheduled_by_job_id = None self.context_carrier = {} super().__init__() @@ -949,7 +951,6 @@ def update_state( """ # Callback to execute in case of Task Failures callback: DagCallbackRequest | None = None - print(f"x: dag_state: {self._state} | span_status: {self.span_status}") class _UnfinishedStates(NamedTuple): tis: Sequence[TI] @@ -1082,10 +1083,10 @@ def recalculate(self) -> _UnfinishedStates: self.span_status == SpanStatus.NOT_STARTED or self.span_status == SpanStatus.NEEDS_CONTINUANCE ): - span = None + dr_span = None continue_ti_spans = False if self.span_status == SpanStatus.NOT_STARTED: - span = Trace.start_root_span( + dr_span = Trace.start_root_span( span_name=f"{self.dag_id}{CTX_PROP_SUFFIX}", component=f"dag{CTX_PROP_SUFFIX}", start_time=self.queued_at, # This is later converted to nano. @@ -1099,7 +1100,7 @@ def recalculate(self) -> _UnfinishedStates: ) as s: s.set_attribute("trace_status", "continued") - span = Trace.start_child_span( + dr_span = Trace.start_child_span( span_name=f"{self.dag_id}_continued{CTX_PROP_SUFFIX}", parent_context=parent_context, component=f"dag{CTX_PROP_SUFFIX}", @@ -1113,7 +1114,7 @@ def recalculate(self) -> _UnfinishedStates: self.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) self.set_span_status(status=SpanStatus.ACTIVE, session=session, with_commit=False) # Set the span in a synchronized dictionary, so that the variable can be used to end the span. - self.active_dagrun_spans.set(self.run_id, span) + self.active_dagrun_spans.set(self.run_id, dr_span) self.log.debug( "DagRun span has been started and the injected context_carrier is: %s", self.context_carrier, @@ -1200,6 +1201,13 @@ def recalculate(self) -> _UnfinishedStates: self.set_span_status( status=SpanStatus.SHOULD_END, session=session, with_commit=False ) + elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: + # This is a corner case where the scheduler exited gracefully + # while the dag_run was almost done. + # Since it reached this point, the dag has finished but there has been no time + # to create a new span for the current scheduler. + # There is no need for more spans, update the status on the db. + self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: self.log.debug( "No active span has been found for dag_id: %s, run_id: %s, state: %s", @@ -1272,6 +1280,46 @@ def _filter_tis_and_exclude_removed(dag: DAG, tis: list[TI]) -> Iterable[TI]: finished_tis=finished_tis, ) + @staticmethod + @internal_api_call + def _set_scheduled_by_job_id(dag_run: DagRun, job_id: int, session: Session, with_commit: bool) -> bool: + if not isinstance(dag_run, DagRun): + dag_run = session.scalars( + select(DagRun).where( + DagRun.dag_id == dag_run.dag_id, + DagRun.run_id == dag_run.run_id, + ) + ).one() + + if dag_run.scheduled_by_job_id == job_id: + return False + + dag_run.log.debug("Setting dag_run scheduled_by_job_id for run_id: %s", dag_run.run_id) + dag_run.scheduled_by_job_id = job_id + + session.merge(dag_run) + + if with_commit: + session.commit() + + return True + + @provide_session + def set_scheduled_by_job_id( + self, job_id: int, session: Session = NEW_SESSION, with_commit: bool = False + ) -> bool: + """ + Set DagRun scheduled_by_job_id. + + :param job_id: integer with the scheduled_by_job_id to set for the dag_run + :param session: SQLAlchemy ORM Session + :param with_commit: should the scheduled_by_job_id be committed? + :return: has the scheduled_by_job_id been changed? + """ + return self._set_scheduled_by_job_id( + dag_run=self, job_id=job_id, session=session, with_commit=with_commit + ) + @staticmethod @internal_api_call def _set_context_carrier( diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 5c0818a7b8087..19e41ce053f8e 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -831,6 +831,7 @@ def _set_ti_attrs(target, source, include_dag_run=False): target.next_kwargs = source.next_kwargs target.dag_version_id = source.dag_version_id target.context_carrier = source.context_carrier + target.span_status = source.span_status if include_dag_run: target.execution_date = source.execution_date @@ -852,7 +853,9 @@ def _set_ti_attrs(target, source, include_dag_run=False): target.dag_run.dag_version_id = source.dag_run.dag_version_id target.dag_run.updated_at = source.dag_run.updated_at target.dag_run.log_template_id = source.dag_run.log_template_id + target.dag_run.scheduled_by_job_id = source.dag_run.scheduled_by_job_id target.dag_run.context_carrier = source.dag_run.context_carrier + target.dag_run.span_status = source.dag_run.span_status def _refresh_from_db( @@ -895,10 +898,11 @@ def _refresh_from_db( # not be available. if not include_dag_run: inspector = inspect(ti) - # Check if the ti is detached or the dag_run isn't loaded. - if not inspector.detached and "dag_run" not in inspector.unloaded: - # It's best to include the dag_run whenever possible, - # in case there are changes to the span context_carrier. + # Check if the ti is detached or not loaded. + if not inspector.detached and "task_instance" not in inspector.unloaded: + # If the scheduler that started the dag_run has exited (gracefully or forcefully), + # there will be changes to the dag_run span context_carrier. + # It's best to include the dag_run whenever possible, so that the ti contains the updates. include_dag_run = True _set_ti_attrs(task_instance, ti, include_dag_run=include_dag_run) diff --git a/airflow/serialization/pydantic/dag_run.py b/airflow/serialization/pydantic/dag_run.py index c11db18ed2ab4..e387b63524bc7 100644 --- a/airflow/serialization/pydantic/dag_run.py +++ b/airflow/serialization/pydantic/dag_run.py @@ -58,6 +58,7 @@ class DagRunPydantic(BaseModelPydantic): consumed_asset_events: List[AssetEventPydantic] # noqa: UP006 log_template_id: Optional[int] triggered_by: Optional[DagRunTriggeredByType] + scheduled_by_job_id: Optional[int] context_carrier: Optional[dict] span_status: Optional[str] diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index 48bb62c7af5b3..ccbb7faaaa2a7 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -44,6 +44,7 @@ extract_spans_from_output, get_parent_child_dict, ) +from tests_common.test_utils.db import initial_db_init log = logging.getLogger("test_otel.TestOtelIntegration") @@ -94,10 +95,10 @@ def wait_for_dag_run_and_check_span_status(dag_id: str, run_id: str, max_wait_ti continue dag_run_state = dag_run.state - log.info("DAG Run state: %s.", dag_run_state) + log.debug("DAG Run state: %s.", dag_run_state) dag_run_span_status = dag_run.span_status - log.info("DAG Run span status: %s.", dag_run_span_status) + log.debug("DAG Run span status: %s.", dag_run_span_status) if dag_run_state in [State.SUCCESS, State.FAILED]: break @@ -111,7 +112,41 @@ def wait_for_dag_run_and_check_span_status(dag_id: str, run_id: str, max_wait_ti ), f"Dag run span status isn't {span_status} as expected.Actual status: {dag_run_span_status}." -def check_spans_with_continuance(output: str, dag: DAG): +def check_dag_run_state_and_span_status(dag_id: str, run_id: str, state: str, span_status: str): + with create_session() as session: + dag_run = ( + session.query(DagRun) + .filter( + DagRun.dag_id == dag_id, + DagRun.run_id == run_id, + ) + .first() + ) + + assert dag_run.state == state, f"Dag Run state isn't {state}. State: {dag_run.state}" + assert ( + dag_run.span_status == span_status + ), f"Dag Run span_status isn't {span_status}. Span_status: {dag_run.span_status}" + + +def check_ti_state_and_span_status(task_id: str, run_id: str, state: str, span_status: str): + with create_session() as session: + ti = ( + session.query(TaskInstance) + .filter( + TaskInstance.task_id == task_id, + TaskInstance.run_id == run_id, + ) + .first() + ) + + assert ti.state == state, f"Task instance state isn't {state}. State: {ti.state}" + assert ( + ti.span_status == span_status + ), f"Task instance span_status isn't {span_status}. Span_status: {ti.span_status}" + + +def check_spans_with_continuance(output: str, dag: DAG, continuance_for_t1: bool = True): # Get a list of lines from the captured output. output_lines = output.splitlines() @@ -134,6 +169,18 @@ def check_spans_with_continuance(output: str, dag: DAG): # |_ sub_span_3 # |_ sub_span_4 # |_ task_2 span + # + # If there is no continuance for task_1, then the span hierarchy is + # dag span + # |_ task_1 span + # |_ sub_span_1 + # |_ sub_span_2 + # |_ sub_span_3 + # |_ sub_span_4 + # |_ scheduler_exited span + # |_ new_scheduler span + # |_ dag span (continued) + # |_ task_2 span dag_id = dag.dag_id @@ -150,10 +197,15 @@ def check_spans_with_continuance(output: str, dag: DAG): f"{dag_id}_continued{CTX_PROP_SUFFIX}", ] - dag_continued_span_children_names = [ - f"{task1_id}_continued{CTX_PROP_SUFFIX}", - f"{task2_id}{CTX_PROP_SUFFIX}", - ] + if continuance_for_t1: + dag_continued_span_children_names = [ + f"{task1_id}_continued{CTX_PROP_SUFFIX}", + f"{task2_id}{CTX_PROP_SUFFIX}", + ] + else: + dag_continued_span_children_names = [ + f"{task2_id}{CTX_PROP_SUFFIX}", + ] task1_span_children_names = [ f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", @@ -203,12 +255,20 @@ def check_spans_with_continuance(output: str, dag: DAG): children_names=dag_continued_span_children_names, ) - # Check children of the continued task1 span. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", - children_names=task1_span_children_names, - ) + if continuance_for_t1: + # Check children of the continued task1 span. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", + children_names=task1_span_children_names, + ) + else: + # Check children of the task1 span. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}{CTX_PROP_SUFFIX}", + children_names=task1_span_children_names, + ) # Check children of task1 sub span1. assert_parent_children_spans_for_non_root( @@ -225,7 +285,11 @@ def check_spans_with_continuance(output: str, dag: DAG): ) -def check_spans_without_continuance(output: str, dag: DAG): +def check_spans_without_continuance( + output: str, dag: DAG, is_recreated: bool = False, check_t1_sub_spans: bool = True +): + recreated_suffix = "_recreated" if is_recreated else "" + # Get a list of lines from the captured output. output_lines = output.splitlines() @@ -245,6 +309,12 @@ def check_spans_without_continuance(output: str, dag: DAG): # |_ sub_span_3 # |_ sub_span_4 # |_ task_2 span + # + # In case task_1 has finished running and the span is recreated, + # the sub spans are lost and can't be recreated. The span hierarchy will be + # dag span + # |_ task_1 span + # |_ task_2 span dag_id = dag.dag_id @@ -252,10 +322,13 @@ def check_spans_without_continuance(output: str, dag: DAG): task1_id = task_instance_ids[0] task2_id = task_instance_ids[1] - dag_root_span_name = f"{dag_id}{CTX_PROP_SUFFIX}" + # Based on the current tests, only the root span and the task1 span will be recreated. + # TODO: Adjust accordingly, if there are more tests in the future + # that require other spans to be recreated as well. + dag_root_span_name = f"{dag_id}{recreated_suffix}{CTX_PROP_SUFFIX}" dag_root_span_children_names = [ - f"{task1_id}{CTX_PROP_SUFFIX}", + f"{task1_id}{recreated_suffix}{CTX_PROP_SUFFIX}", f"{task2_id}{CTX_PROP_SUFFIX}", ] @@ -300,26 +373,27 @@ def check_spans_without_continuance(output: str, dag: DAG): span_exists=False, ) - # Check children of the task1 span. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}{CTX_PROP_SUFFIX}", - children_names=task1_span_children_names, - ) + if check_t1_sub_spans: + # Check children of the task1 span. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}{recreated_suffix}{CTX_PROP_SUFFIX}", + children_names=task1_span_children_names, + ) - # Check children of task1 sub span1. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", - children_names=task1_sub_span1_children_span_names, - ) + # Check children of task1 sub span1. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + children_names=task1_sub_span1_children_span_names, + ) - # Check children of task1 sub span2. - assert_parent_children_spans_for_non_root( - span_dict=span_dict, - parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", - children_names=task1_sub_span2_children_span_names, - ) + # Check children of task1 sub span2. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", + children_names=task1_sub_span2_children_span_names, + ) @pytest.mark.integration("redis") @@ -336,7 +410,7 @@ class TestOtelIntegration: - run the test - check 'http://localhost:36686/' - To get span dumps on the stdout, run 'export log_level=debug'. + To get a db dump on the stdout, run 'export log_level=debug'. """ test_dir = os.path.dirname(os.path.abspath(__file__)) @@ -361,6 +435,8 @@ class TestOtelIntegration: "scheduler", ] + dags: dict[str, DAG] = {} + @classmethod def setup_class(cls): os.environ["AIRFLOW__TRACES__OTEL_ON"] = "True" @@ -374,6 +450,15 @@ def setup_class(cls): os.environ["AIRFLOW__SCHEDULER__STANDALONE_DAG_PROCESSOR"] = "False" os.environ["AIRFLOW__SCHEDULER__PROCESSOR_POLL_INTERVAL"] = "2" + # The heartrate is determined by the conf "AIRFLOW__SCHEDULER__SCHEDULER_HEARTBEAT_SEC". + # By default, the heartrate is 5 seconds. Every iteration of the scheduler loop check the + # time passed since the last heartbeat and if it was longer than the 5 second heartrate, + # it performs a heartbeat update. + # If there hasn't been a heartbeat for an amount of time longer than the + # SCHEDULER_HEALTH_CHECK_THRESHOLD, then the scheduler is considered unhealthy. + # Approximately, there is a scheduler heartbeat every 5-6 seconds. Set the threshold to 15. + os.environ["AIRFLOW__SCHEDULER__SCHEDULER_HEALTH_CHECK_THRESHOLD"] = "15" + os.environ["AIRFLOW__CORE__DAGS_FOLDER"] = f"{cls.dag_folder}" os.environ["AIRFLOW__CORE__LOAD_EXAMPLES"] = "False" @@ -382,6 +467,35 @@ def setup_class(cls): if cls.log_level == "debug": log.setLevel(logging.DEBUG) + initial_db_init() + + cls.dags = cls.serialize_and_get_dags() + + @classmethod + def serialize_and_get_dags(cls) -> dict[str, DAG]: + # Load DAGs from the dag directory. + dag_bag = DagBag(dag_folder=cls.dag_folder, include_examples=False) + + dag_ids = dag_bag.dag_ids + assert len(dag_ids) == 2 + + dag_dict: dict[str, DAG] = {} + with create_session() as session: + for dag_id in dag_ids: + dag = dag_bag.get_dag(dag_id) + dag_dict[dag_id] = dag + + assert dag is not None, f"DAG with ID {dag_id} not found." + + # Sync the DAG to the database. + dag.sync_to_db(session=session) + # Manually serialize the dag and write it to the db to avoid a db error. + SerializedDagModel.write_dag(dag, session=session) + + session.commit() + + return dag_dict + @pytest.fixture def celery_worker_env_vars(self, monkeypatch): os.environ["AIRFLOW__CORE__EXECUTOR"] = "CeleryExecutor" @@ -391,13 +505,17 @@ def celery_worker_env_vars(self, monkeypatch): ) monkeypatch.setattr(executor_loader, "_alias_to_executors", {"CeleryExecutor": executor_name}) + @pytest.fixture(autouse=True) + def reset_db(self): + reset_command = ["airflow", "db", "reset", "--yes"] + + # Reset the db using the cli. + subprocess.run(reset_command, check=True, env=os.environ.copy()) + def test_same_scheduler_processing_the_entire_dag( self, monkeypatch, celery_worker_env_vars, capfd, session ): - """ - Test that a DAG runs successfully and exports the correct spans. - Integration with a scheduler, a celery worker, a postgres db and a redis broker. - """ + """The same scheduler will start and finish the dag processing.""" celery_worker_process = None scheduler_process = None try: @@ -406,7 +524,11 @@ def test_same_scheduler_processing_the_entire_dag( celery_worker_process, scheduler_process = self.start_worker_and_scheduler1() dag_id = "otel_test_dag" - dag = self.serialize_and_get_dag(dag_id=dag_id) + + assert len(self.dags) > 0 + dag = self.dags[dag_id] + + assert dag is not None run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) @@ -414,40 +536,53 @@ def test_same_scheduler_processing_the_entire_dag( dag_id=dag_id, run_id=run_id, max_wait_time=90, span_status=SpanStatus.ENDED ) - if logging.root.level == logging.DEBUG: + # The ti span_status is updated while processing the executor events, + # which is after the dag_run state has been updated. + time.sleep(10) + + with create_session() as session: + tis: list[TaskInstance] = dag.get_task_instances(session=session) + + for ti in tis: + check_ti_state_and_span_status( + task_id=ti.task_id, run_id=run_id, state=State.SUCCESS, span_status=SpanStatus.ENDED + ) + finally: + if self.log_level == "debug": with create_session() as session: dump_airflow_metadata_db(session) - finally: + # Terminate the processes. celery_worker_process.terminate() celery_worker_process.wait() + celery_status = celery_worker_process.poll() + assert ( + celery_status is not None + ), "The celery worker process status is None, which means that it hasn't terminated as expected." + scheduler_process.terminate() scheduler_process.wait() + scheduler_status = scheduler_process.poll() + assert ( + scheduler_status is not None + ), "The scheduler_1 process status is None, which means that it hasn't terminated as expected." + out, err = capfd.readouterr() - log.debug("out-start --\n%s\n-- out-end", out) - log.debug("err-start --\n%s\n-- err-end", err) + log.info("out-start --\n%s\n-- out-end", out) + log.info("err-start --\n%s\n-- err-end", err) if self.use_otel != "true": # Dag run should have succeeded. Test the spans from the output. check_spans_without_continuance(output=out, dag=dag) - def test_another_scheduler_executing_the_second_task_and_the_rest_of_the_dag( + def test_scheduler_change_after_the_first_task_finishes( self, monkeypatch, celery_worker_env_vars, capfd, session ): """ - Similar to test_dag_spans_with_context_propagation but this test will start with one scheduler, - and during the dag execution, it will stop the process and start a new one. - - A txt file will be used for signaling the test and the dag in order to make sure that - the 1st scheduler is stopped while the first task is executing and that - the 2nd scheduler picks up the task and dag processing. - The steps will be - - The dag starts running, creates the file with a signal word and waits until the word is changed. - - The test checks if the file exist, stops the scheduler, starts a new scheduler and updates the file. - - The dag gets the update and continues until the task is finished. - At this point, the second scheduler should handle the rest of the dag processing. + The scheduler thread will be paused after the first task ends and a new scheduler process + will handle the rest of the dag processing. The paused thread will be resumed afterwards. """ celery_worker_process = None @@ -459,7 +594,7 @@ def test_another_scheduler_executing_the_second_task_and_the_rest_of_the_dag( celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() dag_id = "otel_test_dag" - dag = self.serialize_and_get_dag(dag_id=dag_id) + dag = self.dags[dag_id] run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) @@ -489,47 +624,28 @@ def test_another_scheduler_executing_the_second_task_and_the_rest_of_the_dag( with capfd.disabled(): # When we pause the scheduler1 thread, capfd keeps trying to read the # file descriptors for the process and ends up freezing the test. - # There won't be any exported spans from the following code, - # so not capturing the output, doesn't make a difference. + # Temporarily disable capfd to avoid that. scheduler_process_1.send_signal(signal.SIGSTOP) - scheduler_process_2 = subprocess.Popen( - self.scheduler_command_args, - env=os.environ.copy(), - stdout=None, - stderr=None, - ) - - with create_session() as session: - dag_run = ( - session.query(DagRun) - .filter( - DagRun.dag_id == dag_id, - DagRun.run_id == run_id, - ) - .first() - ) - - assert ( - dag_run.state == State.RUNNING - ), f"Dag Run state isn't RUNNING. State: {dag_run.state}" - assert ( - dag_run.span_status == SpanStatus.ACTIVE - ), f"Dag Run span_status isn't ACTIVE. Span_status: {dag_run.span_status}" + scheduler_process_2 = subprocess.Popen( + self.scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) - # Wait for scheduler2 to be up and running. - time.sleep(10) + check_dag_run_state_and_span_status( + dag_id=dag_id, run_id=run_id, state=State.RUNNING, span_status=SpanStatus.ACTIVE + ) - wait_for_dag_run_and_check_span_status( - dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.SHOULD_END - ) + # Wait for scheduler2 to be up and running. + time.sleep(10) - if logging.root.level == logging.DEBUG: - with create_session() as session: - dump_airflow_metadata_db(session) + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.SHOULD_END + ) - scheduler_process_1.send_signal(signal.SIGCONT) - scheduler_process_2.terminate() + scheduler_process_1.send_signal(signal.SIGCONT) # Wait for the scheduler to start again and continue running. time.sleep(10) @@ -539,6 +655,10 @@ def test_another_scheduler_executing_the_second_task_and_the_rest_of_the_dag( ) finally: + if self.log_level == "debug": + with create_session() as session: + dump_airflow_metadata_db(session) + # Terminate the processes. celery_worker_process.terminate() celery_worker_process.wait() @@ -546,26 +666,27 @@ def test_another_scheduler_executing_the_second_task_and_the_rest_of_the_dag( scheduler_process_1.terminate() scheduler_process_1.wait() - # scheduler_process_2.terminate() + scheduler_process_2.terminate() scheduler_process_2.wait() out, err = capfd.readouterr() - log.debug("out-start --\n%s\n-- out-end", out) - log.debug("err-start --\n%s\n-- err-end", err) + log.info("out-start --\n%s\n-- out-end", out) + log.info("err-start --\n%s\n-- err-end", err) if self.use_otel != "true": # Dag run should have succeeded. Test the spans in the output. check_spans_without_continuance(output=out, dag=dag) - def test_scheduler_change_in_the_middle_of_first_task_execution_until_the_end( + def test_scheduler_change_in_the_middle_of_first_task_until_the_end( self, monkeypatch, celery_worker_env_vars, capfd, session ): """ - Similar to test_dag_spans_with_context_propagation but this test will start with one scheduler, - and during the dag execution, it will stop the process and start a new one. + The scheduler that starts the dag run, will be paused and a new scheduler process will handle + the rest of the dag processing. The paused thread will be resumed so that the test + can check that it properly handles the spans. A txt file will be used for signaling the test and the dag in order to make sure that - the 1st scheduler is stopped while the first task is executing and that + the 1st scheduler is handled accordingly while the first task is executing and that the 2nd scheduler picks up the task and dag processing. The steps will be - The dag starts running, creates the file with a signal word and waits until the word is changed. @@ -582,86 +703,68 @@ def test_scheduler_change_in_the_middle_of_first_task_execution_until_the_end( # so that the test can capture their output. celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() - dag_id = "otel_test_dag" - dag = self.serialize_and_get_dag(dag_id=dag_id) + dag_id = "otel_test_dag_with_pause" + dag = self.dags[dag_id] run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) - with create_session() as session: - tis: list[TaskInstance] = dag.get_task_instances(session=session) - - task_1 = tis[0] + # Control file path. + control_file = os.path.join(self.dag_folder, "dag_control.txt") while True: - with create_session() as session: - ti = ( - session.query(TaskInstance) - .filter( - TaskInstance.task_id == task_1.task_id, - TaskInstance.run_id == task_1.run_id, - ) - .first() - ) - - if ti is None: - continue - - # Wait until the task has been finished. - if ti.state in State.finished: + try: + with open(control_file) as file: + # If it reaches inside the block, then the file exists and the test can read it. break + except FileNotFoundError: + print("Control file not found. Waiting...") + time.sleep(1) + continue with capfd.disabled(): # When we pause the scheduler1 thread, capfd keeps trying to read the # file descriptors for the process and ends up freezing the test. - # There won't be any exported spans from the following code, - # so not capturing the output, doesn't make a difference. + # Temporarily disable capfd to avoid that. scheduler_process_1.send_signal(signal.SIGSTOP) - scheduler_process_2 = subprocess.Popen( - self.scheduler_command_args, - env=os.environ.copy(), - stdout=None, - stderr=None, - ) - - with create_session() as session: - dag_run = ( - session.query(DagRun) - .filter( - DagRun.dag_id == dag_id, - DagRun.run_id == run_id, - ) - .first() - ) + scheduler_process_2 = subprocess.Popen( + self.scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) - assert ( - dag_run.state == State.RUNNING - ), f"Dag Run state isn't RUNNING. State: {dag_run.state}" - assert ( - dag_run.span_status == SpanStatus.ACTIVE - ), f"Dag Run span_status isn't ACTIVE. Span_status: {dag_run.span_status}" + # Wait for scheduler2 to be up and running. + time.sleep(10) - # Wait for scheduler2 to be up and running. - time.sleep(10) + check_dag_run_state_and_span_status( + dag_id=dag_id, run_id=run_id, state=State.RUNNING, span_status=SpanStatus.ACTIVE + ) - wait_for_dag_run_and_check_span_status( - dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.SHOULD_END - ) + # Rewrite the file to unpause the dag. + with open(control_file, "w") as file: + file.write("continue") - if logging.root.level == logging.DEBUG: - with create_session() as session: - dump_airflow_metadata_db(session) + # Scheduler2 should finish processing the dag and set the status + # so that scheduler1 can end the spans when it is resumed. + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.SHOULD_END + ) - scheduler_process_1.send_signal(signal.SIGCONT) - scheduler_process_2.terminate() + scheduler_process_1.send_signal(signal.SIGCONT) # Wait for the scheduler to start again and continue running. time.sleep(10) + # Scheduler1 should end the spans and update the status. wait_for_dag_run_and_check_span_status( dag_id=dag_id, run_id=run_id, max_wait_time=30, span_status=SpanStatus.ENDED ) finally: + if self.log_level == "debug": + with create_session() as session: + dump_airflow_metadata_db(session) + # Terminate the processes. celery_worker_process.terminate() celery_worker_process.wait() @@ -669,12 +772,12 @@ def test_scheduler_change_in_the_middle_of_first_task_execution_until_the_end( scheduler_process_1.terminate() scheduler_process_1.wait() - # scheduler_process_2.terminate() + scheduler_process_2.terminate() scheduler_process_2.wait() out, err = capfd.readouterr() - log.debug("out-start --\n%s\n-- out-end", out) - log.debug("err-start --\n%s\n-- err-end", err) + log.info("out-start --\n%s\n-- out-end", out) + log.info("err-start --\n%s\n-- err-end", err) if self.use_otel != "true": # Dag run should have succeeded. Test the spans in the output. @@ -684,17 +787,9 @@ def test_scheduler_exits_gracefully_in_the_middle_of_the_first_task( self, monkeypatch, celery_worker_env_vars, capfd, session ): """ - Similar to test_dag_spans_with_context_propagation but this test will start with one scheduler, - and during the dag execution, it will stop the process and start a new one. - - A txt file will be used for signaling the test and the dag in order to make sure that - the 1st scheduler is stopped while the first task is executing and that - the 2nd scheduler picks up the task and dag processing. - The steps will be - - The dag starts running, creates the file with a signal word and waits until the word is changed. - - The test checks if the file exist, stops the scheduler, starts a new scheduler and updates the file. - - The dag gets the update and continues until the task is finished. - At this point, the second scheduler should handle the rest of the dag processing. + The scheduler that starts the dag run will be stopped, while the first task is executing, + and start a new scheduler will be started. That way, the new process will pick up the dag processing. + The initial scheduler will exit gracefully. """ celery_worker_process = None @@ -706,7 +801,7 @@ def test_scheduler_exits_gracefully_in_the_middle_of_the_first_task( celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() dag_id = "otel_test_dag_with_pause" - dag = self.serialize_and_get_dag(dag_id=dag_id) + dag = self.dags[dag_id] run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) @@ -727,6 +822,10 @@ def test_scheduler_exits_gracefully_in_the_middle_of_the_first_task( # Terminate scheduler1 and start scheduler2. scheduler_process_1.terminate() + check_dag_run_state_and_span_status( + dag_id=dag_id, run_id=run_id, state=State.RUNNING, span_status=SpanStatus.NEEDS_CONTINUANCE + ) + scheduler_process_2 = subprocess.Popen( self.scheduler_command_args, env=os.environ.copy(), @@ -744,12 +843,11 @@ def test_scheduler_exits_gracefully_in_the_middle_of_the_first_task( wait_for_dag_run_and_check_span_status( dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.ENDED ) - - if logging.root.level == logging.DEBUG: + finally: + if self.log_level == "debug": with create_session() as session: dump_airflow_metadata_db(session) - finally: # Terminate the processes. celery_worker_process.terminate() celery_worker_process.wait() @@ -760,8 +858,8 @@ def test_scheduler_exits_gracefully_in_the_middle_of_the_first_task( scheduler_process_2.wait() out, err = capfd.readouterr() - log.debug("out-start --\n%s\n-- out-end", out) - log.debug("err-start --\n%s\n-- err-end", err) + log.info("out-start --\n%s\n-- out-end", out) + log.info("err-start --\n%s\n-- err-end", err) if self.use_otel != "true": # Dag run should have succeeded. Test the spans in the output. @@ -771,21 +869,11 @@ def test_scheduler_exits_forcefully_in_the_middle_of_the_first_task( self, monkeypatch, celery_worker_env_vars, capfd, session ): """ - This test will start with one scheduler, and during the dag execution, - it will stop the process and start a new one. - - A txt file will be used for signaling the test and the dag in order to make sure that - the 1st scheduler is stopped while the first task is executing and that - the 2nd scheduler picks up the task and dag processing. - The steps will be - - The dag starts running, creates the file with a signal word and waits until the word is changed. - - The test checks if the file exist, stops the scheduler, starts a new scheduler and updates the file. - - The dag gets the update and continues until the task is finished. - At this point, the second scheduler should handle the rest of the dag processing. + The first scheduler will exit forcefully while the first task is running, + so that it won't have time end any active spans. """ celery_worker_process = None - scheduler_process_1 = None scheduler_process_2 = None try: # Start the processes here and not as fixtures or in a common setup, @@ -793,7 +881,7 @@ def test_scheduler_exits_forcefully_in_the_middle_of_the_first_task( celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() dag_id = "otel_test_dag_with_pause" - dag = self.serialize_and_get_dag(dag_id=dag_id) + dag = self.dags[dag_id] run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) @@ -811,8 +899,16 @@ def test_scheduler_exits_forcefully_in_the_middle_of_the_first_task( continue # Since, we are past the loop, then the file exists and the dag has been paused. - # Terminate scheduler1 and start scheduler2. - scheduler_process_1.terminate() + # Kill scheduler1 and start scheduler2. + scheduler_process_1.send_signal(signal.SIGKILL) + + # The process shouldn't have changed the span_status. + check_dag_run_state_and_span_status( + dag_id=dag_id, run_id=run_id, state=State.RUNNING, span_status=SpanStatus.ACTIVE + ) + + # Wait so that the health threshold passes and scheduler1 is considered unhealthy. + time.sleep(15) scheduler_process_2 = subprocess.Popen( self.scheduler_command_args, @@ -831,28 +927,115 @@ def test_scheduler_exits_forcefully_in_the_middle_of_the_first_task( wait_for_dag_run_and_check_span_status( dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.ENDED ) - - if logging.root.level == logging.DEBUG: + finally: + if self.log_level == "debug": with create_session() as session: dump_airflow_metadata_db(session) - finally: # Terminate the processes. celery_worker_process.terminate() celery_worker_process.wait() - scheduler_process_1.wait() + scheduler_process_2.terminate() + scheduler_process_2.wait() + + out, err = capfd.readouterr() + log.info("out-start --\n%s\n-- out-end", out) + log.info("err-start --\n%s\n-- err-end", err) + + if self.use_otel != "true": + # Dag run should have succeeded. Test the spans in the output. + check_spans_without_continuance(output=out, dag=dag, is_recreated=True) + + def test_scheduler_exits_forcefully_after_the_first_task_finishes( + self, monkeypatch, celery_worker_env_vars, capfd, session + ): + """ + The first scheduler will exit forcefully after the first task finishes, + so that it won't have time end any active spans. + In this scenario, the sub-spans for the first task will be lost. + The only way to retrieve them, would be to re-run the task. + """ + + celery_worker_process = None + scheduler_process_2 = None + try: + # Start the processes here and not as fixtures or in a common setup, + # so that the test can capture their output. + celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() + + dag_id = "otel_test_dag" + dag = self.dags[dag_id] + + run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) + + with create_session() as session: + tis: list[TaskInstance] = dag.get_task_instances(session=session) + + task_1 = tis[0] + + while True: + with create_session() as session: + ti = ( + session.query(TaskInstance) + .filter( + TaskInstance.task_id == task_1.task_id, + TaskInstance.run_id == task_1.run_id, + ) + .first() + ) + + if ti is None: + continue + + # Wait until the task has been finished. + if ti.state in State.finished: + break + + # Since, we are past the loop, then the file exists and the dag has been paused. + # Kill scheduler1 and start scheduler2. + scheduler_process_1.send_signal(signal.SIGKILL) + + # The process shouldn't have changed the span_status. + check_dag_run_state_and_span_status( + dag_id=dag_id, run_id=run_id, state=State.RUNNING, span_status=SpanStatus.ACTIVE + ) + + time.sleep(15) + # The task should be adopted. + + scheduler_process_2 = subprocess.Popen( + self.scheduler_command_args, + env=os.environ.copy(), + stdout=None, + stderr=None, + ) + + # Wait for scheduler2 to be up and running. + time.sleep(10) + + wait_for_dag_run_and_check_span_status( + dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.ENDED + ) + finally: + if self.log_level == "debug": + with create_session() as session: + dump_airflow_metadata_db(session) + + # Terminate the processes. + celery_worker_process.terminate() + celery_worker_process.wait() scheduler_process_2.terminate() scheduler_process_2.wait() out, err = capfd.readouterr() - log.debug("out-start --\n%s\n-- out-end", out) - log.debug("err-start --\n%s\n-- err-end", err) + log.info("out-start --\n%s\n-- out-end", out) + log.info("err-start --\n%s\n-- err-end", err) if self.use_otel != "true": # Dag run should have succeeded. Test the spans in the output. - check_spans_with_continuance(output=out, dag=dag) + check_spans_without_continuance(output=out, dag=dag, is_recreated=True, check_t1_sub_spans=False) def start_worker_and_scheduler1(self): celery_worker_process = subprocess.Popen( @@ -873,20 +1056,3 @@ def start_worker_and_scheduler1(self): time.sleep(10) return celery_worker_process, scheduler_process - - def serialize_and_get_dag(self, dag_id: str) -> DAG: - # Load DAGs from the dag directory. - dag_bag = DagBag(dag_folder=self.dag_folder, include_examples=False) - - dag = dag_bag.get_dag(dag_id) - - assert dag is not None, f"DAG with ID {dag_id} not found." - - with create_session() as session: - # Sync the DAG to the database. - dag.sync_to_db(session=session) - # Manually serialize the dag and write it to the db to avoid a db error. - SerializedDagModel.write_dag(dag, session=session) - session.commit() - - return dag diff --git a/tests/integration/otel/test_utils.py b/tests/integration/otel/test_utils.py index f79da9a7a2ffd..8654327806016 100644 --- a/tests/integration/otel/test_utils.py +++ b/tests/integration/otel/test_utils.py @@ -56,38 +56,12 @@ def dump_airflow_metadata_db(session): log.debug("\n-----END_airflow_db_dump-----\n") -# Example span from the ConsoleSpanExporter -# { -# "name": "perform_heartbeat", -# "context": { -# "trace_id": "0xa18781ea597c3d07c85e95fd3a6d7d40", -# "span_id": "0x8ae7bb13ec5b28ba", -# "trace_state": "[]" -# }, -# "kind": "SpanKind.INTERNAL", -# "parent_id": "0x17ac77a4a840758d", -# "start_time": "2024-10-30T16:19:33.947155Z", -# "end_time": "2024-10-30T16:19:33.947192Z", -# "status": { -# "status_code": "UNSET" -# }, -# "attributes": {}, -# "events": [], -# "links": [], -# "resource": { -# "attributes": { -# "telemetry.sdk.language": "python", -# "telemetry.sdk.name": "opentelemetry", -# "telemetry.sdk.version": "1.27.0", -# "host.name": "host.local", -# "service.name": "Airflow" -# }, -# "schema_url": "" -# } -# } - - -def extract_spans_from_output(output_lines): +def extract_spans_from_output(output_lines: list): + """ + For a give list of ConsoleSpanExporter output lines, it extracts the json spans + and creates two dictionaries. + :return: root spans dict (key: root_span_id - value: root_span), spans dict (key: span_id - value: span) + """ span_dict = {} root_span_dict = {} total_lines = len(output_lines) @@ -103,7 +77,10 @@ def extract_spans_from_output(output_lines): index += 1 while index < total_lines: line = output_lines[index] - json_lines.append(line) + # The 'command' line uses single quotes, and it results in an error when parsing the json. + # It's not needed when checking for spans. So instead of formatting it properly, just skip it. + if '"command":' not in line: + json_lines.append(line) if line.strip().startswith("}") and line == "}": # Json end. # Since, this is the end of the object, break the loop. break @@ -138,7 +115,10 @@ def get_id_for_a_given_name(span_dict: dict, span_name: str): def get_parent_child_dict(root_span_dict, span_dict): - """Create a dictionary with parent-child span relationships.""" + """ + Create a dictionary with parent-child span relationships. + :return: key: root_span_id - value: list of child spans + """ parent_child_dict = {} for root_span_id, root_span in root_span_dict.items(): # Compare each 'root_span_id' with each 'parent_id' from the span_dict. @@ -163,10 +143,14 @@ def get_child_list_for_non_root(span_dict: dict, span_name: str): """ Get a list of children spans for a parent span that isn't also a root span. e.g. a task span with sub-spans, is a parent span but not a root span. + :return: list of spans """ parent_span_id = get_id_for_a_given_name(span_dict=span_dict, span_name=span_name) parent_span = span_dict.get(parent_span_id) + if parent_span is None: + return [] + child_span_list = [] for span_id, span in span_dict.items(): if span_id == parent_span_id: @@ -271,3 +255,520 @@ def assert_span_not_in_children_spans( ), f"Span '{child_name}' shouldn't be a child of span '{parent_name}', but it is." else: assert child_id is None, f"Span '{child_name}' shouldn't exist but it does." + + +class TestUtilsUnit: + # The method that extracts the spans from the output, + # counts that there is no indentation on the cli, when a span starts and finishes. + example_output = """ +{ + "name": "test_dag", + "context": { + "trace_id": "0x01f441c9c53e793e8808c77939ddbf36", + "span_id": "0x779a3a331684439e", + "trace_state": "[]" + }, + "kind": "SpanKind.INTERNAL", + "parent_id": null, + "start_time": "2024-11-30T14:01:21.738052Z", + "end_time": "2024-11-30T14:01:36.541442Z", + "status": { + "status_code": "UNSET" + }, + "attributes": { + "airflow.category": "DAG runs", + "airflow.dag_run.dag_id": "otel_test_dag_with_pause", + "airflow.dag_run.execution_date": "2024-11-30 14:01:15+00:00", + "airflow.dag_run.run_id": "manual__2024-11-30T14:01:15.333003+00:00", + "airflow.dag_run.queued_at": "2024-11-30 14:01:21.738052+00:00", + "airflow.dag_run.run_start_date": "2024-11-30 14:01:22.192655+00:00", + "airflow.dag_run.run_end_date": "2024-11-30 14:01:36.541442+00:00", + "airflow.dag_run.run_duration": "14.348787", + "airflow.dag_run.state": "success", + "airflow.dag_run.external_trigger": "True", + "airflow.dag_run.run_type": "manual", + "airflow.dag_run.data_interval_start": "2024-11-30 14:01:15+00:00", + "airflow.dag_run.data_interval_end": "2024-11-30 14:01:15+00:00", + "airflow.dag_version.version": "2", + "airflow.dag_run.conf": "{}" + }, + "events": [ + { + "name": "airflow.dag_run.queued", + "timestamp": "2024-11-30T14:01:21.738052Z", + "attributes": {} + }, + { + "name": "airflow.dag_run.started", + "timestamp": "2024-11-30T14:01:22.192655Z", + "attributes": {} + }, + { + "name": "airflow.dag_run.ended", + "timestamp": "2024-11-30T14:01:36.541442Z", + "attributes": {} + } + ], + "links": [], + "resource": { + "attributes": { + "telemetry.sdk.language": "python", + "telemetry.sdk.name": "opentelemetry", + "telemetry.sdk.version": "1.27.0", + "host.name": "351295342ba2", + "service.name": "Airflow" + }, + "schema_url": "" + } +} +{ + "name": "task_1", + "context": { + "trace_id": "0x01f441c9c53e793e8808c77939ddbf36", + "span_id": "0xba9f48dcfac5d40a", + "trace_state": "[]" + }, + "kind": "SpanKind.INTERNAL", + "parent_id": "0x779a3a331684439e", + "start_time": "2024-11-30T14:01:22.220785Z", + "end_time": "2024-11-30T14:01:34.339423Z", + "status": { + "status_code": "UNSET" + }, + "attributes": { + "airflow.category": "scheduler", + "airflow.task.task_id": "task_1", + "airflow.task.dag_id": "otel_test_dag_with_pause", + "airflow.task.state": "success", + "airflow.task.start_date": "2024-11-30 14:01:23.468047+00:00", + "airflow.task.end_date": "2024-11-30 14:01:34.339423+00:00", + "airflow.task.duration": 10.871376, + "airflow.task.executor_config": "{}", + "airflow.task.execution_date": "2024-11-30 14:01:15+00:00", + "airflow.task.hostname": "351295342ba2", + "airflow.task.log_url": "http://localhost:8080/dags/otel_test_dag_with_pause/grid?dag_run_id=manual__2024-11-30T14%3A01%3A15.333003%2B00%3A00&task_id=task_1&base_date=2024-11-30T14%3A01%3A15%2B0000&tab=logs", + "airflow.task.operator": "PythonOperator", + "airflow.task.try_number": 1, + "airflow.task.executor_state": "success", + "airflow.task.pool": "default_pool", + "airflow.task.queue": "default", + "airflow.task.priority_weight": 2, + "airflow.task.queued_dttm": "2024-11-30 14:01:22.216965+00:00", + "airflow.task.queued_by_job_id": 1, + "airflow.task.pid": 1748 + }, + "events": [ + { + "name": "task to trigger", + "timestamp": "2024-11-30T14:01:22.220873Z", + "attributes": { + "command": "['airflow', 'tasks', 'run', 'otel_test_dag_with_pause', 'task_1', 'manual__2024-11-30T14:01:15.333003+00:00', '--local', '--subdir', 'DAGS_FOLDER/otel_test_dag_with_pause.py', '--carrier', '{\"traceparent\": \"00-01f441c9c53e793e8808c77939ddbf36-ba9f48dcfac5d40a-01\"}']", + "conf": "{}" + } + }, + { + "name": "airflow.task.queued", + "timestamp": "2024-11-30T14:01:22.216965Z", + "attributes": {} + }, + { + "name": "airflow.task.started", + "timestamp": "2024-11-30T14:01:23.468047Z", + "attributes": {} + }, + { + "name": "airflow.task.ended", + "timestamp": "2024-11-30T14:01:34.339423Z", + "attributes": {} + } + ], + "links": [ + { + "context": { + "trace_id": "0x01f441c9c53e793e8808c77939ddbf36", + "span_id": "0x779a3a331684439e", + "trace_state": "[]" + }, + "attributes": { + "meta.annotation_type": "link", + "from": "parenttrace" + } + } + ], + "resource": { + "attributes": { + "telemetry.sdk.language": "python", + "telemetry.sdk.name": "opentelemetry", + "telemetry.sdk.version": "1.27.0", + "host.name": "351295342ba2", + "service.name": "Airflow" + }, + "schema_url": "" + } +} +{ + "name": "start_new_processes", + "context": { + "trace_id": "0x3f6d11237d2b2b8cb987e7ec923a4dc4", + "span_id": "0x0b133494760fa56d", + "trace_state": "[]" + }, + "kind": "SpanKind.INTERNAL", + "parent_id": "0xcf656e5db2b777be", + "start_time": "2024-11-30T14:01:29.316313Z", + "end_time": "2024-11-30T14:01:29.316397Z", + "status": { + "status_code": "UNSET" + }, + "attributes": {}, + "events": [], + "links": [], + "resource": { + "attributes": { + "telemetry.sdk.language": "python", + "telemetry.sdk.name": "opentelemetry", + "telemetry.sdk.version": "1.27.0", + "host.name": "351295342ba2", + "service.name": "Airflow" + }, + "schema_url": "" + } +} +{ + "name": "task_2", + "context": { + "trace_id": "0x01f441c9c53e793e8808c77939ddbf36", + "span_id": "0xe573c104743b6d34", + "trace_state": "[]" + }, + "kind": "SpanKind.INTERNAL", + "parent_id": "0x779a3a331684439e", + "start_time": "2024-11-30T14:01:34.698666Z", + "end_time": "2024-11-30T14:01:36.002687Z", + "status": { + "status_code": "UNSET" + }, + "attributes": { + "airflow.category": "scheduler", + "airflow.task.task_id": "task_2", + "airflow.task.dag_id": "otel_test_dag_with_pause", + "airflow.task.state": "success", + "airflow.task.start_date": "2024-11-30 14:01:35.872318+00:00", + "airflow.task.end_date": "2024-11-30 14:01:36.002687+00:00", + "airflow.task.duration": 0.130369, + "airflow.task.executor_config": "{}", + "airflow.task.execution_date": "2024-11-30 14:01:15+00:00", + "airflow.task.hostname": "351295342ba2", + "airflow.task.log_url": "http://localhost:8080/dags/otel_test_dag_with_pause/grid?dag_run_id=manual__2024-11-30T14%3A01%3A15.333003%2B00%3A00&task_id=task_2&base_date=2024-11-30T14%3A01%3A15%2B0000&tab=logs", + "airflow.task.operator": "PythonOperator", + "airflow.task.try_number": 1, + "airflow.task.executor_state": "success", + "airflow.task.pool": "default_pool", + "airflow.task.queue": "default", + "airflow.task.priority_weight": 1, + "airflow.task.queued_dttm": "2024-11-30 14:01:34.694842+00:00", + "airflow.task.queued_by_job_id": 3, + "airflow.task.pid": 1950 + }, + "events": [ + { + "name": "task to trigger", + "timestamp": "2024-11-30T14:01:34.698810Z", + "attributes": { + "command": "['airflow', 'tasks', 'run', 'otel_test_dag_with_pause', 'task_2', 'manual__2024-11-30T14:01:15.333003+00:00', '--local', '--subdir', 'DAGS_FOLDER/otel_test_dag_with_pause.py', '--carrier', '{\"traceparent\": \"00-01f441c9c53e793e8808c77939ddbf36-e573c104743b6d34-01\"}']", + "conf": "{}" + } + }, + { + "name": "airflow.task.queued", + "timestamp": "2024-11-30T14:01:34.694842Z", + "attributes": {} + }, + { + "name": "airflow.task.started", + "timestamp": "2024-11-30T14:01:35.872318Z", + "attributes": {} + }, + { + "name": "airflow.task.ended", + "timestamp": "2024-11-30T14:01:36.002687Z", + "attributes": {} + } + ], + "links": [ + { + "context": { + "trace_id": "0x01f441c9c53e793e8808c77939ddbf36", + "span_id": "0x779a3a331684439e", + "trace_state": "[]" + }, + "attributes": { + "meta.annotation_type": "link", + "from": "parenttrace" + } + } + ], + "resource": { + "attributes": { + "telemetry.sdk.language": "python", + "telemetry.sdk.name": "opentelemetry", + "telemetry.sdk.version": "1.27.0", + "host.name": "351295342ba2", + "service.name": "Airflow" + }, + "schema_url": "" + } +} +{ + "name": "task_1_sub_span", + "context": { + "trace_id": "0x01f441c9c53e793e8808c77939ddbf36", + "span_id": "0x7fc9e2289c7df4b8", + "trace_state": "[]" + }, + "kind": "SpanKind.INTERNAL", + "parent_id": "0xba9f48dcfac5d40a", + "start_time": "2024-11-30T14:01:34.321996Z", + "end_time": "2024-11-30T14:01:34.324249Z", + "status": { + "status_code": "UNSET" + }, + "attributes": { + "attr1": "val1" + }, + "events": [], + "links": [ + { + "context": { + "trace_id": "0x01f441c9c53e793e8808c77939ddbf36", + "span_id": "0xba9f48dcfac5d40a", + "trace_state": "[]" + }, + "attributes": { + "meta.annotation_type": "link", + "from": "parenttrace" + } + } + ], + "resource": { + "attributes": { + "telemetry.sdk.language": "python", + "telemetry.sdk.name": "opentelemetry", + "telemetry.sdk.version": "1.27.0", + "host.name": "351295342ba2", + "service.name": "Airflow" + }, + "schema_url": "" + } +} +{ + "name": "emit_metrics", + "context": { + "trace_id": "0x3f6d11237d2b2b8cb987e7ec923a4dc4", + "span_id": "0xa19a88e8dac9645b", + "trace_state": "[]" + }, + "kind": "SpanKind.INTERNAL", + "parent_id": "0xcf656e5db2b777be", + "start_time": "2024-11-30T14:01:29.315255Z", + "end_time": "2024-11-30T14:01:29.315290Z", + "status": { + "status_code": "UNSET" + }, + "attributes": { + "total_parse_time": 0.9342440839973278, + "dag_bag_size": 2, + "import_errors": 0 + }, + "events": [], + "links": [], + "resource": { + "attributes": { + "telemetry.sdk.language": "python", + "telemetry.sdk.name": "opentelemetry", + "telemetry.sdk.version": "1.27.0", + "host.name": "351295342ba2", + "service.name": "Airflow" + }, + "schema_url": "" + } +} +{ + "name": "dag_parsing_loop", + "context": { + "trace_id": "0x3f6d11237d2b2b8cb987e7ec923a4dc4", + "span_id": "0xcf656e5db2b777be", + "trace_state": "[]" + }, + "kind": "SpanKind.INTERNAL", + "parent_id": null, + "start_time": "2024-11-30T14:01:28.382690Z", + "end_time": "2024-11-30T14:01:29.316499Z", + "status": { + "status_code": "UNSET" + }, + "attributes": {}, + "events": [ + { + "name": "heartbeat", + "timestamp": "2024-11-30T14:01:29.313549Z", + "attributes": {} + }, + { + "name": "_kill_timed_out_processors", + "timestamp": "2024-11-30T14:01:29.314763Z", + "attributes": {} + }, + { + "name": "prepare_file_path_queue", + "timestamp": "2024-11-30T14:01:29.315300Z", + "attributes": {} + }, + { + "name": "start_new_processes", + "timestamp": "2024-11-30T14:01:29.315941Z", + "attributes": {} + }, + { + "name": "collect_results", + "timestamp": "2024-11-30T14:01:29.316409Z", + "attributes": {} + }, + { + "name": "print_stat", + "timestamp": "2024-11-30T14:01:29.316432Z", + "attributes": {} + } + ], + "links": [], + "resource": { + "attributes": { + "telemetry.sdk.language": "python", + "telemetry.sdk.name": "opentelemetry", + "telemetry.sdk.version": "1.27.0", + "host.name": "351295342ba2", + "service.name": "Airflow" + }, + "schema_url": "" + } +} + """ + + # In the example output, there are two parent child relationships. + # + # test_dag + # |_ task_1 + # |_ task_1_sub_span + # |_ task_2 + # + # dag_parsing_loop + # |_ emit_metrics + # |_ start_new_processes + + def test_extract_spans_from_output(self): + output_lines = self.example_output.splitlines() + root_span_dict, span_dict = extract_spans_from_output(output_lines) + + assert len(root_span_dict) == 2 + assert len(span_dict) == 7 + + expected_root_span_names = ["test_dag", "dag_parsing_loop"] + actual_root_span_names = [] + for key, value in root_span_dict.items(): + assert key == value["context"]["span_id"] + assert value["parent_id"] is None + actual_root_span_names.append(value["name"]) + + assert sorted(actual_root_span_names) == sorted(expected_root_span_names) + + expected_span_names = [ + "test_dag", + "task_1", + "task_1_sub_span", + "task_2", + "dag_parsing_loop", + "emit_metrics", + "start_new_processes", + ] + actual_span_names = [] + for key, value in span_dict.items(): + assert key == value["context"]["span_id"] + actual_span_names.append(value["name"]) + + assert sorted(actual_span_names) == sorted(expected_span_names) + + def test_get_id_for_a_given_name(self): + output_lines = self.example_output.splitlines() + root_span_dict, span_dict = extract_spans_from_output(output_lines) + + span_name_to_test = "test_dag" + + span_id = get_id_for_a_given_name(span_dict, span_name_to_test) + + # Get the id from the two dictionaries, and then cross-reference the name. + span_from_root_dict = root_span_dict.get(span_id) + span_from_dict = span_dict.get(span_id) + + assert span_from_root_dict is not None + assert span_from_dict is not None + + assert span_name_to_test == span_from_root_dict["name"] + assert span_name_to_test == span_from_dict["name"] + + def test_get_parent_child_dict(self): + output_lines = self.example_output.splitlines() + root_span_dict, span_dict = extract_spans_from_output(output_lines) + + parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) + + # There are two root spans. The dictionary should also have length equal to two. + assert len(parent_child_dict) == 2 + + assert sorted(root_span_dict.keys()) == sorted(parent_child_dict.keys()) + + for root_span_id, child_spans in parent_child_dict.items(): + # Both root spans have two direct child spans. + assert len(child_spans) == 2 + + root_span = root_span_dict.get(root_span_id) + root_span_trace_id = root_span["context"]["trace_id"] + + expected_child_span_names = [] + if root_span["name"] == "test_dag": + expected_child_span_names.extend(["task_1", "task_2"]) + elif root_span["name"] == "dag_parsing_loop": + expected_child_span_names.extend(["emit_metrics", "start_new_processes"]) + + actual_child_span_names = [] + + for child_span in child_spans: + # root_span_id should be the parent. + assert root_span_id == child_span["parent_id"] + # all spans should have the same trace_id. + assert root_span_trace_id == child_span["context"]["trace_id"] + actual_child_span_names.append(child_span["name"]) + + assert sorted(actual_child_span_names) == sorted(expected_child_span_names) + + def test_get_child_list_for_non_root(self): + output_lines = self.example_output.splitlines() + root_span_dict, span_dict = extract_spans_from_output(output_lines) + + span_name_to_test = "task_1" + span_id = get_id_for_a_given_name(span_dict, span_name_to_test) + + assert span_name_to_test == span_dict.get(span_id)["name"] + + # The span isn't a root span. + assert span_id not in root_span_dict.keys() + assert span_id in span_dict.keys() + + expected_child_span_names = ["task_1_sub_span"] + actual_child_span_names = [] + + task_1_child_spans = get_child_list_for_non_root(span_dict, "task_1") + + for span in task_1_child_spans: + actual_child_span_names.append(span["name"]) + + assert sorted(actual_child_span_names) == sorted(expected_child_span_names) From cf290d6dfb5798f892b96938c7ca70c45401f067 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Mon, 2 Dec 2024 19:20:09 +0200 Subject: [PATCH 11/52] cleanup old dagrun and ti spans --- airflow/jobs/scheduler_job_runner.py | 30 ------- airflow/models/dagrun.py | 4 - airflow/models/taskinstance.py | 22 ----- airflow/traces/otel_tracer.py | 130 +++------------------------ airflow/traces/tracer.py | 48 ---------- tests/core/test_otel_tracer.py | 74 +++------------ 6 files changed, 25 insertions(+), 283 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 71138c383aeb0..9f8495538b6e7 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -823,41 +823,11 @@ def process_executor_events( cls.active_ti_spans.delete(ti.key) ti.set_span_status(status=SpanStatus.ENDED, session=session) else: - # TODO: check if this is needed in case the task is adopted or - # finished not by the executor that started it. if ti.span_status == SpanStatus.ACTIVE: # Another scheduler has started the span. # Update the SpanStatus to let the process know that it must end it. ti.set_span_status(status=SpanStatus.SHOULD_END, session=session) - with Trace.start_span_from_taskinstance(ti=ti) as span: - cls._set_span_attrs__process_executor_events(span, state, ti) - if conf.has_option("traces", "otel_task_log_event") and conf.getboolean( - "traces", "otel_task_log_event" - ): - from airflow.utils.log.log_reader import TaskLogReader - - task_log_reader = TaskLogReader() - if task_log_reader.supports_read: - metadata: dict[str, Any] = {} - logs, metadata = task_log_reader.read_log_chunks(ti, ti.try_number, metadata) - if ti.hostname in dict(logs[0]): - message = str(dict(logs[0])[ti.hostname]).replace("\\n", "\n") - while metadata["end_of_log"] is False: - logs, metadata = task_log_reader.read_log_chunks( - ti, ti.try_number - 1, metadata - ) - if ti.hostname in dict(logs[0]): - message = message + str(dict(logs[0])[ti.hostname]).replace("\\n", "\n") - if span.is_recording(): - span.add_event( - name="task_log", - attributes={ - "message": message, - "metadata": str(metadata), - }, - ) - # There are two scenarios why the same TI with the same try_number is queued # after executor is finished with it: # 1) the TI was killed externally and it had no time to mark itself failed diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index f991ce5507507..16ad6055dd03e 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -1216,10 +1216,6 @@ def recalculate(self) -> _UnfinishedStates: self.state, ) - with Trace.start_span_from_dagrun(dagrun=self) as span: - if span is not None: # To avoid a static-code check error. - self.set_dagrun_span_attrs(span=span, dag_run=self, dagv=dagv) - session.flush() self._emit_true_scheduling_delay_stats_for_finished_state(finished_tis) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 19e41ce053f8e..d72be0e702a17 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -110,7 +110,6 @@ from airflow.templates import SandboxedEnvironment from airflow.ti_deps.dep_context import DepContext from airflow.ti_deps.dependencies_deps import REQUEUEABLE_DEPS, RUNNING_DEPS -from airflow.traces.tracer import Trace from airflow.utils import timezone from airflow.utils.context import ( ConnectionAccessor, @@ -1266,27 +1265,6 @@ def _handle_failure( if not test_mode: TaskInstance.save_to_db(failure_context["ti"], session) - with Trace.start_span_from_taskinstance(ti=task_instance) as span: - # ---- error info ---- - span.set_attribute("error", "true") - span.set_attribute("error_msg", str(error)) - span.set_attribute("context", context) - span.set_attribute("force_fail", force_fail) - # ---- common info ---- - span.set_attribute("category", "DAG runs") - span.set_attribute("task_id", task_instance.task_id) - span.set_attribute("dag_id", task_instance.dag_id) - span.set_attribute("state", task_instance.state) - span.set_attribute("start_date", str(task_instance.start_date)) - span.set_attribute("end_date", str(task_instance.end_date)) - span.set_attribute("duration", task_instance.duration) - span.set_attribute("executor_config", str(task_instance.executor_config)) - span.set_attribute("execution_date", str(task_instance.execution_date)) - span.set_attribute("hostname", task_instance.hostname) - if isinstance(task_instance, TaskInstance): - span.set_attribute("log_url", task_instance.log_url) - span.set_attribute("operator", str(task_instance.operator)) - def _refresh_from_task( *, task_instance: TaskInstance | TaskInstancePydantic, task: Operator, pool_override: str | None = None diff --git a/airflow/traces/otel_tracer.py b/airflow/traces/otel_tracer.py index 7d6889c77eda2..5d8eb1e58b8b4 100644 --- a/airflow/traces/otel_tracer.py +++ b/airflow/traces/otel_tracer.py @@ -26,7 +26,7 @@ from opentelemetry.context import attach, create_key from opentelemetry.exporter.otlp.proto.http.trace_exporter import OTLPSpanExporter from opentelemetry.sdk.resources import HOST_NAME, SERVICE_NAME, Resource -from opentelemetry.sdk.trace import Span, Tracer as OpenTelemetryTracer, TracerProvider +from opentelemetry.sdk.trace import Span, SpanProcessor, Tracer as OpenTelemetryTracer, TracerProvider from opentelemetry.sdk.trace.export import BatchSpanProcessor, ConsoleSpanExporter, SimpleSpanProcessor from opentelemetry.sdk.trace.id_generator import IdGenerator from opentelemetry.trace import Link, NonRecordingSpan, SpanContext, TraceFlags, Tracer @@ -34,14 +34,7 @@ from opentelemetry.trace.span import INVALID_SPAN_ID, INVALID_TRACE_ID from airflow.configuration import conf -from airflow.traces import ( - TRACEPARENT, - TRACESTATE, -) from airflow.traces.utils import ( - gen_dag_span_id, - gen_span_id, - gen_trace_id, parse_traceparent, parse_tracestate, ) @@ -78,7 +71,7 @@ def __init__( # A task can run fast and finish before spans have enough time to get exported to the collector. # When creating spans from inside a task, a SimpleSpanProcessor needs to be used because # it exports the spans immediately after they are created. - self.span_processor = SimpleSpanProcessor(self.span_exporter) + self.span_processor: SpanProcessor = SimpleSpanProcessor(self.span_exporter) else: self.span_processor = BatchSpanProcessor(self.span_exporter) self.tag_string = tag_string @@ -107,7 +100,7 @@ def get_otel_tracer_provider( if debug is True: log.info("[ConsoleSpanExporter] is being used") if self.use_simple_processor: - span_processor_for_tracer_prov = SimpleSpanProcessor(ConsoleSpanExporter()) + span_processor_for_tracer_prov: SpanProcessor = SimpleSpanProcessor(ConsoleSpanExporter()) else: span_processor_for_tracer_prov = BatchSpanProcessor(ConsoleSpanExporter()) else: @@ -170,107 +163,6 @@ def start_span( ) return span - def start_span_from_dagrun( - self, dagrun, span_name: str | None = None, component: str = "dagrun", links=None - ): - """Produce a span from dag run.""" - # check if dagrun has configs - conf = dagrun.conf - trace_id = int(gen_trace_id(dag_run=dagrun, as_int=True)) - span_id = int(gen_dag_span_id(dag_run=dagrun, as_int=True)) - - tracer = self.get_tracer(component=component, span_id=span_id, trace_id=trace_id) - - tag_string = self.tag_string if self.tag_string else "" - tag_string = tag_string + ("," + conf.get(TRACESTATE) if (conf and conf.get(TRACESTATE)) else "") - - if span_name is None: - span_name = dagrun.dag_id - - _links = gen_links_from_kv_list(links) if links else [] - - _links.append( - Link( - context=trace.get_current_span().get_span_context(), - attributes={"meta.annotation_type": "link", "from": "parenttrace"}, - ) - ) - - if conf and conf.get(TRACEPARENT): - # add the trace parent as the link - _links.append(gen_link_from_traceparent(conf.get(TRACEPARENT))) - - span_ctx = SpanContext( - trace_id=INVALID_TRACE_ID, span_id=INVALID_SPAN_ID, is_remote=True, trace_flags=TraceFlags(0x01) - ) - ctx = trace.set_span_in_context(NonRecordingSpan(span_ctx)) - span = tracer.start_as_current_span( - name=span_name, - context=ctx, - links=_links, - start_time=datetime_to_nano(dagrun.queued_at), - attributes=parse_tracestate(tag_string), - ) - return span - - def start_span_from_taskinstance( - self, - ti, - span_name: str | None = None, - component: str = "taskinstance", - child: bool = False, - links=None, - ): - """ - Create and start span from given task instance. - - Essentially the span represents the ti itself if child == True, it will create a 'child' span under the given span. - """ - dagrun = ti.dag_run - conf = dagrun.conf - trace_id = int(gen_trace_id(dag_run=dagrun, as_int=True)) - span_id = int(gen_span_id(ti=ti, as_int=True)) - if span_name is None: - span_name = ti.task_id - - parent_id = span_id if child else int(gen_dag_span_id(dag_run=dagrun, as_int=True)) - - span_ctx = SpanContext( - trace_id=trace_id, span_id=parent_id, is_remote=True, trace_flags=TraceFlags(0x01) - ) - - _links = gen_links_from_kv_list(links) if links else [] - - _links.append( - Link( - context=SpanContext( - trace_id=trace.get_current_span().get_span_context().trace_id, - span_id=span_id, - is_remote=True, - trace_flags=TraceFlags(0x01), - ), - attributes={"meta.annotation_type": "link", "from": "parenttrace"}, - ) - ) - - if child is False: - tracer = self.get_tracer(component=component, span_id=span_id, trace_id=trace_id) - else: - tracer = self.get_tracer(component=component) - - tag_string = self.tag_string if self.tag_string else "" - tag_string = tag_string + ("," + conf.get(TRACESTATE) if (conf and conf.get(TRACESTATE)) else "") - - ctx = trace.set_span_in_context(NonRecordingSpan(span_ctx)) - span = tracer.start_as_current_span( - name=span_name, - context=ctx, - links=_links, - start_time=datetime_to_nano(ti.queued_dttm), - attributes=parse_tracestate(tag_string), - ) - return span - def start_root_span( self, span_name: str, @@ -327,12 +219,14 @@ def start_child_span( _links = [] else: _links = links - _links.append( - Link( - context=parent_span_context, - attributes={"meta.annotation_type": "link", "from": "parenttrace"}, + + if parent_span_context is not None: + _links.append( + Link( + context=parent_span_context, + attributes={"meta.annotation_type": "link", "from": "parenttrace"}, + ) ) - ) return self._new_span( span_name=span_name, @@ -386,7 +280,7 @@ def _new_span( def inject(self) -> dict: """Inject the current span context into a carrier and return it.""" - carrier = {} + carrier: dict[str, str] = {} TraceContextTextMapPropagator().inject(carrier) return carrier @@ -428,7 +322,7 @@ def gen_link_from_traceparent(traceparent: str): return Link(context=span_ctx, attributes={"meta.annotation_type": "link", "from": "traceparent"}) -def get_otel_tracer(cls, use_simple_processor: bool | None = None) -> OtelTrace: +def get_otel_tracer(cls, use_simple_processor: bool = False) -> OtelTrace: """Get OTEL tracer from airflow configuration.""" host = conf.get("traces", "otel_host") port = conf.getint("traces", "otel_port") diff --git a/airflow/traces/tracer.py b/airflow/traces/tracer.py index e726b2fdd2307..7ad3e1a0316a6 100644 --- a/airflow/traces/tracer.py +++ b/airflow/traces/tracer.py @@ -159,30 +159,6 @@ def use_span(cls, span): def get_current_span(self): raise NotImplementedError() - @classmethod - def start_span_from_dagrun( - cls, - dagrun, - span_name=None, - service_name=None, - component=None, - links=None, - ): - """Start a span from dagrun.""" - raise NotImplementedError() - - @classmethod - def start_span_from_taskinstance( - cls, - ti, - span_name=None, - component=None, - child=False, - links=None, - ): - """Start a span from taskinstance.""" - raise NotImplementedError() - @classmethod def start_root_span(cls, span_name=None, component=None, start_time=None, start_as_current=True): """Start a root span.""" @@ -248,30 +224,6 @@ def get_current_span(self) -> EmptySpan: """Get the current span.""" return EMPTY_SPAN - @classmethod - def start_span_from_dagrun( - cls, - dagrun, - span_name=None, - service_name=None, - component=None, - links=None, - ) -> EmptySpan: - """Start a span from dagrun.""" - return EMPTY_SPAN - - @classmethod - def start_span_from_taskinstance( - cls, - ti, - span_name=None, - component=None, - child=False, - links=None, - ) -> EmptySpan: - """Start a span from taskinstance.""" - return EMPTY_SPAN - @classmethod def start_root_span( cls, span_name=None, component=None, start_time=None, start_as_current=True diff --git a/tests/core/test_otel_tracer.py b/tests/core/test_otel_tracer.py index 6ab7a568a2e4b..57709f284ce74 100644 --- a/tests/core/test_otel_tracer.py +++ b/tests/core/test_otel_tracer.py @@ -19,13 +19,15 @@ import json import logging from datetime import datetime -from unittest.mock import MagicMock, patch +from unittest.mock import patch import pytest +from opentelemetry.sdk import util from opentelemetry.sdk.trace.export.in_memory_span_exporter import InMemorySpanExporter -from airflow.traces import TRACEPARENT, TRACESTATE, otel_tracer, utils +from airflow.traces import otel_tracer from airflow.traces.tracer import Trace +from airflow.utils.dates import datetime_to_nano from tests_common.test_utils.config import env_vars @@ -90,71 +92,21 @@ def test_dag_tracer(self, conf_a, exporter): exporter.return_value = in_mem_exporter now = datetime.now() - dag_run = MagicMock() - - parent_trace_id = "0af7651916cd43dd8448eb211c80319c" - parent_span_id = "b7ad6b7169203331" - - dag_run.conf = { - TRACEPARENT: f"00-{parent_trace_id}-{parent_span_id}-01", - TRACESTATE: "key1=val1,key2=val2", - } - dag_run.dag_id = "dag_id" - dag_run.run_id = "run_id" - dag_run.dag_hash = "hashcode" - dag_run.run_type = "manual" - dag_run.queued_at = now - dag_run.start_date = now tracer = otel_tracer.get_otel_tracer(Trace) - with tracer.start_span_from_dagrun(dagrun=dag_run) as s1: - with tracer.start_span(span_name="span2") as s2: - s2.set_attribute("attr2", "val2") - span1 = json.loads(s1.to_json()) - assert span1["context"]["trace_id"] != f"0x{parent_trace_id}" - assert span1["links"][1]["context"]["trace_id"] == f"0x{parent_trace_id}" - assert span1["links"][1]["context"]["span_id"] == f"0x{parent_span_id}" - - @patch("opentelemetry.sdk.trace.export.ConsoleSpanExporter") - @patch("airflow.traces.otel_tracer.conf") - def test_traskinstance_tracer(self, conf_a, exporter): - # necessary to speed up the span to be emitted - with env_vars({"OTEL_BSP_SCHEDULE_DELAY": "1"}): - log = logging.getLogger("TestOtelTrace.test_taskinstance_tracer") - log.setLevel(logging.DEBUG) - conf_a.get.return_value = "abc" - conf_a.getint.return_value = 123 - # this will enable debug to set - which outputs the result to console - conf_a.getboolean.return_value = True - - # mocking console exporter with in mem exporter for better assertion - in_mem_exporter = InMemorySpanExporter() - exporter.return_value = in_mem_exporter - - now = datetime.now() - # magic mock - ti = MagicMock() - ti.dag_run.conf = {} - ti.task_id = "task_id" - ti.start_date = now - ti.dag_run.dag_id = "dag_id" - ti.dag_run.run_id = "run_id" - ti.dag_run.dag_hash = "hashcode" - ti.dag_run.run_type = "manual" - ti.dag_run.queued_at = now - ti.dag_run.start_date = now - - tracer = otel_tracer.get_otel_tracer(Trace) - with tracer.start_span_from_taskinstance(ti=ti, span_name="mydag") as s1: + with tracer.start_root_span(span_name="span1", start_time=now) as s1: with tracer.start_span(span_name="span2") as s2: s2.set_attribute("attr2", "val2") span2 = json.loads(s2.to_json()) span1 = json.loads(s1.to_json()) - log.info(span1) - log.info(span2) - assert span1["context"]["trace_id"] == f"0x{utils.gen_trace_id(ti.dag_run)}" - assert span1["context"]["span_id"] == f"0x{utils.gen_span_id(ti)}" + # The otel sdk, accepts an int for the start_time, and converts it to an iso string, + # using `util.ns_to_iso_str()`. + nano_time = datetime_to_nano(now) + assert span1["start_time"] == util.ns_to_iso_str(nano_time) + # Same trace_id + assert span1["context"]["trace_id"] == span2["context"]["trace_id"] + assert span1["context"]["span_id"] == span2["parent_id"] @patch("opentelemetry.sdk.trace.export.ConsoleSpanExporter") @patch("airflow.traces.otel_tracer.conf") @@ -177,7 +129,7 @@ def test_context_propagation(self, conf_a, exporter): # - extracting the context # - using the context to create a new span # The new span should be associated with the span from the injected context carrier. - def _task_func(otel_tr: Trace, carrier: dict): + def _task_func(otel_tr, carrier): parent_context = otel_tr.extract(carrier) with otel_tr.start_child_span(span_name="sub_span", parent_context=parent_context) as span: From 691865bfdbd556d8a216f531a2b91d6b9f4b3147 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Mon, 2 Dec 2024 19:25:23 +0200 Subject: [PATCH 12/52] remove CTX_PROP_SUFFIX --- airflow/executors/base_executor.py | 5 +- airflow/jobs/scheduler_job_runner.py | 5 +- airflow/models/dagrun.py | 11 ++-- tests/integration/otel/dags/otel_test_dag.py | 13 ++--- .../otel/dags/otel_test_dag_with_pause.py | 13 ++--- tests/integration/otel/test_otel.py | 55 +++++++++---------- 6 files changed, 48 insertions(+), 54 deletions(-) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index e696f3a7ea27c..3d1acc320b282 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -33,7 +33,6 @@ from airflow.models import Log from airflow.stats import Stats from airflow.traces import NO_TRACE_ID -from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import Trace, add_span, gen_context from airflow.traces.utils import gen_span_id_from_ti_key, gen_trace_id from airflow.utils.log.logging_mixin import LoggingMixin @@ -349,9 +348,9 @@ def trigger_tasks(self, open_slots: int) -> None: # Attributes will be set once the task has finished so that all # values will be available (end_time, duration, etc.). span = Trace.start_child_span( - span_name=f"{ti.task_id}{CTX_PROP_SUFFIX}", + span_name=f"{ti.task_id}", parent_context=parent_context, - component=f"task{CTX_PROP_SUFFIX}", + component="task", start_time=ti.queued_dttm, start_as_current=False, ) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 9f8495538b6e7..0ca6409614b7d 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -63,7 +63,6 @@ from airflow.ti_deps.dependencies_states import EXECUTION_STATES from airflow.timetables.simple import AssetTriggeredTimetable from airflow.traces import utils as trace_utils -from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import Trace, add_span from airflow.utils import timezone from airflow.utils.dates import datetime_to_nano @@ -1137,7 +1136,7 @@ def _recreate_dead_scheduler_spans_if_needed(self, dag_run: DagRun, session: Ses if not is_healthy: # Start a new span for the dag_run. dr_span = Trace.start_root_span( - span_name=f"{dr.dag_id}_recreated{CTX_PROP_SUFFIX}", + span_name=f"{dr.dag_id}_recreated", component="dag", start_time=dr.queued_at, start_as_current=False, @@ -1161,7 +1160,7 @@ def _recreate_dead_scheduler_spans_if_needed(self, dag_run: DagRun, session: Ses dr_context = Trace.extract(dr.context_carrier) for ti in tis_needing_spans: ti_span = Trace.start_child_span( - span_name=f"{ti.task_id}_recreated{CTX_PROP_SUFFIX}", + span_name=f"{ti.task_id}_recreated", parent_context=dr_context, start_time=ti.queued_dttm, start_as_current=False, diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 16ad6055dd03e..86087d492157a 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -79,7 +79,6 @@ from airflow.stats import Stats from airflow.ti_deps.dep_context import DepContext from airflow.ti_deps.dependencies_states import SCHEDULEABLE_STATES -from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import EmptySpan, Trace from airflow.utils import timezone from airflow.utils.dates import datetime_to_nano @@ -1087,8 +1086,8 @@ def recalculate(self) -> _UnfinishedStates: continue_ti_spans = False if self.span_status == SpanStatus.NOT_STARTED: dr_span = Trace.start_root_span( - span_name=f"{self.dag_id}{CTX_PROP_SUFFIX}", - component=f"dag{CTX_PROP_SUFFIX}", + span_name=f"{self.dag_id}", + component="dag", start_time=self.queued_at, # This is later converted to nano. start_as_current=False, ) @@ -1101,9 +1100,9 @@ def recalculate(self) -> _UnfinishedStates: s.set_attribute("trace_status", "continued") dr_span = Trace.start_child_span( - span_name=f"{self.dag_id}_continued{CTX_PROP_SUFFIX}", + span_name=f"{self.dag_id}_continued", parent_context=parent_context, - component=f"dag{CTX_PROP_SUFFIX}", + component="dag", # No start time start_as_current=False, ) @@ -1125,7 +1124,7 @@ def recalculate(self) -> _UnfinishedStates: for ti in tis: if ti.span_status == SpanStatus.NEEDS_CONTINUANCE: ti_span = Trace.start_child_span( - span_name=f"{ti.task_id}_continued{CTX_PROP_SUFFIX}", + span_name=f"{ti.task_id}_continued", parent_context=new_dagrun_context, start_as_current=False, ) diff --git a/tests/integration/otel/dags/otel_test_dag.py b/tests/integration/otel/dags/otel_test_dag.py index 4af68fc799ea9..3be234f7eb7e3 100644 --- a/tests/integration/otel/dags/otel_test_dag.py +++ b/tests/integration/otel/dags/otel_test_dag.py @@ -24,7 +24,6 @@ from airflow import DAG from airflow.providers.standard.operators.python import PythonOperator from airflow.traces import otel_tracer -from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import Trace logger = logging.getLogger("airflow.otel_test_dag") @@ -57,26 +56,26 @@ def task1_func(**dag_context): logger.info("Extracting the span context from the context_carrier.") parent_context = Trace.extract(context_carrier) with otel_task_tracer.start_child_span( - span_name=f"{ti.task_id}_sub_span1{CTX_PROP_SUFFIX}", + span_name=f"{ti.task_id}_sub_span1", parent_context=parent_context, - component=f"dag{CTX_PROP_SUFFIX}", + component="dag", ) as s1: s1.set_attribute("attr1", "val1") logger.info("From task sub_span1.") - with otel_task_tracer.start_child_span(f"{ti.task_id}_sub_span2{CTX_PROP_SUFFIX}") as s2: + with otel_task_tracer.start_child_span(f"{ti.task_id}_sub_span2") as s2: s2.set_attribute("attr2", "val2") logger.info("From task sub_span2.") tracer = trace.get_tracer("trace_test.tracer", tracer_provider=tracer_provider) - with tracer.start_as_current_span(name=f"{ti.task_id}_sub_span3{CTX_PROP_SUFFIX}") as s3: + with tracer.start_as_current_span(name=f"{ti.task_id}_sub_span3") as s3: s3.set_attribute("attr3", "val3") logger.info("From task sub_span3.") with otel_task_tracer.start_child_span( - span_name=f"{ti.task_id}_sub_span4{CTX_PROP_SUFFIX}", + span_name=f"{ti.task_id}_sub_span4", parent_context=parent_context, - component=f"dag{CTX_PROP_SUFFIX}", + component="dag", ) as s4: s4.set_attribute("attr4", "val4") logger.info("From task sub_span4.") diff --git a/tests/integration/otel/dags/otel_test_dag_with_pause.py b/tests/integration/otel/dags/otel_test_dag_with_pause.py index fc3a0205f00c5..e02d7ef3520bf 100644 --- a/tests/integration/otel/dags/otel_test_dag_with_pause.py +++ b/tests/integration/otel/dags/otel_test_dag_with_pause.py @@ -27,7 +27,6 @@ from airflow.models import TaskInstance from airflow.providers.standard.operators.python import PythonOperator from airflow.traces import otel_tracer -from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.traces.tracer import Trace from airflow.utils.session import create_session @@ -95,19 +94,19 @@ def task1_func(**dag_context): parent_context = Trace.extract(context_carrier) with otel_task_tracer.start_child_span( - span_name=f"{ti.task_id}_sub_span1{CTX_PROP_SUFFIX}", + span_name=f"{ti.task_id}_sub_span1", parent_context=parent_context, - component=f"dag{CTX_PROP_SUFFIX}", + component="dag", ) as s1: s1.set_attribute("attr1", "val1") logger.info("From task sub_span1.") - with otel_task_tracer.start_child_span(f"{ti.task_id}_sub_span2{CTX_PROP_SUFFIX}") as s2: + with otel_task_tracer.start_child_span(f"{ti.task_id}_sub_span2") as s2: s2.set_attribute("attr2", "val2") logger.info("From task sub_span2.") tracer = trace.get_tracer("trace_test.tracer", tracer_provider=tracer_provider) - with tracer.start_as_current_span(name=f"{ti.task_id}_sub_span3{CTX_PROP_SUFFIX}") as s3: + with tracer.start_as_current_span(name=f"{ti.task_id}_sub_span3") as s3: s3.set_attribute("attr3", "val3") logger.info("From task sub_span3.") @@ -122,9 +121,9 @@ def task1_func(**dag_context): parent_context = Trace.extract(context_carrier) with otel_task_tracer.start_child_span( - span_name=f"{ti.task_id}_sub_span4{CTX_PROP_SUFFIX}", + span_name=f"{ti.task_id}_sub_span4", parent_context=parent_context, - component=f"dag{CTX_PROP_SUFFIX}", + component="dag", ) as s4: s4.set_attribute("attr4", "val4") logger.info("From task sub_span4.") diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index ccbb7faaaa2a7..21fc5f2a8edb1 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -30,7 +30,6 @@ from airflow.models import DAG, DagBag, DagRun from airflow.models.serialized_dag import SerializedDagModel from airflow.models.taskinstance import TaskInstance -from airflow.traces.otel_tracer import CTX_PROP_SUFFIX from airflow.utils.session import create_session from airflow.utils.span_status import SpanStatus from airflow.utils.state import State @@ -188,33 +187,33 @@ def check_spans_with_continuance(output: str, dag: DAG, continuance_for_t1: bool task1_id = task_instance_ids[0] task2_id = task_instance_ids[1] - dag_root_span_name = f"{dag_id}{CTX_PROP_SUFFIX}" + dag_root_span_name = f"{dag_id}" dag_root_span_children_names = [ - f"{task1_id}{CTX_PROP_SUFFIX}", + f"{task1_id}", "current_scheduler_exited", "new_scheduler", - f"{dag_id}_continued{CTX_PROP_SUFFIX}", + f"{dag_id}_continued", ] if continuance_for_t1: dag_continued_span_children_names = [ - f"{task1_id}_continued{CTX_PROP_SUFFIX}", - f"{task2_id}{CTX_PROP_SUFFIX}", + f"{task1_id}_continued", + f"{task2_id}", ] else: dag_continued_span_children_names = [ - f"{task2_id}{CTX_PROP_SUFFIX}", + f"{task2_id}", ] task1_span_children_names = [ - f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", - f"{task1_id}_sub_span4{CTX_PROP_SUFFIX}", + f"{task1_id}_sub_span1", + f"{task1_id}_sub_span4", ] # Single element lists. - task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}"] - task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3{CTX_PROP_SUFFIX}"] + task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2"] + task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3"] assert_span_name_belongs_to_root_span( root_span_dict=root_span_dict, span_name=dag_root_span_name, should_succeed=True @@ -234,7 +233,7 @@ def check_spans_with_continuance(output: str, dag: DAG, continuance_for_t1: bool root_span_dict=root_span_dict, span_dict=span_dict, parent_name=dag_root_span_name, - child_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", + child_name=f"{task1_id}_continued", span_exists=True, ) @@ -251,7 +250,7 @@ def check_spans_with_continuance(output: str, dag: DAG, continuance_for_t1: bool # Check children of the continued dag span. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{dag_id}_continued{CTX_PROP_SUFFIX}", + parent_name=f"{dag_id}_continued", children_names=dag_continued_span_children_names, ) @@ -259,28 +258,28 @@ def check_spans_with_continuance(output: str, dag: DAG, continuance_for_t1: bool # Check children of the continued task1 span. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{task1_id}_continued{CTX_PROP_SUFFIX}", + parent_name=f"{task1_id}_continued", children_names=task1_span_children_names, ) else: # Check children of the task1 span. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{task1_id}{CTX_PROP_SUFFIX}", + parent_name=f"{task1_id}", children_names=task1_span_children_names, ) # Check children of task1 sub span1. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + parent_name=f"{task1_id}_sub_span1", children_names=task1_sub_span1_children_span_names, ) # Check children of task1 sub span2. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", + parent_name=f"{task1_id}_sub_span2", children_names=task1_sub_span2_children_span_names, ) @@ -325,21 +324,21 @@ def check_spans_without_continuance( # Based on the current tests, only the root span and the task1 span will be recreated. # TODO: Adjust accordingly, if there are more tests in the future # that require other spans to be recreated as well. - dag_root_span_name = f"{dag_id}{recreated_suffix}{CTX_PROP_SUFFIX}" + dag_root_span_name = f"{dag_id}{recreated_suffix}" dag_root_span_children_names = [ - f"{task1_id}{recreated_suffix}{CTX_PROP_SUFFIX}", - f"{task2_id}{CTX_PROP_SUFFIX}", + f"{task1_id}{recreated_suffix}", + f"{task2_id}", ] task1_span_children_names = [ - f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", - f"{task1_id}_sub_span4{CTX_PROP_SUFFIX}", + f"{task1_id}_sub_span1", + f"{task1_id}_sub_span4", ] # Single element lists. - task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}"] - task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3{CTX_PROP_SUFFIX}"] + task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2"] + task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3"] assert_span_name_belongs_to_root_span( root_span_dict=root_span_dict, span_name=dag_root_span_name, should_succeed=True @@ -359,7 +358,7 @@ def check_spans_without_continuance( root_span_dict=root_span_dict, span_dict=span_dict, parent_name=dag_root_span_name, - child_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + child_name=f"{task1_id}_sub_span1", span_exists=True, ) @@ -377,21 +376,21 @@ def check_spans_without_continuance( # Check children of the task1 span. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{task1_id}{recreated_suffix}{CTX_PROP_SUFFIX}", + parent_name=f"{task1_id}{recreated_suffix}", children_names=task1_span_children_names, ) # Check children of task1 sub span1. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{task1_id}_sub_span1{CTX_PROP_SUFFIX}", + parent_name=f"{task1_id}_sub_span1", children_names=task1_sub_span1_children_span_names, ) # Check children of task1 sub span2. assert_parent_children_spans_for_non_root( span_dict=span_dict, - parent_name=f"{task1_id}_sub_span2{CTX_PROP_SUFFIX}", + parent_name=f"{task1_id}_sub_span2", children_names=task1_sub_span2_children_span_names, ) From 87d6efb91d8305f8e1e49fcc79ee5fef5a0ae304 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Mon, 2 Dec 2024 19:43:36 +0200 Subject: [PATCH 13/52] cleaning up comments --- airflow/jobs/scheduler_job_runner.py | 12 ++---------- airflow/models/taskinstance.py | 4 ++-- airflow/traces/tracer.py | 23 ----------------------- 3 files changed, 4 insertions(+), 35 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 0ca6409614b7d..c72c19a6de943 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -806,13 +806,6 @@ def process_executor_events( ti.pid, ) - # Each scheduler processes its own executor events. - # If the executor of the scheduler has run the task, - # then only that scheduler will mark the task as finished - # unless the process dies and the task has to be adopted by another scheduler. - # There is no point in notifying another scheduler that the task span has to be ended. - # But because the span might not be ended immediately, - # the task end time must be set as the span end time. active_ti_span = cls.active_ti_spans.get(ti.key) if conf.getboolean("traces", "otel_use_context_propagation"): if active_ti_span is not None: @@ -1066,8 +1059,8 @@ def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): # This also means that the process that started them, is the only one that can end them. # # If another scheduler has finished processing a dag_run or a task and there is a reference - # on the active_spans dictionary, then that means that the current scheduler started - # the span, and therefore must end it. + # on the active_spans dictionary, then the current scheduler started the span, + # and therefore must end it. dag_runs_should_end: list[DagRun] = session.scalars( select(DagRun).where(DagRun.span_status == SpanStatus.SHOULD_END) ).all() @@ -1172,7 +1165,6 @@ def _recreate_dead_scheduler_spans_if_needed(self, dag_run: DagRun, session: Ses ti_span.end(end_time=datetime_to_nano(ti.end_date)) ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: - # TODO: revisit this. Will the current scheduler end the span? ti.set_span_status(status=SpanStatus.ACTIVE, session=session, with_commit=False) self.active_ti_spans.set(ti.key, ti_span) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index d72be0e702a17..1f1704b1b7df2 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -901,7 +901,7 @@ def _refresh_from_db( if not inspector.detached and "task_instance" not in inspector.unloaded: # If the scheduler that started the dag_run has exited (gracefully or forcefully), # there will be changes to the dag_run span context_carrier. - # It's best to include the dag_run whenever possible, so that the ti contains the updates. + # It's best to include the dag_run whenever possible, so that the ti will contain the updates. include_dag_run = True _set_ti_attrs(task_instance, ti, include_dag_run=include_dag_run) @@ -2447,7 +2447,7 @@ def set_span_status( """ Set TaskInstance span_status. - :param status: dict with the injected carrier to set for the dag_run + :param status: dict with the injected carrier to set for the ti :param session: SQLAlchemy ORM Session :param with_commit: should the status be committed? :return: has the span_status been changed? diff --git a/airflow/traces/tracer.py b/airflow/traces/tracer.py index 7ad3e1a0316a6..47f7fe8d58366 100644 --- a/airflow/traces/tracer.py +++ b/airflow/traces/tracer.py @@ -22,8 +22,6 @@ import socket from typing import TYPE_CHECKING, Any, Callable -from sqlalchemy.util.compat import contextmanager - from airflow.configuration import conf from airflow.typing_compat import Protocol @@ -278,27 +276,6 @@ def __init__(cls, *args, **kwargs) -> None: else: cls.__class__.factory = EmptyTrace - @classmethod - @contextmanager - def tracer_with_params(cls, *args, **kwargs): - """Context manager to temporarily set parameters for the tracer factory.""" - initial_factory = cls.factory - initial_instance = cls.instance - try: - if conf.has_option("traces", "otel_on") and conf.getboolean("traces", "otel_on"): - from airflow.traces import otel_tracer - - cls.factory = lambda: otel_tracer.get_otel_tracer(*args, **kwargs) - else: - cls.factory = EmptyTrace - # Reset the instance to ensure the new parameters are used - cls.instance = None - yield - finally: - # Restore the factory and the instance, to their initial values. - cls.factory = initial_factory - cls.instance = initial_instance - @classmethod def get_constant_tags(cls) -> str | None: """Get constant tags to add to all traces.""" From 8329733116939225a83b501eb4a8ea51ddcd3318 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Mon, 2 Dec 2024 19:48:57 +0200 Subject: [PATCH 14/52] remove context propagation config flag --- airflow/config_templates/config.yml | 10 --- airflow/executors/base_executor.py | 49 ++++++------- airflow/jobs/scheduler_job_runner.py | 23 +++--- airflow/models/dagrun.py | 71 ++++++++----------- .../logging-monitoring/traces.rst | 1 - tests/integration/otel/test_otel.py | 2 - 6 files changed, 65 insertions(+), 91 deletions(-) diff --git a/airflow/config_templates/config.yml b/airflow/config_templates/config.yml index 35fbb2a60125d..4d58de3c37e28 100644 --- a/airflow/config_templates/config.yml +++ b/airflow/config_templates/config.yml @@ -1291,16 +1291,6 @@ traces: type: string example: ~ default: "False" - otel_use_context_propagation: - description: | - If True, a separate trace with a new set of spans will be created for each dag run. - All the children spans will be created using context propagation. - This doesn't replace any of the existing spans, but it creates additional ones. - The default value is False. - version_added: 2.10.3 - type: string - example: ~ - default: "False" secrets: description: ~ options: diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index 3d1acc320b282..5b341c8c2b077 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -129,8 +129,6 @@ class BaseExecutor(LoggingMixin): name: None | ExecutorName = None callback_sink: BaseCallbackSink | None = None - otel_use_context_propagation = conf.getboolean("traces", "otel_use_context_propagation") - def __init__(self, parallelism: int = PARALLELISM): super().__init__() self.parallelism: int = parallelism @@ -340,30 +338,29 @@ def trigger_tasks(self, open_slots: int) -> None: for _ in range(min((open_slots, len(self.queued_tasks)))): key, (command, _, queue, ti) = sorted_queue.pop(0) - if self.otel_use_context_propagation: - # If it's None, then the span for the current TaskInstanceKey hasn't been started. - if self.active_spans is not None and self.active_spans.get(key) is None: - parent_context = Trace.extract(ti.dag_run.context_carrier) - # Start a new span using the context from the parent. - # Attributes will be set once the task has finished so that all - # values will be available (end_time, duration, etc.). - span = Trace.start_child_span( - span_name=f"{ti.task_id}", - parent_context=parent_context, - component="task", - start_time=ti.queued_dttm, - start_as_current=False, - ) - self.active_spans.set(key, span) - # Inject the current context into the carrier. - carrier = Trace.inject() - # The carrier needs to be set on the ti, but it can't happen here because db calls are expensive. - # By the time the db update has finished, another heartbeat will have started - # and the tasks will have been triggered again. - # So set the carrier as an argument to the command. - # The command execution will set it on the ti, and it will be propagated to the task itself. - command.append("--carrier") - command.append(json.dumps(carrier)) + # If it's None, then the span for the current TaskInstanceKey hasn't been started. + if self.active_spans is not None and self.active_spans.get(key) is None: + parent_context = Trace.extract(ti.dag_run.context_carrier) + # Start a new span using the context from the parent. + # Attributes will be set once the task has finished so that all + # values will be available (end_time, duration, etc.). + span = Trace.start_child_span( + span_name=f"{ti.task_id}", + parent_context=parent_context, + component="task", + start_time=ti.queued_dttm, + start_as_current=False, + ) + self.active_spans.set(key, span) + # Inject the current context into the carrier. + carrier = Trace.inject() + # The carrier needs to be set on the ti, but it can't happen here because db calls are expensive. + # By the time the db update has finished, another heartbeat will have started + # and the tasks will have been triggered again. + # So set the carrier as an argument to the command. + # The command execution will set it on the ti, and it will be propagated to the task itself. + command.append("--carrier") + command.append(json.dumps(carrier)) # If a task makes it here but is still understood by the executor # to be running, it generally means that the task has been killed diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index c72c19a6de943..eba8501ab1d1f 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -807,18 +807,17 @@ def process_executor_events( ) active_ti_span = cls.active_ti_spans.get(ti.key) - if conf.getboolean("traces", "otel_use_context_propagation"): - if active_ti_span is not None: - cls._set_span_attrs__process_executor_events(span=active_ti_span, state=state, ti=ti) - # End the span and remove it from the active_ti_spans dict. - active_ti_span.end(end_time=datetime_to_nano(ti.end_date)) - cls.active_ti_spans.delete(ti.key) - ti.set_span_status(status=SpanStatus.ENDED, session=session) - else: - if ti.span_status == SpanStatus.ACTIVE: - # Another scheduler has started the span. - # Update the SpanStatus to let the process know that it must end it. - ti.set_span_status(status=SpanStatus.SHOULD_END, session=session) + if active_ti_span is not None: + cls._set_span_attrs__process_executor_events(span=active_ti_span, state=state, ti=ti) + # End the span and remove it from the active_ti_spans dict. + active_ti_span.end(end_time=datetime_to_nano(ti.end_date)) + cls.active_ti_spans.delete(ti.key) + ti.set_span_status(status=SpanStatus.ENDED, session=session) + else: + if ti.span_status == SpanStatus.ACTIVE: + # Another scheduler has started the span. + # Update the SpanStatus to let the process know that it must end it. + ti.set_span_status(status=SpanStatus.SHOULD_END, session=session) # There are two scenarios why the same TI with the same try_number is queued # after executor is finished with it: diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 86087d492157a..0654ef857bb19 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -252,8 +252,6 @@ class DagRun(Base, LoggingMixin): fallback=20, ) - otel_use_context_propagation = airflow_conf.getboolean("traces", "otel_use_context_propagation") - def __init__( self, dag_id: str | None = None, @@ -1073,11 +1071,7 @@ def recalculate(self) -> _UnfinishedStates: # finally, if the leaves aren't done, the dag is still running else: # If there is no value in active_dagrun_spans, then the span hasn't already been started. - if ( - self.otel_use_context_propagation - and self.active_dagrun_spans is not None - and self.active_dagrun_spans.get(self.run_id) is None - ): + if self.active_dagrun_spans is not None and self.active_dagrun_spans.get(self.run_id) is None: if ( self.span_status == SpanStatus.NOT_STARTED or self.span_status == SpanStatus.NEEDS_CONTINUANCE @@ -1176,45 +1170,42 @@ def recalculate(self) -> _UnfinishedStates: dagv.version if dagv else None, ) - if self.otel_use_context_propagation: - if self.active_dagrun_spans is not None: - active_span = self.active_dagrun_spans.get(self.run_id) - if active_span is not None: + if self.active_dagrun_spans is not None: + active_span = self.active_dagrun_spans.get(self.run_id) + if active_span is not None: + self.log.debug( + "Found active span with span_id: %s, for dag_id: %s, run_id: %s, state: %s", + active_span.get_span_context().span_id, + self.dag_id, + self.run_id, + self.state, + ) + + self.set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) + active_span.end(end_time=datetime_to_nano(self.end_date)) + # Remove the span from the dict. + self.active_dagrun_spans.delete(self.run_id) + self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + else: + if self.span_status == SpanStatus.ACTIVE: + # Another scheduler has started the span. + # Update the DB SpanStatus to notify the owner to end it. + self.set_span_status(status=SpanStatus.SHOULD_END, session=session, with_commit=False) + elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: + # This is a corner case where the scheduler exited gracefully + # while the dag_run was almost done. + # Since it reached this point, the dag has finished but there has been no time + # to create a new span for the current scheduler. + # There is no need for more spans, update the status on the db. + self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + else: self.log.debug( - "Found active span with span_id: %s, for dag_id: %s, run_id: %s, state: %s", - active_span.get_span_context().span_id, + "No active span has been found for dag_id: %s, run_id: %s, state: %s", self.dag_id, self.run_id, self.state, ) - self.set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) - active_span.end(end_time=datetime_to_nano(self.end_date)) - # Remove the span from the dict. - self.active_dagrun_spans.delete(self.run_id) - self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) - else: - if self.span_status == SpanStatus.ACTIVE: - # Another scheduler has started the span. - # Update the DB SpanStatus to notify the owner to end it. - self.set_span_status( - status=SpanStatus.SHOULD_END, session=session, with_commit=False - ) - elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: - # This is a corner case where the scheduler exited gracefully - # while the dag_run was almost done. - # Since it reached this point, the dag has finished but there has been no time - # to create a new span for the current scheduler. - # There is no need for more spans, update the status on the db. - self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) - else: - self.log.debug( - "No active span has been found for dag_id: %s, run_id: %s, state: %s", - self.dag_id, - self.run_id, - self.state, - ) - session.flush() self._emit_true_scheduling_delay_stats_for_finished_state(finished_tis) diff --git a/docs/apache-airflow/administration-and-deployment/logging-monitoring/traces.rst b/docs/apache-airflow/administration-and-deployment/logging-monitoring/traces.rst index b1de367f48dba..93fe6e87b40bf 100644 --- a/docs/apache-airflow/administration-and-deployment/logging-monitoring/traces.rst +++ b/docs/apache-airflow/administration-and-deployment/logging-monitoring/traces.rst @@ -42,7 +42,6 @@ Add the following lines to your configuration file e.g. ``airflow.cfg`` otel_application = airflow otel_ssl_active = False otel_task_log_event = True - otel_use_context_propagation = True Enable Https ----------------- diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index 21fc5f2a8edb1..3cc1fd0c3ae9a 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -444,8 +444,6 @@ def setup_class(cls): if cls.use_otel != "true": os.environ["AIRFLOW__TRACES__OTEL_DEBUGGING_ON"] = "True" - os.environ["AIRFLOW__TRACES__OTEL_USE_CONTEXT_PROPAGATION"] = "True" - os.environ["AIRFLOW__SCHEDULER__STANDALONE_DAG_PROCESSOR"] = "False" os.environ["AIRFLOW__SCHEDULER__PROCESSOR_POLL_INTERVAL"] = "2" From 2ad643b6d22bf0cf487f67556f1c24aa64d1f7d9 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Mon, 2 Dec 2024 21:02:07 +0200 Subject: [PATCH 15/52] add attributes while cleaning up ended spans --- airflow/jobs/scheduler_job_runner.py | 4 ++++ tests/integration/otel/test_utils.py | 8 ++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index d01b059c5bc7b..8fe08c5bdb2c1 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -1053,6 +1053,9 @@ def _cleanup_active_spans_before_process_exit(self, session: Session = NEW_SESSI for run_id, span in self.active_dagrun_spans.get_all().items(): dag_run: DagRun = session.scalars(select(DagRun).where(DagRun.run_id == run_id)).one() if dag_run.state in State.finished_dr_states: + dagv = session.scalar(select(DagVersion).where(DagVersion.id == dag_run.dag_version_id)) + DagRun.set_dagrun_span_attrs(span=span, dag_run=dag_run, dagv=dagv) + span.end(end_time=datetime_to_nano(dag_run.end_date)) dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: @@ -1076,6 +1079,7 @@ def _cleanup_active_spans_before_process_exit(self, session: Session = NEW_SESSI ) ).one() if ti.state in State.finished: + self._set_span_attrs__process_executor_events(span=span, state=ti.state, ti=ti) span.end(end_time=datetime_to_nano(ti.end_date)) ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: diff --git a/tests/integration/otel/test_utils.py b/tests/integration/otel/test_utils.py index 8654327806016..69a0899987f44 100644 --- a/tests/integration/otel/test_utils.py +++ b/tests/integration/otel/test_utils.py @@ -58,7 +58,7 @@ def dump_airflow_metadata_db(session): def extract_spans_from_output(output_lines: list): """ - For a give list of ConsoleSpanExporter output lines, it extracts the json spans + For a given list of ConsoleSpanExporter output lines, it extracts the json spans and creates two dictionaries. :return: root spans dict (key: root_span_id - value: root_span), spans dict (key: span_id - value: span) """ @@ -278,7 +278,7 @@ class TestUtilsUnit: "attributes": { "airflow.category": "DAG runs", "airflow.dag_run.dag_id": "otel_test_dag_with_pause", - "airflow.dag_run.execution_date": "2024-11-30 14:01:15+00:00", + "airflow.dag_run.logical_date": "2024-11-30 14:01:15+00:00", "airflow.dag_run.run_id": "manual__2024-11-30T14:01:15.333003+00:00", "airflow.dag_run.queued_at": "2024-11-30 14:01:21.738052+00:00", "airflow.dag_run.run_start_date": "2024-11-30 14:01:22.192655+00:00", @@ -344,7 +344,7 @@ class TestUtilsUnit: "airflow.task.end_date": "2024-11-30 14:01:34.339423+00:00", "airflow.task.duration": 10.871376, "airflow.task.executor_config": "{}", - "airflow.task.execution_date": "2024-11-30 14:01:15+00:00", + "airflow.task.logical_date": "2024-11-30 14:01:15+00:00", "airflow.task.hostname": "351295342ba2", "airflow.task.log_url": "http://localhost:8080/dags/otel_test_dag_with_pause/grid?dag_run_id=manual__2024-11-30T14%3A01%3A15.333003%2B00%3A00&task_id=task_1&base_date=2024-11-30T14%3A01%3A15%2B0000&tab=logs", "airflow.task.operator": "PythonOperator", @@ -457,7 +457,7 @@ class TestUtilsUnit: "airflow.task.end_date": "2024-11-30 14:01:36.002687+00:00", "airflow.task.duration": 0.130369, "airflow.task.executor_config": "{}", - "airflow.task.execution_date": "2024-11-30 14:01:15+00:00", + "airflow.task.logical_date": "2024-11-30 14:01:15+00:00", "airflow.task.hostname": "351295342ba2", "airflow.task.log_url": "http://localhost:8080/dags/otel_test_dag_with_pause/grid?dag_run_id=manual__2024-11-30T14%3A01%3A15.333003%2B00%3A00&task_id=task_2&base_date=2024-11-30T14%3A01%3A15%2B0000&tab=logs", "airflow.task.operator": "PythonOperator", From 03e1383c090bf3a4946b1bfea98c7bec14587630 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 3 Dec 2024 14:59:09 +0200 Subject: [PATCH 16/52] fix integration and system test failures --- airflow/executors/base_executor.py | 7 ++++++- airflow/models/taskinstance.py | 9 +++++++++ airflow/traces/tracer.py | 1 + tests/integration/executors/test_celery_executor.py | 4 ++-- 4 files changed, 18 insertions(+), 3 deletions(-) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index aa66d2c02750b..482fafcfa095c 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -347,7 +347,12 @@ def trigger_tasks(self, open_slots: int) -> None: # If it's None, then the span for the current TaskInstanceKey hasn't been started. if self.active_spans is not None and self.active_spans.get(key) is None: - parent_context = Trace.extract(ti.dag_run.context_carrier) + from airflow.models.taskinstance import SimpleTaskInstance + + if isinstance(ti, SimpleTaskInstance): + parent_context = Trace.extract(ti.parent_context_carrier) + else: + parent_context = Trace.extract(ti.dag_run.context_carrier) # Start a new span using the context from the parent. # Attributes will be set once the task has finished so that all # values will be available (end_time, duration, etc.). diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 4460f4fb35ed5..8ba0233a219ba 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -3743,6 +3743,7 @@ def __init__( dag_id: str, task_id: str, run_id: str, + queued_dttm: datetime | None, start_date: datetime | None, end_date: datetime | None, try_number: int, @@ -3755,6 +3756,7 @@ def __init__( key: TaskInstanceKey, run_as_user: str | None = None, priority_weight: int | None = None, + parent_context_carrier: dict | None = None, context_carrier: dict | None = None, span_status: str | None = None, ): @@ -3762,6 +3764,7 @@ def __init__( self.task_id = task_id self.run_id = run_id self.map_index = map_index + self.queued_dttm = queued_dttm self.start_date = start_date self.end_date = end_date self.try_number = try_number @@ -3773,6 +3776,7 @@ def __init__( self.priority_weight = priority_weight self.queue = queue self.key = key + self.parent_context_carrier = parent_context_carrier self.context_carrier = context_carrier self.span_status = span_status @@ -3792,6 +3796,7 @@ def from_ti(cls, ti: TaskInstance) -> SimpleTaskInstance: task_id=ti.task_id, run_id=ti.run_id, map_index=ti.map_index, + queued_dttm=ti.queued_dttm, start_date=ti.start_date, end_date=ti.end_date, try_number=ti.try_number, @@ -3803,7 +3808,11 @@ def from_ti(cls, ti: TaskInstance) -> SimpleTaskInstance: key=ti.key, run_as_user=ti.run_as_user if hasattr(ti, "run_as_user") else None, priority_weight=ti.priority_weight if hasattr(ti, "priority_weight") else None, + parent_context_carrier=ti.dag_run.context_carrier + if (hasattr(ti, "dag_run") and hasattr(ti.dag_run, "context_carrier")) + else None, context_carrier=ti.context_carrier if hasattr(ti, "context_carrier") else None, + span_status=ti.span_status if hasattr(ti, "span_status") else None, ) diff --git a/airflow/traces/tracer.py b/airflow/traces/tracer.py index 6b5b8148966fc..82d1fe0b31b38 100644 --- a/airflow/traces/tracer.py +++ b/airflow/traces/tracer.py @@ -61,6 +61,7 @@ class EmptyContext: """If no Tracer is configured, EmptyContext is used as a fallback.""" trace_id = 1 + span_id = 1 class EmptySpan: diff --git a/tests/integration/executors/test_celery_executor.py b/tests/integration/executors/test_celery_executor.py index 0f9f0b45ae9c1..e43bd88a06481 100644 --- a/tests/integration/executors/test_celery_executor.py +++ b/tests/integration/executors/test_celery_executor.py @@ -218,7 +218,7 @@ def fake_execute_command(): ) when = datetime.now() value_tuple = ( - "command", + ["command"], 1, None, SimpleTaskInstance.from_ti(ti=TaskInstance(task=task, run_id=None)), @@ -256,7 +256,7 @@ def test_retry_on_error_sending_task(self, caplog): ) when = datetime.now() value_tuple = ( - "command", + ["command"], 1, None, SimpleTaskInstance.from_ti(ti=TaskInstance(task=task, run_id=None)), From 204233467c4391ce8c4774609418c865378acd97 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 4 Dec 2024 16:41:20 +0200 Subject: [PATCH 17/52] fix unit test failures --- airflow/models/dagrun.py | 3 --- airflow/models/taskinstance.py | 12 +++++++++--- tests/assets/test_manager.py | 2 ++ tests/models/test_taskinstance.py | 2 ++ tests/www/views/test_views_tasks.py | 14 ++++++++++++++ 5 files changed, 27 insertions(+), 6 deletions(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 423e9dfb69b19..f6fa68723f689 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -1111,9 +1111,6 @@ def recalculate(self) -> _UnfinishedStates: self.run_id, ) - for ti in schedulable_tis: - ti.dag_run = self - self.set_state(DagRunState.RUNNING) if self._state == DagRunState.FAILED or self._state == DagRunState.SUCCESS: diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 8ba0233a219ba..5f1257e4f924f 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -866,7 +866,8 @@ def _refresh_from_db( # If the scheduler that started the dag_run has exited (gracefully or forcefully), # there will be changes to the dag_run span context_carrier. # It's best to include the dag_run whenever possible, so that the ti will contain the updates. - include_dag_run = not inspector.detached and "task_instance" not in inspector.unloaded + include_dag_run = not inspector.detached and "dag_run" not in inspector.unloaded + log.debug("Unloaded: %s", inspector.unloaded) _set_ti_attrs(task_instance, ti, include_dag_run=include_dag_run) else: task_instance.state = None @@ -1003,7 +1004,11 @@ def get_triggering_events() -> dict[str, list[AssetEvent | AssetEventPydantic]]: # Re-attach it if we get called. nonlocal dag_run if dag_run not in session: - dag_run = session.merge(dag_run, load=False) + # In case, refresh_from_db has also included the dag_run, + # the object will be considered dirty by the session. + # Trying to merge the dirty dag_run with load=False, will result to an SQLAlchemy error. + # Regular merge, with the default load value. + dag_run = session.merge(dag_run) asset_events = dag_run.consumed_asset_events triggering_events: dict[str, list[AssetEvent | AssetEventPydantic]] = defaultdict(list) for event in asset_events: @@ -3808,8 +3813,9 @@ def from_ti(cls, ti: TaskInstance) -> SimpleTaskInstance: key=ti.key, run_as_user=ti.run_as_user if hasattr(ti, "run_as_user") else None, priority_weight=ti.priority_weight if hasattr(ti, "priority_weight") else None, + # Inspect the ti, to check if the 'dag_run' relationship is loaded. parent_context_carrier=ti.dag_run.context_carrier - if (hasattr(ti, "dag_run") and hasattr(ti.dag_run, "context_carrier")) + if "dag_run" not in inspect(ti).unloaded else None, context_carrier=ti.context_carrier if hasattr(ti, "context_carrier") else None, span_status=ti.span_status if hasattr(ti, "span_status") else None, diff --git a/tests/assets/test_manager.py b/tests/assets/test_manager.py index b716056e81466..17654652dea83 100644 --- a/tests/assets/test_manager.py +++ b/tests/assets/test_manager.py @@ -100,6 +100,8 @@ def mock_task_instance(): dag_model=None, raw=False, is_trigger_log_context=False, + context_carrier={}, + span_status="ended", ) diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index d9c17d4714388..90810dfa714a4 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -90,6 +90,7 @@ from airflow.utils.db import merge_conn from airflow.utils.module_loading import qualname from airflow.utils.session import create_session, provide_session +from airflow.utils.span_status import SpanStatus from airflow.utils.state import DagRunState, State, TaskInstanceState from airflow.utils.task_group import TaskGroup from airflow.utils.task_instance_session import set_current_task_instance_session @@ -3913,6 +3914,7 @@ def test_refresh_from_db(self, create_task_instance): "task_display_name": "Test Refresh from DB Task", "dag_version_id": None, "context_carrier": {}, + "span_status": SpanStatus.ENDED, } # Make sure we aren't missing any new value in our expected_values list. expected_keys = {f"task_instance.{key}" for key in expected_values} diff --git a/tests/www/views/test_views_tasks.py b/tests/www/views/test_views_tasks.py index 2f902395e6c9f..65463d13fc5b4 100644 --- a/tests/www/views/test_views_tasks.py +++ b/tests/www/views/test_views_tasks.py @@ -1123,6 +1123,8 @@ def test_task_instances(admin_client): "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), "dag_version_id": None, + "context_carrier": None, + "span_status": "not_started", }, "run_after_loop": { "custom_operator_name": None, @@ -1160,6 +1162,8 @@ def test_task_instances(admin_client): "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), "dag_version_id": None, + "context_carrier": None, + "span_status": "not_started", }, "run_this_last": { "custom_operator_name": None, @@ -1197,6 +1201,8 @@ def test_task_instances(admin_client): "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), "dag_version_id": None, + "context_carrier": None, + "span_status": "not_started", }, "runme_0": { "custom_operator_name": None, @@ -1234,6 +1240,8 @@ def test_task_instances(admin_client): "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), "dag_version_id": None, + "context_carrier": None, + "span_status": "not_started", }, "runme_1": { "custom_operator_name": None, @@ -1271,6 +1279,8 @@ def test_task_instances(admin_client): "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), "dag_version_id": None, + "context_carrier": None, + "span_status": "not_started", }, "runme_2": { "custom_operator_name": None, @@ -1308,6 +1318,8 @@ def test_task_instances(admin_client): "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), "dag_version_id": None, + "context_carrier": None, + "span_status": "not_started", }, "this_will_skip": { "custom_operator_name": None, @@ -1345,5 +1357,7 @@ def test_task_instances(admin_client): "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), "dag_version_id": None, + "context_carrier": None, + "span_status": "not_started", }, } From 5178e14398f37cc75c6924c95bc4ad4059ff0d4a Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 4 Dec 2024 19:07:28 +0200 Subject: [PATCH 18/52] cleanup --- airflow/cli/commands/task_command.py | 2 +- airflow/jobs/scheduler_job_runner.py | 21 +++++++++++---------- airflow/models/taskinstance.py | 4 ++-- airflow/traces/otel_tracer.py | 2 -- tests/integration/otel/test_otel.py | 8 ++++---- tests/integration/otel/test_utils.py | 2 +- 6 files changed, 19 insertions(+), 20 deletions(-) diff --git a/airflow/cli/commands/task_command.py b/airflow/cli/commands/task_command.py index 8ae1cff5346fd..a987975bf3fa6 100644 --- a/airflow/cli/commands/task_command.py +++ b/airflow/cli/commands/task_command.py @@ -464,7 +464,7 @@ def task_run(args, dag: DAG | None = None) -> TaskReturnCode | None: log.info("Running %s on host %s", ti, hostname) if args.carrier is not None: - log.info("Found args.carrier: %s. Setting the value in the ti instance.", args.carrier) + log.info("Found args.carrier: %s. Setting the value on the ti instance.", args.carrier) # The arg value is a dict string, and it needs to be converted back to a dict. carrier_dict = json.loads(args.carrier) ti.set_context_carrier(context_carrier=carrier_dict, with_commit=True) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 8fe08c5bdb2c1..2342236e80c23 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -171,9 +171,9 @@ class SchedulerJobRunner(BaseJobRunner, LoggingMixin): job_type = "SchedulerJob" - # Dict entries: dag_run.run_id - span + # key: dag_run.run_id | value: span active_dagrun_spans = ThreadSafeDict() - # Dict entries: ti.key - span + # key: ti.key | value: span active_ti_spans = ThreadSafeDict() def __init__( @@ -246,7 +246,7 @@ def register_signals(self) -> ExitStack: def _exit_gracefully(self, signum: int, frame: FrameType | None) -> None: """Clean up processor_agent to avoid leaving orphan processes.""" - self._cleanup_active_spans_before_process_exit() + self._end_active_spans() if not _is_parent_process(): # Only the parent process should perform the cleanup. @@ -839,7 +839,7 @@ def process_executor_events( active_ti_span = cls.active_ti_spans.get(ti.key) if active_ti_span is not None: - cls._set_span_attrs__process_executor_events(span=active_ti_span, state=state, ti=ti) + cls.set_ti_span_attrs(span=active_ti_span, state=state, ti=ti) # End the span and remove it from the active_ti_spans dict. active_ti_span.end(end_time=datetime_to_nano(ti.end_date)) cls.active_ti_spans.delete(ti.key) @@ -908,7 +908,7 @@ def process_executor_events( return len(event_buffer) @classmethod - def _set_span_attrs__process_executor_events(cls, span, state, ti): + def set_ti_span_attrs(cls, span, state, ti): span.set_attributes( { "airflow.category": "scheduler", @@ -1048,7 +1048,7 @@ def _update_dag_run_state_for_paused_dags(self, session: Session = NEW_SESSION) self.log.exception("Failed to update dag run state for paused dags due to %s", e) @provide_session - def _cleanup_active_spans_before_process_exit(self, session: Session = NEW_SESSION): + def _end_active_spans(self, session: Session = NEW_SESSION): # No need to do a commit for every update. The annotation will commit all of them once at the end. for run_id, span in self.active_dagrun_spans.get_all().items(): dag_run: DagRun = session.scalars(select(DagRun).where(DagRun.run_id == run_id)).one() @@ -1079,7 +1079,7 @@ def _cleanup_active_spans_before_process_exit(self, session: Session = NEW_SESSI ) ).one() if ti.state in State.finished: - self._set_span_attrs__process_executor_events(span=span, state=ti.state, ti=ti) + self.set_ti_span_attrs(span=span, state=ti.state, ti=ti) span.end(end_time=datetime_to_nano(ti.end_date)) ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: @@ -1125,7 +1125,7 @@ def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): active_ti_span = self.active_ti_spans.get(ti.key) if active_ti_span is not None: if ti.state in State.finished: - self._set_span_attrs__process_executor_events(span=active_ti_span, state=ti.state, ti=ti) + self.set_ti_span_attrs(span=active_ti_span, state=ti.state, ti=ti) active_ti_span.end(end_time=datetime_to_nano(ti.end_date)) else: active_ti_span.end() @@ -1133,7 +1133,7 @@ def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) @provide_session - def _recreate_dead_scheduler_spans_if_needed(self, dag_run: DagRun, session: Session = NEW_SESSION): + def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session: Session = NEW_SESSION): scheduler_health_timeout = conf.getint("scheduler", "scheduler_health_check_threshold") # There are two scenarios: @@ -1202,6 +1202,7 @@ def _recreate_dead_scheduler_spans_if_needed(self, dag_run: DagRun, session: Ses ti.set_context_carrier(context_carrier=ti_carrier, session=session, with_commit=False) if ti.state in State.finished: + self.set_ti_span_attrs(span=ti_span, state=ti.state, ti=ti) ti_span.end(end_time=datetime_to_nano(ti.end_date)) ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: @@ -1953,7 +1954,7 @@ def _schedule_dag_run( # If the dag_run has been previously scheduled by another job and there is no active span, # then check if the job is still healthy. # If it's not healthy, then recreate the spans. - self._recreate_dead_scheduler_spans_if_needed(dag_run, session) + self._recreate_unhealthy_scheduler_spans_if_needed(dag_run, session) dag_run.set_scheduled_by_job_id(job_id=self.job.id, session=session, with_commit=False) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 5f1257e4f924f..36db37b3e0257 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -862,7 +862,7 @@ def _refresh_from_db( if ti: inspector = inspect(ti) - # Check if the ti is detached or not loaded. + # Check if the ti is detached or the dag_run relationship isn't loaded. # If the scheduler that started the dag_run has exited (gracefully or forcefully), # there will be changes to the dag_run span context_carrier. # It's best to include the dag_run whenever possible, so that the ti will contain the updates. @@ -2279,7 +2279,7 @@ def set_span_status( """ Set TaskInstance span_status. - :param status: dict with the injected carrier to set for the ti + :param status: dict with the injected carrier to set for the TI :param session: SQLAlchemy ORM Session :param with_commit: should the status be committed? :return: has the span_status been changed? diff --git a/airflow/traces/otel_tracer.py b/airflow/traces/otel_tracer.py index aa94f98b47d20..4feaf6810a8a5 100644 --- a/airflow/traces/otel_tracer.py +++ b/airflow/traces/otel_tracer.py @@ -48,8 +48,6 @@ _NEXT_ID = create_key("next_id") -CTX_PROP_SUFFIX = "_ctx_prop" - class OtelTrace: """ diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index 3cc1fd0c3ae9a..934ebb9809ac7 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -45,7 +45,7 @@ ) from tests_common.test_utils.db import initial_db_init -log = logging.getLogger("test_otel.TestOtelIntegration") +log = logging.getLogger("integration.otel.test_otel") def unpause_trigger_dag_and_get_run_id(dag_id: str) -> str: @@ -448,7 +448,7 @@ def setup_class(cls): os.environ["AIRFLOW__SCHEDULER__PROCESSOR_POLL_INTERVAL"] = "2" # The heartrate is determined by the conf "AIRFLOW__SCHEDULER__SCHEDULER_HEARTBEAT_SEC". - # By default, the heartrate is 5 seconds. Every iteration of the scheduler loop check the + # By default, the heartrate is 5 seconds. Every iteration of the scheduler loop, checks the # time passed since the last heartbeat and if it was longer than the 5 second heartrate, # it performs a heartbeat update. # If there hasn't been a heartbeat for an amount of time longer than the @@ -619,7 +619,7 @@ def test_scheduler_change_after_the_first_task_finishes( break with capfd.disabled(): - # When we pause the scheduler1 thread, capfd keeps trying to read the + # When the scheduler1 thread is paused, capfd keeps trying to read the # file descriptors for the process and ends up freezing the test. # Temporarily disable capfd to avoid that. scheduler_process_1.send_signal(signal.SIGSTOP) @@ -719,7 +719,7 @@ def test_scheduler_change_in_the_middle_of_first_task_until_the_end( continue with capfd.disabled(): - # When we pause the scheduler1 thread, capfd keeps trying to read the + # When the scheduler1 thread is paused, capfd keeps trying to read the # file descriptors for the process and ends up freezing the test. # Temporarily disable capfd to avoid that. scheduler_process_1.send_signal(signal.SIGSTOP) diff --git a/tests/integration/otel/test_utils.py b/tests/integration/otel/test_utils.py index 69a0899987f44..8a60768c0b932 100644 --- a/tests/integration/otel/test_utils.py +++ b/tests/integration/otel/test_utils.py @@ -24,7 +24,7 @@ from airflow.models import Base -log = logging.getLogger("tests.otel.test_utils") +log = logging.getLogger("integration.otel.test_utils") def dump_airflow_metadata_db(session): From ea3fbc158e3f5fc12515dad38dfbe58b0ca0d635 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 4 Dec 2024 20:23:00 +0200 Subject: [PATCH 19/52] fix sqlite migration failure --- ...istency_between_ORM_and_migration_files.py | 3 + docs/apache-airflow/img/airflow_erd.sha256 | 2 +- docs/apache-airflow/img/airflow_erd.svg | 2196 +++++++++-------- 3 files changed, 1116 insertions(+), 1085 deletions(-) diff --git a/airflow/migrations/versions/0017_2_9_2_fix_inconsistency_between_ORM_and_migration_files.py b/airflow/migrations/versions/0017_2_9_2_fix_inconsistency_between_ORM_and_migration_files.py index 0a62b550d40b9..ff9f776a19ff6 100644 --- a/airflow/migrations/versions/0017_2_9_2_fix_inconsistency_between_ORM_and_migration_files.py +++ b/airflow/migrations/versions/0017_2_9_2_fix_inconsistency_between_ORM_and_migration_files.py @@ -236,6 +236,9 @@ def upgrade(): log_template_id INTEGER, updated_at TIMESTAMP, clear_number INTEGER DEFAULT '0' NOT NULL, + scheduled_by_job_id INTEGER, + context_carrier VARCHAR(50), + span_status VARCHAR(50), CONSTRAINT dag_run_pkey PRIMARY KEY (id), CONSTRAINT dag_run_dag_id_execution_date_key UNIQUE (dag_id, execution_date), CONSTRAINT dag_run_dag_id_run_id_key UNIQUE (dag_id, run_id), diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index 37e52ffe2516b..93e29fc323573 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -ae1ba5e05c5775211358665dc956b1e6b9793596be1c955672bbe31a285e56cb \ No newline at end of file +90aefc347131dc1b7af6dfdee27cbb26afa5d895dc7ebab28ecfbdb71580e408 \ No newline at end of file diff --git a/docs/apache-airflow/img/airflow_erd.svg b/docs/apache-airflow/img/airflow_erd.svg index ad5fd1ee2cf96..6a443adc94d60 100644 --- a/docs/apache-airflow/img/airflow_erd.svg +++ b/docs/apache-airflow/img/airflow_erd.svg @@ -4,11 +4,11 @@ - - + + %3 - + log @@ -105,8 +105,8 @@ callback_data - [JSON] - NOT NULL + [JSONB] + NOT NULL callback_type @@ -604,74 +604,74 @@ asset_event - -asset_event - -id - - [INTEGER] - NOT NULL - -asset_id - - [INTEGER] - NOT NULL - -extra - - [JSON] - NOT NULL - -source_dag_id - - [VARCHAR(250)] - -source_map_index - - [INTEGER] - -source_run_id - - [VARCHAR(250)] - -source_task_id - - [VARCHAR(250)] - -timestamp - - [TIMESTAMP] - NOT NULL + +asset_event + +id + + [INTEGER] + NOT NULL + +asset_id + + [INTEGER] + NOT NULL + +extra + + [JSON] + NOT NULL + +source_dag_id + + [VARCHAR(250)] + +source_map_index + + [INTEGER] + +source_run_id + + [VARCHAR(250)] + +source_task_id + + [VARCHAR(250)] + +timestamp + + [TIMESTAMP] + NOT NULL asset_event--asset_alias_asset_event - -0..N -1 + +0..N +1 dagrun_asset_event - -dagrun_asset_event - -dag_run_id - - [INTEGER] - NOT NULL - -event_id - - [INTEGER] - NOT NULL + +dagrun_asset_event + +dag_run_id + + [INTEGER] + NOT NULL + +event_id + + [INTEGER] + NOT NULL asset_event--dagrun_asset_event - -0..N -1 + +0..N +1 @@ -713,687 +713,703 @@ task_instance - -task_instance - -id - - [UUID] - NOT NULL - -custom_operator_name - - [VARCHAR(1000)] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -last_heartbeat_at - - [TIMESTAMP] - -map_index - - [INTEGER] - NOT NULL - -max_tries - - [INTEGER] - -next_kwargs - - [JSON] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -task_id - - [VARCHAR(250)] - NOT NULL - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance + +id + + [UUID] + NOT NULL + +context_carrier + + [JSONB] + +custom_operator_name + + [VARCHAR(1000)] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +last_heartbeat_at + + [TIMESTAMP] + +map_index + + [INTEGER] + NOT NULL + +max_tries + + [INTEGER] + +next_kwargs + + [JSONB] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +run_id + + [VARCHAR(250)] + NOT NULL + +span_status + + [VARCHAR(50)] + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +task_id + + [VARCHAR(250)] + NOT NULL + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] trigger--task_instance - -0..N + +0..N {0,1} task_reschedule - -task_reschedule - -id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -duration - - [INTEGER] - NOT NULL - -end_date - - [TIMESTAMP] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -reschedule_date - - [TIMESTAMP] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -try_number - - [INTEGER] - NOT NULL + +task_reschedule + +id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +duration + + [INTEGER] + NOT NULL + +end_date + + [TIMESTAMP] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +reschedule_date + + [TIMESTAMP] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +try_number + + [INTEGER] + NOT NULL task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 rendered_task_instance_fields - -rendered_task_instance_fields - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -k8s_pod_yaml - - [JSON] - -rendered_fields - - [JSON] - NOT NULL + +rendered_task_instance_fields + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +k8s_pod_yaml + + [JSON] + +rendered_fields + + [JSON] + NOT NULL task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_map - -task_map - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -keys - - [JSON] - -length - - [INTEGER] - NOT NULL + +task_map + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +keys + + [JSONB] + +length + + [INTEGER] + NOT NULL task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 xcom - -xcom - -dag_run_id - - [INTEGER] - NOT NULL - -key - - [VARCHAR(512)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -timestamp - - [TIMESTAMP] - NOT NULL - -value - - [JSONB] + +xcom + +dag_run_id + + [INTEGER] + NOT NULL + +key + + [VARCHAR(512)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +timestamp + + [TIMESTAMP] + NOT NULL + +value + + [JSONB] task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance_note - -task_instance_note - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] + +task_instance_note + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [VARCHAR(128)] task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance_history - -task_instance_history - -id - - [INTEGER] - NOT NULL - -custom_operator_name - - [VARCHAR(1000)] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -map_index - - [INTEGER] - NOT NULL - -max_tries - - [INTEGER] - -next_kwargs - - [JSON] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -task_id - - [VARCHAR(250)] - NOT NULL - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - NOT NULL - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance_history + +id + + [INTEGER] + NOT NULL + +context_carrier + + [JSONB] + +custom_operator_name + + [VARCHAR(1000)] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +map_index + + [INTEGER] + NOT NULL + +max_tries + + [INTEGER] + +next_kwargs + + [JSONB] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +run_id + + [VARCHAR(250)] + NOT NULL + +span_status + + [VARCHAR(50)] + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +task_id + + [VARCHAR(250)] + NOT NULL + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + NOT NULL + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 @@ -1413,7 +1429,7 @@ kwargs - [JSON] + [JSONB] last_refreshed @@ -1697,8 +1713,8 @@ dag_version--task_instance - -0..N + +0..N {0,1} @@ -1752,82 +1768,94 @@ dag_run - -dag_run + +dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + +clear_number + + [INTEGER] + NOT NULL -id - - [INTEGER] - NOT NULL +conf + + [BYTEA] -backfill_id - - [INTEGER] +context_carrier + + [JSONB] -clear_number - - [INTEGER] - NOT NULL +creating_job_id + + [INTEGER] -conf - - [BYTEA] +dag_id + + [VARCHAR(250)] + NOT NULL -creating_job_id +dag_version_id - [INTEGER] + [UUID] -dag_id - - [VARCHAR(250)] - NOT NULL +data_interval_end + + [TIMESTAMP] -dag_version_id - - [UUID] +data_interval_start + + [TIMESTAMP] -data_interval_end - - [TIMESTAMP] +end_date + + [TIMESTAMP] -data_interval_start - - [TIMESTAMP] +external_trigger + + [BOOLEAN] -end_date - - [TIMESTAMP] +last_scheduling_decision + + [TIMESTAMP] -external_trigger - - [BOOLEAN] +log_template_id + + [INTEGER] -last_scheduling_decision - - [TIMESTAMP] +logical_date + + [TIMESTAMP] + NOT NULL -log_template_id - - [INTEGER] +queued_at + + [TIMESTAMP] -logical_date - - [TIMESTAMP] - NOT NULL +run_id + + [VARCHAR(250)] + NOT NULL -queued_at - - [TIMESTAMP] +run_type + + [VARCHAR(50)] + NOT NULL -run_id - - [VARCHAR(250)] - NOT NULL +scheduled_by_job_id + + [INTEGER] -run_type - - [VARCHAR(50)] - NOT NULL +span_status + + [VARCHAR(50)] start_date @@ -1848,9 +1876,9 @@ dag_version--dag_run - -0..N -{0,1} + +0..N +{0,1} @@ -1905,536 +1933,536 @@ dag_run--dagrun_asset_event - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 backfill_dag_run - -backfill_dag_run - -id - - [INTEGER] - NOT NULL - -backfill_id - - [INTEGER] - NOT NULL - -dag_run_id - - [INTEGER] - -exception_reason - - [VARCHAR(250)] - -logical_date - - [TIMESTAMP] - NOT NULL - -sort_ordinal - - [INTEGER] - NOT NULL + +backfill_dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + NOT NULL + +dag_run_id + + [INTEGER] + +exception_reason + + [VARCHAR(250)] + +logical_date + + [TIMESTAMP] + NOT NULL + +sort_ordinal + + [INTEGER] + NOT NULL dag_run--backfill_dag_run - -0..N -{0,1} + +0..N +{0,1} dag_run_note - -dag_run_note - -dag_run_id - - [INTEGER] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] + +dag_run_note + +dag_run_id + + [INTEGER] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [VARCHAR(128)] dag_run--dag_run_note - -1 -1 + +1 +1 dag_run--task_reschedule - -0..N -1 + +0..N +1 dag_run--task_reschedule - -0..N -1 + +0..N +1 log_template - -log_template - -id - - [INTEGER] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -elasticsearch_id - - [TEXT] - NOT NULL - -filename - - [TEXT] - NOT NULL + +log_template + +id + + [INTEGER] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +elasticsearch_id + + [TEXT] + NOT NULL + +filename + + [TEXT] + NOT NULL log_template--dag_run - -0..N -{0,1} + +0..N +{0,1} backfill - -backfill - -id - - [INTEGER] - NOT NULL - -completed_at - - [TIMESTAMP] - -created_at - - [TIMESTAMP] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_run_conf - - [JSON] - NOT NULL - -from_date - - [TIMESTAMP] - NOT NULL - -is_paused - - [BOOLEAN] - -max_active_runs - - [INTEGER] - NOT NULL - -reprocess_behavior - - [VARCHAR(250)] - NOT NULL - -to_date - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +backfill + +id + + [INTEGER] + NOT NULL + +completed_at + + [TIMESTAMP] + +created_at + + [TIMESTAMP] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_run_conf + + [JSON] + NOT NULL + +from_date + + [TIMESTAMP] + NOT NULL + +is_paused + + [BOOLEAN] + +max_active_runs + + [INTEGER] + NOT NULL + +reprocess_behavior + + [VARCHAR(250)] + NOT NULL + +to_date + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL backfill--dag_run - -0..N -{0,1} + +0..N +{0,1} backfill--backfill_dag_run - -0..N -1 + +0..N +1 session - -session - -id - - [INTEGER] - NOT NULL - -data - - [BYTEA] - -expiry - - [TIMESTAMP] - -session_id - - [VARCHAR(255)] + +session + +id + + [INTEGER] + NOT NULL + +data + + [BYTEA] + +expiry + + [TIMESTAMP] + +session_id + + [VARCHAR(255)] alembic_version - -alembic_version - -version_num - - [VARCHAR(32)] - NOT NULL + +alembic_version + +version_num + + [VARCHAR(32)] + NOT NULL ab_user - -ab_user - -id - - [INTEGER] - NOT NULL - -active - - [BOOLEAN] - -changed_by_fk - - [INTEGER] - -changed_on - - [TIMESTAMP] - -created_by_fk - - [INTEGER] - -created_on - - [TIMESTAMP] - -email - - [VARCHAR(512)] - NOT NULL - -fail_login_count - - [INTEGER] - -first_name - - [VARCHAR(256)] - NOT NULL - -last_login - - [TIMESTAMP] - -last_name - - [VARCHAR(256)] - NOT NULL - -login_count - - [INTEGER] - -password - - [VARCHAR(256)] - -username - - [VARCHAR(512)] - NOT NULL + +ab_user + +id + + [INTEGER] + NOT NULL + +active + + [BOOLEAN] + +changed_by_fk + + [INTEGER] + +changed_on + + [TIMESTAMP] + +created_by_fk + + [INTEGER] + +created_on + + [TIMESTAMP] + +email + + [VARCHAR(512)] + NOT NULL + +fail_login_count + + [INTEGER] + +first_name + + [VARCHAR(256)] + NOT NULL + +last_login + + [TIMESTAMP] + +last_name + + [VARCHAR(256)] + NOT NULL + +login_count + + [INTEGER] + +password + + [VARCHAR(256)] + +username + + [VARCHAR(512)] + NOT NULL ab_user--ab_user - -0..N -{0,1} + +0..N +{0,1} ab_user--ab_user - -0..N -{0,1} + +0..N +{0,1} ab_user_role - -ab_user_role - -id - - [INTEGER] - NOT NULL - -role_id - - [INTEGER] - -user_id - - [INTEGER] + +ab_user_role + +id + + [INTEGER] + NOT NULL + +role_id + + [INTEGER] + +user_id + + [INTEGER] ab_user--ab_user_role - -0..N -{0,1} + +0..N +{0,1} ab_register_user - -ab_register_user - -id - - [INTEGER] - NOT NULL - -email - - [VARCHAR(512)] - NOT NULL - -first_name - - [VARCHAR(256)] - NOT NULL - -last_name - - [VARCHAR(256)] - NOT NULL - -password - - [VARCHAR(256)] - -registration_date - - [TIMESTAMP] - -registration_hash - - [VARCHAR(256)] - -username - - [VARCHAR(512)] - NOT NULL + +ab_register_user + +id + + [INTEGER] + NOT NULL + +email + + [VARCHAR(512)] + NOT NULL + +first_name + + [VARCHAR(256)] + NOT NULL + +last_name + + [VARCHAR(256)] + NOT NULL + +password + + [VARCHAR(256)] + +registration_date + + [TIMESTAMP] + +registration_hash + + [VARCHAR(256)] + +username + + [VARCHAR(512)] + NOT NULL ab_permission - -ab_permission - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(100)] - NOT NULL + +ab_permission + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(100)] + NOT NULL ab_permission_view - -ab_permission_view - -id - - [INTEGER] - NOT NULL - -permission_id - - [INTEGER] - -view_menu_id - - [INTEGER] + +ab_permission_view + +id + + [INTEGER] + NOT NULL + +permission_id + + [INTEGER] + +view_menu_id + + [INTEGER] ab_permission--ab_permission_view - -0..N -{0,1} + +0..N +{0,1} ab_permission_view_role - -ab_permission_view_role - -id - - [INTEGER] - NOT NULL - -permission_view_id - - [INTEGER] - -role_id - - [INTEGER] + +ab_permission_view_role + +id + + [INTEGER] + NOT NULL + +permission_view_id + + [INTEGER] + +role_id + + [INTEGER] ab_permission_view--ab_permission_view_role - -0..N -{0,1} + +0..N +{0,1} ab_view_menu - -ab_view_menu - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(250)] - NOT NULL + +ab_view_menu + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(250)] + NOT NULL ab_view_menu--ab_permission_view - -0..N -{0,1} + +0..N +{0,1} ab_role - -ab_role - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(64)] - NOT NULL + +ab_role + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(64)] + NOT NULL ab_role--ab_user_role - -0..N -{0,1} + +0..N +{0,1} ab_role--ab_permission_view_role - -0..N -{0,1} + +0..N +{0,1} alembic_version_fab - -alembic_version_fab - -version_num - - [VARCHAR(32)] - NOT NULL + +alembic_version_fab + +version_num + + [VARCHAR(32)] + NOT NULL From e3be7e30e72b7aa8a31295950ec8960e810ffb31 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Fri, 13 Dec 2024 11:51:53 +0200 Subject: [PATCH 20/52] task output not captured by tests - fixed --- airflow/cli/cli_config.py | 4 +- airflow/traces/otel_tracer.py | 9 +++- tests/core/test_otel_tracer.py | 24 ++++++++- tests/integration/otel/dags/otel_test_dag.py | 2 +- .../otel/dags/otel_test_dag_with_pause.py | 2 +- tests/integration/otel/test_otel.py | 52 +++++++++++++++++-- 6 files changed, 83 insertions(+), 10 deletions(-) diff --git a/airflow/cli/cli_config.py b/airflow/cli/cli_config.py index 5a384b8bdc136..8124ea6380cf7 100644 --- a/airflow/cli/cli_config.py +++ b/airflow/cli/cli_config.py @@ -552,7 +552,9 @@ def string_lower_type(val): ) ARG_RAW = Arg(("-r", "--raw"), argparse.SUPPRESS, "store_true") ARG_CARRIER = Arg( - ("-c", "--carrier"), help="Context Carrier, containing the injected context for the task span", nargs="?" + ("-c", "--carrier"), + help="Context Carrier, containing the injected context for the Otel task span", + type=str, ) ARG_IGNORE_ALL_DEPENDENCIES = Arg( ("-A", "--ignore-all-dependencies"), diff --git a/airflow/traces/otel_tracer.py b/airflow/traces/otel_tracer.py index 4feaf6810a8a5..60e64b738462c 100644 --- a/airflow/traces/otel_tracer.py +++ b/airflow/traces/otel_tracer.py @@ -59,8 +59,8 @@ class OtelTrace: def __init__( self, span_exporter: OTLPSpanExporter, + use_simple_processor: bool, tag_string: str | None = None, - use_simple_processor: bool = False, ): self.span_exporter = span_exporter self.use_simple_processor = use_simple_processor @@ -69,8 +69,10 @@ def __init__( # A task can run fast and finish before spans have enough time to get exported to the collector. # When creating spans from inside a task, a SimpleSpanProcessor needs to be used because # it exports the spans immediately after they are created. + log.info("(__init__) - [SimpleSpanProcessor] is being used") self.span_processor: SpanProcessor = SimpleSpanProcessor(self.span_exporter) else: + log.info("(__init__) - [BatchSpanProcessor] is being used") self.span_processor = BatchSpanProcessor(self.span_exporter) self.tag_string = tag_string self.otel_service = conf.get("traces", "otel_service") @@ -98,8 +100,10 @@ def get_otel_tracer_provider( if debug is True: log.info("[ConsoleSpanExporter] is being used") if self.use_simple_processor: + log.info("[SimpleSpanProcessor] is being used") span_processor_for_tracer_prov: SpanProcessor = SimpleSpanProcessor(ConsoleSpanExporter()) else: + log.info("[BatchSpanProcessor] is being used") span_processor_for_tracer_prov = BatchSpanProcessor(ConsoleSpanExporter()) else: span_processor_for_tracer_prov = self.span_processor @@ -330,10 +334,11 @@ def get_otel_tracer(cls, use_simple_processor: bool = False) -> OtelTrace: protocol = "https" if ssl_active else "http" endpoint = f"{protocol}://{host}:{port}/v1/traces" log.info("[OTLPSpanExporter] Connecting to OpenTelemetry Collector at %s", endpoint) + log.info("Should use simple processor: %s", use_simple_processor) return OtelTrace( span_exporter=OTLPSpanExporter(endpoint=endpoint), - tag_string=tag_string, use_simple_processor=use_simple_processor, + tag_string=tag_string, ) diff --git a/tests/core/test_otel_tracer.py b/tests/core/test_otel_tracer.py index 57709f284ce74..6de8e933b990f 100644 --- a/tests/core/test_otel_tracer.py +++ b/tests/core/test_otel_tracer.py @@ -25,8 +25,10 @@ from opentelemetry.sdk import util from opentelemetry.sdk.trace.export.in_memory_span_exporter import InMemorySpanExporter +from airflow.configuration import conf from airflow.traces import otel_tracer -from airflow.traces.tracer import Trace +from airflow.traces.otel_tracer import OtelTrace +from airflow.traces.tracer import EmptyTrace, Trace from airflow.utils.dates import datetime_to_nano from tests_common.test_utils.config import env_vars @@ -38,6 +40,26 @@ def name(): class TestOtelTrace: + def test_get_otel_tracer_from_trace_metaclass(self): + """Test that `Trace.some_method()`, uses an `OtelTrace` instance when otel is configured.""" + conf.add_section("traces") + conf.set("traces", "otel_on", "True") + conf.set("traces", "otel_debugging_on", "True") + + tracer = otel_tracer.get_otel_tracer(Trace) + assert tracer.use_simple_processor is False + + assert isinstance(Trace.factory(), EmptyTrace) + + Trace.configure_factory() + assert isinstance(Trace.factory(), OtelTrace) + + task_tracer = otel_tracer.get_otel_tracer_for_task(Trace) + assert task_tracer.use_simple_processor is True + + task_tracer.get_otel_tracer_provider() + assert task_tracer.use_simple_processor is True + @patch("opentelemetry.sdk.trace.export.ConsoleSpanExporter") @patch("airflow.traces.otel_tracer.conf") def test_tracer(self, conf_a, exporter): diff --git a/tests/integration/otel/dags/otel_test_dag.py b/tests/integration/otel/dags/otel_test_dag.py index 3be234f7eb7e3..92b03dcd1f0cd 100644 --- a/tests/integration/otel/dags/otel_test_dag.py +++ b/tests/integration/otel/dags/otel_test_dag.py @@ -54,7 +54,7 @@ def task1_func(**dag_context): if context_carrier is not None: logger.info("Found ti.context_carrier: %s.", context_carrier) logger.info("Extracting the span context from the context_carrier.") - parent_context = Trace.extract(context_carrier) + parent_context = otel_task_tracer.extract(context_carrier) with otel_task_tracer.start_child_span( span_name=f"{ti.task_id}_sub_span1", parent_context=parent_context, diff --git a/tests/integration/otel/dags/otel_test_dag_with_pause.py b/tests/integration/otel/dags/otel_test_dag_with_pause.py index e02d7ef3520bf..8b8b3e5d5b5b2 100644 --- a/tests/integration/otel/dags/otel_test_dag_with_pause.py +++ b/tests/integration/otel/dags/otel_test_dag_with_pause.py @@ -34,7 +34,7 @@ args = { "owner": "airflow", - "start_date": datetime(2024, 9, 1), + "start_date": datetime(2024, 9, 2), "retries": 0, } diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index 934ebb9809ac7..c0822c5d2bcba 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -21,6 +21,7 @@ import signal import subprocess import time +from typing import Any import pendulum import pytest @@ -43,7 +44,6 @@ extract_spans_from_output, get_parent_child_dict, ) -from tests_common.test_utils.db import initial_db_init log = logging.getLogger("integration.otel.test_otel") @@ -395,6 +395,35 @@ def check_spans_without_continuance( ) +def print_output_for_dag_tis(dag: DAG): + with create_session() as session: + tis: list[TaskInstance] = dag.get_task_instances(session=session) + + for ti in tis: + print_ti_output(ti) + + +def print_ti_output(ti: TaskInstance): + from airflow.utils.log.log_reader import TaskLogReader + + task_log_reader = TaskLogReader() + if task_log_reader.supports_read: + metadata: dict[str, Any] = {} + logs, metadata = task_log_reader.read_log_chunks(ti, ti.try_number, metadata) + if ti.hostname in dict(logs[0]): + output = ( + str(dict(logs[0])[ti.hostname]) + .replace("\\n", "\n") + .replace("{log.py:232} WARNING - {", "\n{") + ) + while metadata["end_of_log"] is False: + logs, metadata = task_log_reader.read_log_chunks(ti, ti.try_number - 1, metadata) + if ti.hostname in dict(logs[0]): + output = output + str(dict(logs[0])[ti.hostname]).replace("\\n", "\n") + # Logging the output is enough for capfd to capture it. + log.info(format(output)) + + @pytest.mark.integration("redis") @pytest.mark.backend("postgres") class TestOtelIntegration: @@ -464,12 +493,12 @@ def setup_class(cls): if cls.log_level == "debug": log.setLevel(logging.DEBUG) - initial_db_init() - - cls.dags = cls.serialize_and_get_dags() + db_init_command = ["airflow", "db", "init"] + subprocess.run(db_init_command, check=True, env=os.environ.copy()) @classmethod def serialize_and_get_dags(cls) -> dict[str, DAG]: + log.info("Serializing Dags from directory %s", cls.dag_folder) # Load DAGs from the dag directory. dag_bag = DagBag(dag_folder=cls.dag_folder, include_examples=False) @@ -509,6 +538,11 @@ def reset_db(self): # Reset the db using the cli. subprocess.run(reset_command, check=True, env=os.environ.copy()) + migrate_command = ["airflow", "db", "migrate"] + subprocess.run(migrate_command, check=True, env=os.environ.copy()) + + self.dags = self.serialize_and_get_dags() + def test_same_scheduler_processing_the_entire_dag( self, monkeypatch, celery_worker_env_vars, capfd, session ): @@ -544,6 +578,7 @@ def test_same_scheduler_processing_the_entire_dag( check_ti_state_and_span_status( task_id=ti.task_id, run_id=run_id, state=State.SUCCESS, span_status=SpanStatus.ENDED ) + print_ti_output(ti) finally: if self.log_level == "debug": with create_session() as session: @@ -651,6 +686,7 @@ def test_scheduler_change_after_the_first_task_finishes( dag_id=dag_id, run_id=run_id, max_wait_time=30, span_status=SpanStatus.ENDED ) + print_output_for_dag_tis(dag=dag) finally: if self.log_level == "debug": with create_session() as session: @@ -757,6 +793,8 @@ def test_scheduler_change_in_the_middle_of_first_task_until_the_end( wait_for_dag_run_and_check_span_status( dag_id=dag_id, run_id=run_id, max_wait_time=30, span_status=SpanStatus.ENDED ) + + print_output_for_dag_tis(dag=dag) finally: if self.log_level == "debug": with create_session() as session: @@ -840,6 +878,8 @@ def test_scheduler_exits_gracefully_in_the_middle_of_the_first_task( wait_for_dag_run_and_check_span_status( dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.ENDED ) + + print_output_for_dag_tis(dag=dag) finally: if self.log_level == "debug": with create_session() as session: @@ -924,6 +964,8 @@ def test_scheduler_exits_forcefully_in_the_middle_of_the_first_task( wait_for_dag_run_and_check_span_status( dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.ENDED ) + + print_output_for_dag_tis(dag=dag) finally: if self.log_level == "debug": with create_session() as session: @@ -1014,6 +1056,8 @@ def test_scheduler_exits_forcefully_after_the_first_task_finishes( wait_for_dag_run_and_check_span_status( dag_id=dag_id, run_id=run_id, max_wait_time=120, span_status=SpanStatus.ENDED ) + + print_output_for_dag_tis(dag=dag) finally: if self.log_level == "debug": with create_session() as session: From 975f533d084570036268598bcd7b2c48bf1ad5fb Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Fri, 13 Dec 2024 12:03:39 +0200 Subject: [PATCH 21/52] cleanup inaccurate comment --- airflow/executors/base_executor.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index 482fafcfa095c..d72c3ef9b26a7 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -367,8 +367,6 @@ def trigger_tasks(self, open_slots: int) -> None: # Inject the current context into the carrier. carrier = Trace.inject() # The carrier needs to be set on the ti, but it can't happen here because db calls are expensive. - # By the time the db update has finished, another heartbeat will have started - # and the tasks will have been triggered again. # So set the carrier as an argument to the command. # The command execution will set it on the ti, and it will be propagated to the task itself. command.append("--carrier") From e7c4cfdd9505491a6fa8ff2a424989c69902fc59 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Fri, 13 Dec 2024 12:17:17 +0200 Subject: [PATCH 22/52] use one common dictionary for active spans --- airflow/jobs/scheduler_job_runner.py | 47 ++++++++++++++-------------- airflow/models/dagrun.py | 22 ++++++------- 2 files changed, 33 insertions(+), 36 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 4f7a78f1128d2..89d8fef8e55a0 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -172,10 +172,11 @@ class SchedulerJobRunner(BaseJobRunner, LoggingMixin): job_type = "SchedulerJob" - # key: dag_run.run_id | value: span - active_dagrun_spans = ThreadSafeDict() - # key: ti.key | value: span - active_ti_spans = ThreadSafeDict() + # For a dagrun span + # - key: dag_run.run_id | value: span + # For a ti span + # - key: ti.key | value: span + active_spans = ThreadSafeDict() def __init__( self, @@ -838,12 +839,12 @@ def process_executor_events( ti.pid, ) - active_ti_span = cls.active_ti_spans.get(ti.key) + active_ti_span = cls.active_spans.get(ti.key) if active_ti_span is not None: cls.set_ti_span_attrs(span=active_ti_span, state=state, ti=ti) - # End the span and remove it from the active_ti_spans dict. + # End the span and remove it from the active_spans dict. active_ti_span.end(end_time=datetime_to_nano(ti.end_date)) - cls.active_ti_spans.delete(ti.key) + cls.active_spans.delete(ti.key) ti.set_span_status(status=SpanStatus.ENDED, session=session) else: if ti.span_status == SpanStatus.ACTIVE: @@ -1051,7 +1052,7 @@ def _update_dag_run_state_for_paused_dags(self, session: Session = NEW_SESSION) @provide_session def _end_active_spans(self, session: Session = NEW_SESSION): # No need to do a commit for every update. The annotation will commit all of them once at the end. - for run_id, span in self.active_dagrun_spans.get_all().items(): + for run_id, span in self.active_spans.get_all().items(): dag_run: DagRun = session.scalars(select(DagRun).where(DagRun.run_id == run_id)).one() if dag_run.state in State.finished_dr_states: dagv = session.scalar(select(DagVersion).where(DagVersion.id == dag_run.dag_version_id)) @@ -1070,7 +1071,7 @@ def _end_active_spans(self, session: Session = NEW_SESSION): ) as s: s.set_attribute("trace_status", "needs continuance") - for key, span in self.active_ti_spans.get_all().items(): + for key, span in self.active_spans.get_all().items(): # Can't compare the key directly because the try_number or the map_index might not be the same. ti: TaskInstance = session.scalars( select(TaskInstance).where( @@ -1087,8 +1088,8 @@ def _end_active_spans(self, session: Session = NEW_SESSION): span.end() ti.set_span_status(status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False) - self.active_dagrun_spans.clear() - self.active_ti_spans.clear() + self.active_spans.clear() + self.active_spans.clear() @provide_session def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): @@ -1110,7 +1111,7 @@ def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): ).all() for dag_run in dag_runs_should_end: - active_dagrun_span = self.active_dagrun_spans.get(dag_run.run_id) + active_dagrun_span = self.active_spans.get(dag_run.run_id) if active_dagrun_span is not None: if dag_run.state in State.finished_dr_states: dagv = session.scalar(select(DagVersion).where(DagVersion.id == dag_run.dag_version_id)) @@ -1119,18 +1120,18 @@ def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): active_dagrun_span.end(end_time=datetime_to_nano(dag_run.end_date)) else: active_dagrun_span.end() - self.active_dagrun_spans.delete(dag_run.run_id) + self.active_spans.delete(dag_run.run_id) dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) for ti in tis_should_end: - active_ti_span = self.active_ti_spans.get(ti.key) + active_ti_span = self.active_spans.get(ti.key) if active_ti_span is not None: if ti.state in State.finished: self.set_ti_span_attrs(span=active_ti_span, state=ti.state, ti=ti) active_ti_span.end(end_time=datetime_to_nano(ti.end_date)) else: active_ti_span.end() - self.active_ti_spans.delete(ti.key) + self.active_spans.delete(ti.key) ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) @provide_session @@ -1178,7 +1179,7 @@ def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session carrier = Trace.inject() # Update the context_carrier and leave the SpanStatus as ACTIVE. dr.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) - self.active_dagrun_spans.set(dr.run_id, dr_span) + self.active_spans.set(dr.run_id, dr_span) tis = dr.get_task_instances(session=session) @@ -1188,7 +1189,7 @@ def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session tis_needing_spans = [ ti for ti in tis - if ti.span_status == SpanStatus.ACTIVE and self.active_ti_spans.get(ti.key) is None + if ti.span_status == SpanStatus.ACTIVE and self.active_spans.get(ti.key) is None ] dr_context = Trace.extract(dr.context_carrier) @@ -1208,7 +1209,7 @@ def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: ti.set_span_status(status=SpanStatus.ACTIVE, session=session, with_commit=False) - self.active_ti_spans.set(ti.key, ti_span) + self.active_spans.set(ti.key, ti_span) def _run_scheduler_loop(self) -> None: """ @@ -1298,15 +1299,13 @@ def _run_scheduler_loop(self) -> None: self._end_spans_of_externally_ended_ops() # Pass a reference to the dictionary. - # Any changes made by a dag_run instance, will be reflected to the dictionaries of this class. - DagRun.set_active_spans( - active_dagrun_spans=self.active_dagrun_spans, active_ti_spans=self.active_ti_spans - ) + # Any changes made by a dag_run instance, will be reflected to the dictionary of this class. + DagRun.set_active_spans(active_spans=self.active_spans) # local import due to type_checking. from airflow.executors.base_executor import BaseExecutor - BaseExecutor.set_active_spans(active_spans=self.active_ti_spans) + BaseExecutor.set_active_spans(active_spans=self.active_spans) with create_session() as session: # This will schedule for as many executors as possible. @@ -1950,7 +1949,7 @@ def _schedule_dag_run( if ( dag_run.scheduled_by_job_id is not None and dag_run.set_scheduled_by_job_id != self.job.id - and self.active_dagrun_spans.get(dag_run.run_id) is None + and self.active_spans.get(dag_run.run_id) is None ): # If the dag_run has been previously scheduled by another job and there is no active span, # then check if the job is still healthy. diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index f6fa68723f689..e620b11139784 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -139,8 +139,7 @@ class DagRun(Base, LoggingMixin): external trigger (i.e. manual runs). """ - active_dagrun_spans = ThreadSafeDict() - active_ti_spans = ThreadSafeDict() + active_spans = ThreadSafeDict() __tablename__ = "dag_run" @@ -313,9 +312,8 @@ def stats_tags(self) -> dict[str, str]: return prune_dict({"dag_id": self.dag_id, "run_type": self.run_type}) @classmethod - def set_active_spans(cls, active_dagrun_spans: ThreadSafeDict, active_ti_spans: ThreadSafeDict): - cls.active_dagrun_spans = active_dagrun_spans - cls.active_ti_spans = active_ti_spans + def set_active_spans(cls, active_spans: ThreadSafeDict): + cls.active_spans = active_spans def get_state(self): return self._state @@ -1044,8 +1042,8 @@ def recalculate(self) -> _UnfinishedStates: # finally, if the leaves aren't done, the dag is still running else: - # If there is no value in active_dagrun_spans, then the span hasn't already been started. - if self.active_dagrun_spans is not None and self.active_dagrun_spans.get(self.run_id) is None: + # If there is no value in active_spans, then the span hasn't already been started. + if self.active_spans is not None and self.active_spans.get(self.run_id) is None: if ( self.span_status == SpanStatus.NOT_STARTED or self.span_status == SpanStatus.NEEDS_CONTINUANCE @@ -1081,7 +1079,7 @@ def recalculate(self) -> _UnfinishedStates: self.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) self.set_span_status(status=SpanStatus.ACTIVE, session=session, with_commit=False) # Set the span in a synchronized dictionary, so that the variable can be used to end the span. - self.active_dagrun_spans.set(self.run_id, dr_span) + self.active_spans.set(self.run_id, dr_span) self.log.debug( "DagRun span has been started and the injected context_carrier is: %s", self.context_carrier, @@ -1103,7 +1101,7 @@ def recalculate(self) -> _UnfinishedStates: ti.set_span_status( status=SpanStatus.ACTIVE, session=session, with_commit=False ) - self.active_ti_spans.set(ti.key, ti_span) + self.active_spans.set(ti.key, ti_span) else: self.log.info( "Found span_status '%s', while updating state for dag_run '%s'", @@ -1141,8 +1139,8 @@ def recalculate(self) -> _UnfinishedStates: dagv.version if dagv else None, ) - if self.active_dagrun_spans is not None: - active_span = self.active_dagrun_spans.get(self.run_id) + if self.active_spans is not None: + active_span = self.active_spans.get(self.run_id) if active_span is not None: self.log.debug( "Found active span with span_id: %s, for dag_id: %s, run_id: %s, state: %s", @@ -1155,7 +1153,7 @@ def recalculate(self) -> _UnfinishedStates: self.set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) active_span.end(end_time=datetime_to_nano(self.end_date)) # Remove the span from the dict. - self.active_dagrun_spans.delete(self.run_id) + self.active_spans.delete(self.run_id) self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) else: if self.span_status == SpanStatus.ACTIVE: From 48928f9973ab79aa2bf0112ee7eb81876dc11c78 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Fri, 13 Dec 2024 12:48:38 +0200 Subject: [PATCH 23/52] rephrase comment about otel spans --- airflow/jobs/scheduler_job_runner.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 89d8fef8e55a0..6c2b0ecdc4686 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -1096,9 +1096,9 @@ def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): # The scheduler that starts a dag_run or a task is also the one that starts the spans. # Each scheduler should end the spans that it has started. # - # Otel spans are designed so that only the process that starts them, - # has full control over their lifecycle. - # This also means that the process that started them, is the only one that can end them. + # Otel spans are implemented in a certain way so that the objects + # can't be shared between processes or get recreated. + # It is done so that the process that starts a span, is also the one that ends it. # # If another scheduler has finished processing a dag_run or a task and there is a reference # on the active_spans dictionary, then the current scheduler started the span, From 3943ffc6ac4375826fe5c8b89da204118f9a3168 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Fri, 13 Dec 2024 14:22:08 +0200 Subject: [PATCH 24/52] scheduler cleanup --- airflow/jobs/scheduler_job_runner.py | 47 ++++++++++------------------ 1 file changed, 16 insertions(+), 31 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 6c2b0ecdc4686..07fe9a71d06e9 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -1091,8 +1091,7 @@ def _end_active_spans(self, session: Session = NEW_SESSION): self.active_spans.clear() self.active_spans.clear() - @provide_session - def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): + def _end_spans_of_externally_ended_ops(self, session: Session): # The scheduler that starts a dag_run or a task is also the one that starts the spans. # Each scheduler should end the spans that it has started. # @@ -1136,52 +1135,39 @@ def _end_spans_of_externally_ended_ops(self, session: Session = NEW_SESSION): @provide_session def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session: Session = NEW_SESSION): - scheduler_health_timeout = conf.getint("scheduler", "scheduler_health_check_threshold") - # There are two scenarios: # 1. scheduler is unhealthy but managed to update span_status # 2. scheduler is unhealthy and didn't manage to make any updates # Check the span_status first, in case the 2nd db query can be avoided (scenario 1). - # Get the latest values from the db. - dr: DagRun = session.scalars( - select(DagRun).where( - DagRun.run_id == dag_run.run_id, - DagRun.dag_id == dag_run.dag_id, - ) - ).one() - # If the dag_run is scheduled by a different scheduler, and it's still running and the span is active, # then check the Job table to determine if the initial scheduler is still healthy. if ( - dr.scheduled_by_job_id != self.job.id - and dr.state in State.unfinished_dr_states - and dr.span_status == SpanStatus.ACTIVE + dag_run.scheduled_by_job_id != self.job.id + and dag_run.state in State.unfinished_dr_states + and dag_run.span_status == SpanStatus.ACTIVE ): job: Job = session.scalars( select(Job).where( - Job.id == dr.scheduled_by_job_id, + Job.id == dag_run.scheduled_by_job_id, Job.job_type == "SchedulerJob", ) ).one() - # If the time passed since the last heartbeat is less than the timeout. - is_healthy = scheduler_health_timeout > (timezone.utcnow() - job.latest_heartbeat).total_seconds() - - if not is_healthy: + if not job.is_alive(): # Start a new span for the dag_run. dr_span = Trace.start_root_span( - span_name=f"{dr.dag_id}_recreated", + span_name=f"{dag_run.dag_id}_recreated", component="dag", - start_time=dr.queued_at, + start_time=dag_run.queued_at, start_as_current=False, ) carrier = Trace.inject() # Update the context_carrier and leave the SpanStatus as ACTIVE. - dr.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) - self.active_spans.set(dr.run_id, dr_span) + dag_run.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) + self.active_spans.set(dag_run.run_id, dr_span) - tis = dr.get_task_instances(session=session) + tis = dag_run.get_task_instances(session=session) # At this point, any tis will have been adopted by the current scheduler. # If the span_status is ACTIVE but there isn't an entry on the active spans, @@ -1192,7 +1178,7 @@ def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session if ti.span_status == SpanStatus.ACTIVE and self.active_spans.get(ti.key) is None ] - dr_context = Trace.extract(dr.context_carrier) + dr_context = Trace.extract(dag_run.context_carrier) for ti in tis_needing_spans: ti_span = Trace.start_child_span( span_name=f"{ti.task_id}_recreated", @@ -1295,9 +1281,6 @@ def _run_scheduler_loop(self) -> None: self.log.debug("Waiting for processors to finish since we're using sqlite") self.processor_agent.wait_until_finished() - # This is using a new session. - self._end_spans_of_externally_ended_ops() - # Pass a reference to the dictionary. # Any changes made by a dag_run instance, will be reflected to the dictionary of this class. DagRun.set_active_spans(active_spans=self.active_spans) @@ -1308,6 +1291,8 @@ def _run_scheduler_loop(self) -> None: BaseExecutor.set_active_spans(active_spans=self.active_spans) with create_session() as session: + self._end_spans_of_externally_ended_ops(session) + # This will schedule for as many executors as possible. num_queued_tis = self._do_scheduling(session) @@ -1948,7 +1933,7 @@ def _schedule_dag_run( if ( dag_run.scheduled_by_job_id is not None - and dag_run.set_scheduled_by_job_id != self.job.id + and dag_run.scheduled_by_job_id != self.job.id and self.active_spans.get(dag_run.run_id) is None ): # If the dag_run has been previously scheduled by another job and there is no active span, @@ -1956,7 +1941,7 @@ def _schedule_dag_run( # If it's not healthy, then recreate the spans. self._recreate_unhealthy_scheduler_spans_if_needed(dag_run, session) - dag_run.set_scheduled_by_job_id(job_id=self.job.id, session=session, with_commit=False) + dag_run.scheduled_by_job_id = self.job.id # TODO[HA]: Rename update_state -> schedule_dag_run, ?? something else? schedulable_tis, callback_to_run = dag_run.update_state(session=session, execute_callbacks=False) From 7f122f621e9db31a7720119d0ed481a0183d6b44 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sun, 15 Dec 2024 12:41:42 +0200 Subject: [PATCH 25/52] scheduler unit tests --- airflow/jobs/scheduler_job_runner.py | 73 +++++----- tests/jobs/test_scheduler_job.py | 192 +++++++++++++++++++++++++++ 2 files changed, 230 insertions(+), 35 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 07fe9a71d06e9..1ea74eee47279 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -1052,41 +1052,45 @@ def _update_dag_run_state_for_paused_dags(self, session: Session = NEW_SESSION) @provide_session def _end_active_spans(self, session: Session = NEW_SESSION): # No need to do a commit for every update. The annotation will commit all of them once at the end. - for run_id, span in self.active_spans.get_all().items(): - dag_run: DagRun = session.scalars(select(DagRun).where(DagRun.run_id == run_id)).one() - if dag_run.state in State.finished_dr_states: - dagv = session.scalar(select(DagVersion).where(DagVersion.id == dag_run.dag_version_id)) - DagRun.set_dagrun_span_attrs(span=span, dag_run=dag_run, dagv=dagv) - - span.end(end_time=datetime_to_nano(dag_run.end_date)) - dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) - else: - span.end() - dag_run.set_span_status( - status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False - ) - initial_dag_run_context = Trace.extract(dag_run.context_carrier) - with Trace.start_child_span( - span_name="current_scheduler_exited", parent_context=initial_dag_run_context - ) as s: - s.set_attribute("trace_status", "needs continuance") - for key, span in self.active_spans.get_all().items(): - # Can't compare the key directly because the try_number or the map_index might not be the same. - ti: TaskInstance = session.scalars( - select(TaskInstance).where( - TaskInstance.dag_id == key.dag_id, - TaskInstance.task_id == key.task_id, - TaskInstance.run_id == key.run_id, - ) - ).one() - if ti.state in State.finished: - self.set_ti_span_attrs(span=span, state=ti.state, ti=ti) - span.end(end_time=datetime_to_nano(ti.end_date)) - ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + from airflow.models.taskinstance import TaskInstanceKey + + if isinstance(key, TaskInstanceKey): # ti span. + # Can't compare the key directly because the try_number or the map_index might not be the same. + ti: TaskInstance = session.scalars( + select(TaskInstance).where( + TaskInstance.dag_id == key.dag_id, + TaskInstance.task_id == key.task_id, + TaskInstance.run_id == key.run_id, + ) + ).one() + if ti.state in State.finished: + self.set_ti_span_attrs(span=span, state=ti.state, ti=ti) + span.end(end_time=datetime_to_nano(ti.end_date)) + ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + else: + span.end() + ti.set_span_status( + status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False + ) else: - span.end() - ti.set_span_status(status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False) + dag_run: DagRun = session.scalars(select(DagRun).where(DagRun.run_id == key)).one() + if dag_run.state in State.finished_dr_states: + dagv = session.scalar(select(DagVersion).where(DagVersion.id == dag_run.dag_version_id)) + DagRun.set_dagrun_span_attrs(span=span, dag_run=dag_run, dagv=dagv) + + span.end(end_time=datetime_to_nano(dag_run.end_date)) + dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + else: + span.end() + dag_run.set_span_status( + status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False + ) + initial_dag_run_context = Trace.extract(dag_run.context_carrier) + with Trace.start_child_span( + span_name="current_scheduler_exited", parent_context=initial_dag_run_context + ) as s: + s.set_attribute("trace_status", "needs continuance") self.active_spans.clear() self.active_spans.clear() @@ -1133,8 +1137,7 @@ def _end_spans_of_externally_ended_ops(self, session: Session): self.active_spans.delete(ti.key) ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) - @provide_session - def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session: Session = NEW_SESSION): + def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session: Session): # There are two scenarios: # 1. scheduler is unhealthy but managed to update span_status # 2. scheduler is unhealthy and didn't manage to make any updates diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index 6b413135bfc3d..b27b78abd7353 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -68,10 +68,13 @@ from airflow.sdk.definitions.asset import Asset from airflow.serialization.serialized_objects import SerializedDAG from airflow.timetables.base import DataInterval +from airflow.traces.tracer import Trace from airflow.utils import timezone from airflow.utils.file import list_py_file_paths from airflow.utils.session import create_session, provide_session +from airflow.utils.span_status import SpanStatus from airflow.utils.state import DagRunState, State, TaskInstanceState +from airflow.utils.thread_safe_dict import ThreadSafeDict from airflow.utils.types import DagRunType from tests.listeners import dag_listener @@ -2422,6 +2425,195 @@ def test_runs_are_created_after_max_active_runs_was_reached(self, dag_maker, ses dag_runs = DagRun.find(dag_id=dag.dag_id, session=session) assert len(dag_runs) == 2 + @pytest.mark.parametrize( + "ti_state, final_ti_span_status", + [(State.SUCCESS, SpanStatus.ENDED), (State.RUNNING, SpanStatus.ACTIVE)], + ) + def test_recreate_unhealthy_scheduler_spans_if_needed(self, ti_state, final_ti_span_status, dag_maker): + with dag_maker( + dag_id="test_recreate_unhealthy_scheduler_spans_if_needed", + start_date=DEFAULT_DATE, + max_active_runs=1, + processor_subdir=TEST_DAG_FOLDER, + dagrun_timeout=datetime.timedelta(seconds=60), + ): + EmptyOperator(task_id="dummy") + + session = settings.Session() + + old_job = Job() + old_job.id = 1 + old_job.job_type = SchedulerJobRunner.job_type + + session.add(old_job) + session.commit() + + assert old_job.is_alive() is False + + new_job = Job() + new_job.id = 2 + new_job.job_type = SchedulerJobRunner.job_type + + self.job_runner = SchedulerJobRunner(job=new_job, subdir=os.devnull) + self.job_runner.active_spans = ThreadSafeDict() + assert len(self.job_runner.active_spans.get_all()) == 0 + + dr = dag_maker.create_dagrun(external_trigger=True) + dr.state = State.RUNNING + dr.span_status = SpanStatus.ACTIVE + dr.scheduled_by_job_id = old_job.id + + ti = dr.get_task_instances(session=session)[0] + ti.state = ti_state + ti.span_status = SpanStatus.ACTIVE + ti.queued_by_job_id = old_job.id + session.merge(ti) + session.merge(dr) + session.commit() + + # Given + assert dr.scheduled_by_job_id != self.job_runner.job.id + assert dr.scheduled_by_job_id == old_job.id + assert dr.run_id is not None + assert dr.state == State.RUNNING + assert dr.span_status == SpanStatus.ACTIVE + assert self.job_runner.active_spans.get(dr.run_id) is None + + assert self.job_runner.active_spans.get(ti.key) is None + assert ti.state == ti_state + assert ti.span_status == SpanStatus.ACTIVE + assert ti.context_carrier == {} + + # When + self.job_runner._recreate_unhealthy_scheduler_spans_if_needed(dr, session) + + # Then + assert self.job_runner.active_spans.get(dr.run_id) is not None + + if final_ti_span_status == SpanStatus.ACTIVE: + assert self.job_runner.active_spans.get(ti.key) is not None + else: + assert self.job_runner.active_spans.get(ti.key) is None + + assert dr.span_status == SpanStatus.ACTIVE + assert ti.span_status == final_ti_span_status + + def test_end_spans_of_externally_ended_ops(self, dag_maker): + with dag_maker( + dag_id="test_end_spans_of_externally_ended_ops", + start_date=DEFAULT_DATE, + max_active_runs=1, + processor_subdir=TEST_DAG_FOLDER, + dagrun_timeout=datetime.timedelta(seconds=60), + ): + EmptyOperator(task_id="dummy") + + session = settings.Session() + + job = Job() + job.id = 1 + job.job_type = SchedulerJobRunner.job_type + + self.job_runner = SchedulerJobRunner(job=job, subdir=os.devnull) + self.job_runner.active_spans = ThreadSafeDict() + assert len(self.job_runner.active_spans.get_all()) == 0 + + dr = dag_maker.create_dagrun(external_trigger=True) + dr.state = State.SUCCESS + dr.span_status = SpanStatus.SHOULD_END + + ti = dr.get_task_instances(session=session)[0] + ti.state = State.SUCCESS + ti.span_status = SpanStatus.SHOULD_END + ti.context_carrier = {} + session.merge(ti) + session.merge(dr) + session.commit() + + dr_span = Trace.start_root_span(span_name="dag_run_span", start_as_current=False) + ti_span = Trace.start_child_span(span_name="ti_span", start_as_current=False) + + self.job_runner.active_spans.set(dr.run_id, dr_span) + self.job_runner.active_spans.set(ti.key, ti_span) + + # Given + assert dr.span_status == SpanStatus.SHOULD_END + assert ti.span_status == SpanStatus.SHOULD_END + + assert self.job_runner.active_spans.get(dr.run_id) is not None + assert self.job_runner.active_spans.get(ti.key) is not None + + # When + self.job_runner._end_spans_of_externally_ended_ops(session) + + # Then + assert dr.span_status == SpanStatus.ENDED + assert ti.span_status == SpanStatus.ENDED + + assert self.job_runner.active_spans.get(dr.run_id) is None + assert self.job_runner.active_spans.get(ti.key) is None + + @pytest.mark.parametrize( + "state, final_span_status", + [(State.SUCCESS, SpanStatus.ENDED), (State.RUNNING, SpanStatus.NEEDS_CONTINUANCE)], + ) + def test_end_active_spans(self, state, final_span_status, dag_maker): + with dag_maker( + dag_id="test_end_active_spans", + start_date=DEFAULT_DATE, + max_active_runs=1, + processor_subdir=TEST_DAG_FOLDER, + dagrun_timeout=datetime.timedelta(seconds=60), + ): + EmptyOperator(task_id="dummy") + + session = settings.Session() + + job = Job() + job.id = 1 + job.job_type = SchedulerJobRunner.job_type + + self.job_runner = SchedulerJobRunner(job=job, subdir=os.devnull) + self.job_runner.active_spans = ThreadSafeDict() + assert len(self.job_runner.active_spans.get_all()) == 0 + + dr = dag_maker.create_dagrun(external_trigger=True) + dr.state = state + dr.span_status = SpanStatus.ACTIVE + + ti = dr.get_task_instances(session=session)[0] + ti.state = state + ti.span_status = SpanStatus.ACTIVE + ti.context_carrier = {} + session.merge(ti) + session.merge(dr) + session.commit() + + dr_span = Trace.start_root_span(span_name="dag_run_span", start_as_current=False) + ti_span = Trace.start_child_span(span_name="ti_span", start_as_current=False) + + self.job_runner.active_spans.set(dr.run_id, dr_span) + self.job_runner.active_spans.set(ti.key, ti_span) + + # Given + assert dr.span_status == SpanStatus.ACTIVE + assert ti.span_status == SpanStatus.ACTIVE + + assert self.job_runner.active_spans.get(dr.run_id) is not None + assert self.job_runner.active_spans.get(ti.key) is not None + assert len(self.job_runner.active_spans.get_all()) == 2 + + # When + self.job_runner._end_active_spans(session) + + # Then + assert dr.span_status == final_span_status + assert ti.span_status == final_span_status + + assert self.job_runner.active_spans.get(dr.run_id) is None + assert self.job_runner.active_spans.get(ti.key) is None + assert len(self.job_runner.active_spans.get_all()) == 0 + def test_dagrun_timeout_verify_max_active_runs(self, dag_maker): """ Test if a a dagrun will not be scheduled if max_dag_runs From 8511bd79e52058e4e2ca81a15488b86cb8b95573 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sun, 15 Dec 2024 15:04:03 +0200 Subject: [PATCH 26/52] revert changes in old migration file --- ..._2_9_2_fix_inconsistency_between_ORM_and_migration_files.py | 3 --- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/airflow/migrations/versions/0017_2_9_2_fix_inconsistency_between_ORM_and_migration_files.py b/airflow/migrations/versions/0017_2_9_2_fix_inconsistency_between_ORM_and_migration_files.py index ff9f776a19ff6..0a62b550d40b9 100644 --- a/airflow/migrations/versions/0017_2_9_2_fix_inconsistency_between_ORM_and_migration_files.py +++ b/airflow/migrations/versions/0017_2_9_2_fix_inconsistency_between_ORM_and_migration_files.py @@ -236,9 +236,6 @@ def upgrade(): log_template_id INTEGER, updated_at TIMESTAMP, clear_number INTEGER DEFAULT '0' NOT NULL, - scheduled_by_job_id INTEGER, - context_carrier VARCHAR(50), - span_status VARCHAR(50), CONSTRAINT dag_run_pkey PRIMARY KEY (id), CONSTRAINT dag_run_dag_id_execution_date_key UNIQUE (dag_id, execution_date), CONSTRAINT dag_run_dag_id_run_id_key UNIQUE (dag_id, run_id), diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index 93e29fc323573..37e52ffe2516b 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -90aefc347131dc1b7af6dfdee27cbb26afa5d895dc7ebab28ecfbdb71580e408 \ No newline at end of file +ae1ba5e05c5775211358665dc956b1e6b9793596be1c955672bbe31a285e56cb \ No newline at end of file From 08de46ff3fd44a579d3d495c09bd7b057e3f26b1 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sun, 15 Dec 2024 15:20:51 +0200 Subject: [PATCH 27/52] move passing reference to active_spans dict out of the scheduler loop --- airflow/jobs/scheduler_job_runner.py | 17 ++++++++--------- 1 file changed, 8 insertions(+), 9 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index 1ea74eee47279..bc874baefb5e8 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -988,6 +988,14 @@ def _execute(self) -> int | None: execute_start_time = timezone.utcnow() + # local import due to type_checking. + from airflow.executors.base_executor import BaseExecutor + + # Pass a reference to the dictionary. + # Any changes made by a dag_run instance, will be reflected to the dictionary of this class. + DagRun.set_active_spans(active_spans=self.active_spans) + BaseExecutor.set_active_spans(active_spans=self.active_spans) + self._run_scheduler_loop() if self.processor_agent: @@ -1284,15 +1292,6 @@ def _run_scheduler_loop(self) -> None: self.log.debug("Waiting for processors to finish since we're using sqlite") self.processor_agent.wait_until_finished() - # Pass a reference to the dictionary. - # Any changes made by a dag_run instance, will be reflected to the dictionary of this class. - DagRun.set_active_spans(active_spans=self.active_spans) - - # local import due to type_checking. - from airflow.executors.base_executor import BaseExecutor - - BaseExecutor.set_active_spans(active_spans=self.active_spans) - with create_session() as session: self._end_spans_of_externally_ended_ops(session) From ec321525f003dbd8e7914f52d5e618061aa275b1 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sun, 15 Dec 2024 15:28:01 +0200 Subject: [PATCH 28/52] make set_dagrun_span_attrs not a static method --- airflow/jobs/scheduler_job_runner.py | 4 +-- airflow/models/dagrun.py | 41 +++++++++++++--------------- 2 files changed, 21 insertions(+), 24 deletions(-) diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index bc874baefb5e8..ffea163a7cf83 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -1085,7 +1085,7 @@ def _end_active_spans(self, session: Session = NEW_SESSION): dag_run: DagRun = session.scalars(select(DagRun).where(DagRun.run_id == key)).one() if dag_run.state in State.finished_dr_states: dagv = session.scalar(select(DagVersion).where(DagVersion.id == dag_run.dag_version_id)) - DagRun.set_dagrun_span_attrs(span=span, dag_run=dag_run, dagv=dagv) + dag_run.set_dagrun_span_attrs(span=span, dagv=dagv) span.end(end_time=datetime_to_nano(dag_run.end_date)) dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) @@ -1126,7 +1126,7 @@ def _end_spans_of_externally_ended_ops(self, session: Session): if active_dagrun_span is not None: if dag_run.state in State.finished_dr_states: dagv = session.scalar(select(DagVersion).where(DagVersion.id == dag_run.dag_version_id)) - DagRun.set_dagrun_span_attrs(span=active_dagrun_span, dag_run=dag_run, dagv=dagv) + dag_run.set_dagrun_span_attrs(span=active_dagrun_span, dagv=dagv) active_dagrun_span.end(end_time=datetime_to_nano(dag_run.end_date)) else: diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index e620b11139784..e63e3ad0778be 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -860,9 +860,8 @@ def is_effective_leaf(task): leaf_tis = {ti for ti in tis if ti.task_id in leaf_task_ids if ti.state != TaskInstanceState.REMOVED} return leaf_tis - @staticmethod - def set_dagrun_span_attrs(span: Span | EmptySpan, dag_run: DagRun, dagv: DagVersion): - if dag_run._state == DagRunState.FAILED: + def set_dagrun_span_attrs(self, span: Span | EmptySpan, dagv: DagVersion): + if self._state == DagRunState.FAILED: span.set_attribute("airflow.dag_run.error", True) attribute_value_type = Union[ @@ -879,29 +878,27 @@ def set_dagrun_span_attrs(span: Span | EmptySpan, dag_run: DagRun, dagv: DagVers # Explicitly set the value type to Union[...] to avoid a mypy error. attributes: dict[str, attribute_value_type] = { "airflow.category": "DAG runs", - "airflow.dag_run.dag_id": str(dag_run.dag_id), - "airflow.dag_run.logical_date": str(dag_run.logical_date), - "airflow.dag_run.run_id": str(dag_run.run_id), - "airflow.dag_run.queued_at": str(dag_run.queued_at), - "airflow.dag_run.run_start_date": str(dag_run.start_date), - "airflow.dag_run.run_end_date": str(dag_run.end_date), + "airflow.dag_run.dag_id": str(self.dag_id), + "airflow.dag_run.logical_date": str(self.logical_date), + "airflow.dag_run.run_id": str(self.run_id), + "airflow.dag_run.queued_at": str(self.queued_at), + "airflow.dag_run.run_start_date": str(self.start_date), + "airflow.dag_run.run_end_date": str(self.end_date), "airflow.dag_run.run_duration": str( - (dag_run.end_date - dag_run.start_date).total_seconds() - if dag_run.start_date and dag_run.end_date - else 0 + (self.end_date - self.start_date).total_seconds() if self.start_date and self.end_date else 0 ), - "airflow.dag_run.state": str(dag_run._state), - "airflow.dag_run.external_trigger": str(dag_run.external_trigger), - "airflow.dag_run.run_type": str(dag_run.run_type), - "airflow.dag_run.data_interval_start": str(dag_run.data_interval_start), - "airflow.dag_run.data_interval_end": str(dag_run.data_interval_end), + "airflow.dag_run.state": str(self._state), + "airflow.dag_run.external_trigger": str(self.external_trigger), + "airflow.dag_run.run_type": str(self.run_type), + "airflow.dag_run.data_interval_start": str(self.data_interval_start), + "airflow.dag_run.data_interval_end": str(self.data_interval_end), "airflow.dag_version.version": str(dagv.version if dagv else None), - "airflow.dag_run.conf": str(dag_run.conf), + "airflow.dag_run.conf": str(self.conf), } if span.is_recording(): - span.add_event(name="airflow.dag_run.queued", timestamp=datetime_to_nano(dag_run.queued_at)) - span.add_event(name="airflow.dag_run.started", timestamp=datetime_to_nano(dag_run.start_date)) - span.add_event(name="airflow.dag_run.ended", timestamp=datetime_to_nano(dag_run.end_date)) + span.add_event(name="airflow.dag_run.queued", timestamp=datetime_to_nano(self.queued_at)) + span.add_event(name="airflow.dag_run.started", timestamp=datetime_to_nano(self.start_date)) + span.add_event(name="airflow.dag_run.ended", timestamp=datetime_to_nano(self.end_date)) span.set_attributes(attributes) @provide_session @@ -1150,7 +1147,7 @@ def recalculate(self) -> _UnfinishedStates: self.state, ) - self.set_dagrun_span_attrs(span=active_span, dag_run=self, dagv=dagv) + self.set_dagrun_span_attrs(span=active_span, dagv=dagv) active_span.end(end_time=datetime_to_nano(self.end_date)) # Remove the span from the dict. self.active_spans.delete(self.run_id) From 5a234bef924c95e4fc4187e56efca35fea398d2d Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sun, 15 Dec 2024 17:34:34 +0200 Subject: [PATCH 29/52] remove set methods from dagrun and ti --- .../commands/remote_commands/task_command.py | 7 +- airflow/jobs/scheduler_job_runner.py | 28 ++-- airflow/models/dagrun.py | 135 +----------------- airflow/models/taskinstance.py | 80 +---------- tests/jobs/test_scheduler_job.py | 1 - 5 files changed, 25 insertions(+), 226 deletions(-) diff --git a/airflow/cli/commands/remote_commands/task_command.py b/airflow/cli/commands/remote_commands/task_command.py index a987975bf3fa6..7ca75bd273716 100644 --- a/airflow/cli/commands/remote_commands/task_command.py +++ b/airflow/cli/commands/remote_commands/task_command.py @@ -467,8 +467,11 @@ def task_run(args, dag: DAG | None = None) -> TaskReturnCode | None: log.info("Found args.carrier: %s. Setting the value on the ti instance.", args.carrier) # The arg value is a dict string, and it needs to be converted back to a dict. carrier_dict = json.loads(args.carrier) - ti.set_context_carrier(context_carrier=carrier_dict, with_commit=True) - ti.set_span_status(status=SpanStatus.ACTIVE, with_commit=True) + with create_session() as session: + ti.context_carrier = carrier_dict + ti.span_status = SpanStatus.ACTIVE + session.merge(ti) + session.commit() task_return_code = None try: diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index ffea163a7cf83..c64242b9080ab 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -845,12 +845,12 @@ def process_executor_events( # End the span and remove it from the active_spans dict. active_ti_span.end(end_time=datetime_to_nano(ti.end_date)) cls.active_spans.delete(ti.key) - ti.set_span_status(status=SpanStatus.ENDED, session=session) + ti.span_status = SpanStatus.ENDED else: if ti.span_status == SpanStatus.ACTIVE: # Another scheduler has started the span. # Update the SpanStatus to let the process know that it must end it. - ti.set_span_status(status=SpanStatus.SHOULD_END, session=session) + ti.span_status = SpanStatus.SHOULD_END # There are two scenarios why the same TI with the same try_number is queued # after executor is finished with it: @@ -1075,12 +1075,10 @@ def _end_active_spans(self, session: Session = NEW_SESSION): if ti.state in State.finished: self.set_ti_span_attrs(span=span, state=ti.state, ti=ti) span.end(end_time=datetime_to_nano(ti.end_date)) - ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + ti.span_status = SpanStatus.ENDED else: span.end() - ti.set_span_status( - status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False - ) + ti.span_status = SpanStatus.NEEDS_CONTINUANCE else: dag_run: DagRun = session.scalars(select(DagRun).where(DagRun.run_id == key)).one() if dag_run.state in State.finished_dr_states: @@ -1088,12 +1086,10 @@ def _end_active_spans(self, session: Session = NEW_SESSION): dag_run.set_dagrun_span_attrs(span=span, dagv=dagv) span.end(end_time=datetime_to_nano(dag_run.end_date)) - dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + dag_run.span_status = SpanStatus.ENDED else: span.end() - dag_run.set_span_status( - status=SpanStatus.NEEDS_CONTINUANCE, session=session, with_commit=False - ) + dag_run.span_status = SpanStatus.NEEDS_CONTINUANCE initial_dag_run_context = Trace.extract(dag_run.context_carrier) with Trace.start_child_span( span_name="current_scheduler_exited", parent_context=initial_dag_run_context @@ -1132,7 +1128,7 @@ def _end_spans_of_externally_ended_ops(self, session: Session): else: active_dagrun_span.end() self.active_spans.delete(dag_run.run_id) - dag_run.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + dag_run.span_status = SpanStatus.ENDED for ti in tis_should_end: active_ti_span = self.active_spans.get(ti.key) @@ -1143,7 +1139,7 @@ def _end_spans_of_externally_ended_ops(self, session: Session): else: active_ti_span.end() self.active_spans.delete(ti.key) - ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + ti.span_status = SpanStatus.ENDED def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session: Session): # There are two scenarios: @@ -1175,7 +1171,7 @@ def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session ) carrier = Trace.inject() # Update the context_carrier and leave the SpanStatus as ACTIVE. - dag_run.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) + dag_run.context_carrier = carrier self.active_spans.set(dag_run.run_id, dr_span) tis = dag_run.get_task_instances(session=session) @@ -1198,14 +1194,14 @@ def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session start_as_current=False, ) ti_carrier = Trace.inject() - ti.set_context_carrier(context_carrier=ti_carrier, session=session, with_commit=False) + ti.context_carrier = ti_carrier if ti.state in State.finished: self.set_ti_span_attrs(span=ti_span, state=ti.state, ti=ti) ti_span.end(end_time=datetime_to_nano(ti.end_date)) - ti.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + ti.span_status = SpanStatus.ENDED else: - ti.set_span_status(status=SpanStatus.ACTIVE, session=session, with_commit=False) + ti.span_status = SpanStatus.ACTIVE self.active_spans.set(ti.key, ti_span) def _run_scheduler_loop(self) -> None: diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index e63e3ad0778be..0c980f8166c2d 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -1073,8 +1073,8 @@ def recalculate(self) -> _UnfinishedStates: # New task span will use this span as the parent. continue_ti_spans = True carrier = Trace.inject() - self.set_context_carrier(context_carrier=carrier, session=session, with_commit=False) - self.set_span_status(status=SpanStatus.ACTIVE, session=session, with_commit=False) + self.context_carrier = carrier + self.span_status = SpanStatus.ACTIVE # Set the span in a synchronized dictionary, so that the variable can be used to end the span. self.active_spans.set(self.run_id, dr_span) self.log.debug( @@ -1092,12 +1092,8 @@ def recalculate(self) -> _UnfinishedStates: start_as_current=False, ) ti_carrier = Trace.inject() - ti.set_context_carrier( - context_carrier=ti_carrier, session=session, with_commit=False - ) - ti.set_span_status( - status=SpanStatus.ACTIVE, session=session, with_commit=False - ) + ti.context_carrier = ti_carrier + ti.span_status = SpanStatus.ACTIVE self.active_spans.set(ti.key, ti_span) else: self.log.info( @@ -1151,19 +1147,19 @@ def recalculate(self) -> _UnfinishedStates: active_span.end(end_time=datetime_to_nano(self.end_date)) # Remove the span from the dict. self.active_spans.delete(self.run_id) - self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + self.span_status = SpanStatus.ENDED else: if self.span_status == SpanStatus.ACTIVE: # Another scheduler has started the span. # Update the DB SpanStatus to notify the owner to end it. - self.set_span_status(status=SpanStatus.SHOULD_END, session=session, with_commit=False) + self.span_status = SpanStatus.SHOULD_END elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: # This is a corner case where the scheduler exited gracefully # while the dag_run was almost done. # Since it reached this point, the dag has finished but there has been no time # to create a new span for the current scheduler. # There is no need for more spans, update the status on the db. - self.set_span_status(status=SpanStatus.ENDED, session=session, with_commit=False) + self.span_status = SpanStatus.ENDED else: self.log.debug( "No active span has been found for dag_id: %s, run_id: %s, state: %s", @@ -1232,123 +1228,6 @@ def _filter_tis_and_exclude_removed(dag: DAG, tis: list[TI]) -> Iterable[TI]: finished_tis=finished_tis, ) - @staticmethod - def _set_scheduled_by_job_id(dag_run: DagRun, job_id: int, session: Session, with_commit: bool) -> bool: - if not isinstance(dag_run, DagRun): - dag_run = session.scalars( - select(DagRun).where( - DagRun.dag_id == dag_run.dag_id, - DagRun.run_id == dag_run.run_id, - ) - ).one() - - if dag_run.scheduled_by_job_id == job_id: - return False - - dag_run.log.debug("Setting dag_run scheduled_by_job_id for run_id: %s", dag_run.run_id) - dag_run.scheduled_by_job_id = job_id - - session.merge(dag_run) - - if with_commit: - session.commit() - - return True - - @provide_session - def set_scheduled_by_job_id( - self, job_id: int, session: Session = NEW_SESSION, with_commit: bool = False - ) -> bool: - """ - Set DagRun scheduled_by_job_id. - - :param job_id: integer with the scheduled_by_job_id to set for the dag_run - :param session: SQLAlchemy ORM Session - :param with_commit: should the scheduled_by_job_id be committed? - :return: has the scheduled_by_job_id been changed? - """ - return self._set_scheduled_by_job_id( - dag_run=self, job_id=job_id, session=session, with_commit=with_commit - ) - - @staticmethod - def _set_context_carrier( - dag_run: DagRun, context_carrier: dict, session: Session, with_commit: bool - ) -> bool: - if not isinstance(dag_run, DagRun): - dag_run = session.scalars( - select(DagRun).where( - DagRun.dag_id == dag_run.dag_id, - DagRun.run_id == dag_run.run_id, - ) - ).one() - - if dag_run.context_carrier == context_carrier: - return False - - dag_run.log.debug("Setting dag_run context_carrier for run_id: %s", dag_run.run_id) - dag_run.context_carrier = context_carrier - - session.merge(dag_run) - - if with_commit: - session.commit() - - return True - - @provide_session - def set_context_carrier( - self, context_carrier: dict, session: Session = NEW_SESSION, with_commit: bool = False - ) -> bool: - """ - Set DagRun span context_carrier. - - :param context_carrier: dict with the injected carrier to set for the dag_run - :param session: SQLAlchemy ORM Session - :param with_commit: should the carrier be committed? - :return: has the context_carrier been changed? - """ - return self._set_context_carrier( - dag_run=self, context_carrier=context_carrier, session=session, with_commit=with_commit - ) - - @staticmethod - def _set_span_status(dag_run: DagRun, status: SpanStatus, session: Session, with_commit: bool) -> bool: - if not isinstance(dag_run, DagRun): - dag_run = session.scalars( - select(DagRun).where( - DagRun.dag_id == dag_run.dag_id, - DagRun.run_id == dag_run.run_id, - ) - ).one() - - if dag_run.span_status == status: - return False - - dag_run.log.debug("Setting dag_run span_status for run_id: %s", dag_run.run_id) - dag_run.span_status = status - - session.merge(dag_run) - - if with_commit: - session.commit() - - return True - - @provide_session - def set_span_status( - self, status: SpanStatus, session: Session = NEW_SESSION, with_commit: bool = False - ) -> bool: - """ - Set DagRun span_status. - - :param status: dict with the injected carrier to set for the dag_run - :param session: SQLAlchemy ORM Session - :param with_commit: should the status be committed? - :return: has the span_status been changed? - """ - return self._set_span_status(dag_run=self, status=status, session=session, with_commit=with_commit) - def notify_dagrun_state_changed(self, msg: str = ""): if self.state == DagRunState.RUNNING: get_listener_manager().hook.on_dag_run_running(dag_run=self, msg=msg) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index d59880e910350..5229f1d2ea508 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -2209,84 +2209,6 @@ def set_state(self, state: str | None, session: Session = NEW_SESSION) -> bool: """ return self._set_state(ti=self, state=state, session=session) - @staticmethod - def _set_context_carrier(ti: TaskInstance, context_carrier, session: Session, with_commit: bool) -> bool: - if not isinstance(ti, TaskInstance): - ti = session.scalars( - select(TaskInstance).where( - TaskInstance.task_id == ti.task_id, - TaskInstance.dag_id == ti.dag_id, - TaskInstance.run_id == ti.run_id, - ) - ).one() - - if ti.context_carrier == context_carrier: - return False - - ti.log.debug("Setting task context_carrier for %s", ti.task_id) - ti.context_carrier = context_carrier - - session.merge(ti) - - if with_commit: - session.commit() - - return True - - @provide_session - def set_context_carrier( - self, context_carrier: dict, session: Session = NEW_SESSION, with_commit: bool = False - ) -> bool: - """ - Set TaskInstance span context_carrier. - - :param context_carrier: dict with the injected carrier to set for the TI - :param session: SQLAlchemy ORM Session - :param with_commit: should the carrier be committed? - :return: has the context_carrier been changed? - """ - return self._set_context_carrier( - ti=self, context_carrier=context_carrier, session=session, with_commit=with_commit - ) - - @staticmethod - def _set_span_status(ti: TaskInstance, status: SpanStatus, session: Session, with_commit: bool) -> bool: - if not isinstance(ti, TaskInstance): - ti = session.scalars( - select(TaskInstance).where( - TaskInstance.task_id == ti.task_id, - TaskInstance.dag_id == ti.dag_id, - TaskInstance.run_id == ti.run_id, - ) - ).one() - - if ti.span_status == status: - return False - - ti.log.debug("Setting task span_status for %s", ti.task_id) - ti.span_status = status - - session.merge(ti) - - if with_commit: - session.commit() - - return True - - @provide_session - def set_span_status( - self, status: SpanStatus, session: Session = NEW_SESSION, with_commit: bool = False - ) -> bool: - """ - Set TaskInstance span_status. - - :param status: dict with the injected carrier to set for the TI - :param session: SQLAlchemy ORM Session - :param with_commit: should the status be committed? - :return: has the span_status been changed? - """ - return self._set_span_status(ti=self, status=status, session=session, with_commit=with_commit) - @property def is_premature(self) -> bool: """Returns whether a task is in UP_FOR_RETRY state and its retry interval has elapsed.""" @@ -3819,7 +3741,7 @@ def from_ti(cls, ti: TaskInstance) -> SimpleTaskInstance: if "dag_run" not in inspect(ti).unloaded else None, context_carrier=ti.context_carrier if hasattr(ti, "context_carrier") else None, - span_status=ti.span_status if hasattr(ti, "span_status") else None, + span_status=ti.span_status, ) diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index b27b78abd7353..fa96749ad9ea5 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -2482,7 +2482,6 @@ def test_recreate_unhealthy_scheduler_spans_if_needed(self, ti_state, final_ti_s assert self.job_runner.active_spans.get(ti.key) is None assert ti.state == ti_state assert ti.span_status == SpanStatus.ACTIVE - assert ti.context_carrier == {} # When self.job_runner._recreate_unhealthy_scheduler_spans_if_needed(dr, session) From b3f544ab6e6b23051f378057d05188936ea71249 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Mon, 16 Dec 2024 17:34:47 +0200 Subject: [PATCH 30/52] add a migration file --- .../0052_3_0_0_add_new_otel_span_fields.py | 58 + airflow/utils/db.py | 2 +- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- docs/apache-airflow/img/airflow_erd.svg | 2198 +++++++++-------- docs/apache-airflow/migrations-ref.rst | 4 +- 5 files changed, 1176 insertions(+), 1088 deletions(-) create mode 100644 airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py diff --git a/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py new file mode 100644 index 0000000000000..55013fa41cd5d --- /dev/null +++ b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py @@ -0,0 +1,58 @@ +# +# 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. + +""" +Add new otel span fields. + +Revision ID: 0eb040b3eb12 +Revises: 038dc8bc6284 +Create Date: 2024-12-16 15:08:27.304594 + +""" + +from __future__ import annotations + +import sqlalchemy as sa +from alembic import op + +# revision identifiers, used by Alembic. +revision = "0eb040b3eb12" +down_revision = "038dc8bc6284" +branch_labels = None +depends_on = None +airflow_version = "3.0.0" + + +def upgrade(): + """Apply add new otel span fields.""" + op.add_column("dag_run", sa.Column("scheduled_by_job_id", sa.Integer, nullable=True)) + op.add_column("dag_run", sa.Column("context_carrier", sa.String(2000), nullable=False)) + op.add_column("dag_run", sa.Column("span_status", sa.String(2000), nullable=False)) + + op.add_column("task_instance", sa.Column("context_carrier", sa.String(2000), nullable=False)) + op.add_column("task_instance", sa.Column("span_status", sa.String(2000), nullable=False)) + + +def downgrade(): + """Unapply add new otel span fields.""" + op.drop_column("dag_run", "scheduled_by_job_id") + op.drop_column("dag_run", "context_carrier") + op.drop_column("dag_run", "span_status") + + op.drop_column("task_instance", "context_carrier") + op.drop_column("task_instance", "span_status") diff --git a/airflow/utils/db.py b/airflow/utils/db.py index 3cd3c206a66d2..8a7bf7d568eda 100644 --- a/airflow/utils/db.py +++ b/airflow/utils/db.py @@ -94,7 +94,7 @@ class MappedClassProtocol(Protocol): "2.9.2": "686269002441", "2.10.0": "22ed7efa9da2", "2.10.3": "5f2621c13b39", - "3.0.0": "038dc8bc6284", + "3.0.0": "0eb040b3eb12", } diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index 6ee5972cc1624..642bfc8a85ff5 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -ccb8ef5583b2a6b3ee3ab4212139c112b92953675655010a6775fffb4945b206 \ No newline at end of file +250b25e09d2b7ef2ef650dff6dd23c15eb7067dd6d924ce83ddcef6464c6d5c5 \ No newline at end of file diff --git a/docs/apache-airflow/img/airflow_erd.svg b/docs/apache-airflow/img/airflow_erd.svg index 9c37f5c320686..38a5843a5631c 100644 --- a/docs/apache-airflow/img/airflow_erd.svg +++ b/docs/apache-airflow/img/airflow_erd.svg @@ -4,11 +4,11 @@ - - + + %3 - + log @@ -105,8 +105,8 @@ callback_data - [JSON] - NOT NULL + [JSONB] + NOT NULL callback_type @@ -604,74 +604,74 @@ asset_event - -asset_event - -id - - [INTEGER] - NOT NULL - -asset_id - - [INTEGER] - NOT NULL - -extra - - [JSON] - NOT NULL - -source_dag_id - - [VARCHAR(250)] - -source_map_index - - [INTEGER] - -source_run_id - - [VARCHAR(250)] - -source_task_id - - [VARCHAR(250)] - -timestamp - - [TIMESTAMP] - NOT NULL + +asset_event + +id + + [INTEGER] + NOT NULL + +asset_id + + [INTEGER] + NOT NULL + +extra + + [JSON] + NOT NULL + +source_dag_id + + [VARCHAR(250)] + +source_map_index + + [INTEGER] + +source_run_id + + [VARCHAR(250)] + +source_task_id + + [VARCHAR(250)] + +timestamp + + [TIMESTAMP] + NOT NULL asset_event--asset_alias_asset_event - -0..N -1 + +0..N +1 dagrun_asset_event - -dagrun_asset_event - -dag_run_id - - [INTEGER] - NOT NULL - -event_id - - [INTEGER] - NOT NULL + +dagrun_asset_event + +dag_run_id + + [INTEGER] + NOT NULL + +event_id + + [INTEGER] + NOT NULL asset_event--dagrun_asset_event - -0..N -1 + +0..N +1 @@ -713,687 +713,703 @@ task_instance - -task_instance - -id - - [UUID] - NOT NULL - -custom_operator_name - - [VARCHAR(1000)] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -last_heartbeat_at - - [TIMESTAMP] - -map_index - - [INTEGER] - NOT NULL - -max_tries - - [INTEGER] - -next_kwargs - - [JSON] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -task_id - - [VARCHAR(250)] - NOT NULL - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance + +id + + [UUID] + NOT NULL + +context_carrier + + [JSONB] + +custom_operator_name + + [VARCHAR(1000)] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +last_heartbeat_at + + [TIMESTAMP] + +map_index + + [INTEGER] + NOT NULL + +max_tries + + [INTEGER] + +next_kwargs + + [JSONB] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +run_id + + [VARCHAR(250)] + NOT NULL + +span_status + + [VARCHAR(50)] + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +task_id + + [VARCHAR(250)] + NOT NULL + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] trigger--task_instance - -0..N + +0..N {0,1} task_reschedule - -task_reschedule - -id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -duration - - [INTEGER] - NOT NULL - -end_date - - [TIMESTAMP] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -reschedule_date - - [TIMESTAMP] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -try_number - - [INTEGER] - NOT NULL + +task_reschedule + +id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +duration + + [INTEGER] + NOT NULL + +end_date + + [TIMESTAMP] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +reschedule_date + + [TIMESTAMP] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +try_number + + [INTEGER] + NOT NULL task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 rendered_task_instance_fields - -rendered_task_instance_fields - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -k8s_pod_yaml - - [JSON] - -rendered_fields - - [JSON] - NOT NULL + +rendered_task_instance_fields + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +k8s_pod_yaml + + [JSON] + +rendered_fields + + [JSON] + NOT NULL task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_map - -task_map - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -keys - - [JSON] - -length - - [INTEGER] - NOT NULL + +task_map + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +keys + + [JSONB] + +length + + [INTEGER] + NOT NULL task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 xcom - -xcom - -dag_run_id - - [INTEGER] - NOT NULL - -key - - [VARCHAR(512)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -timestamp - - [TIMESTAMP] - NOT NULL - -value - - [JSONB] + +xcom + +dag_run_id + + [INTEGER] + NOT NULL + +key + + [VARCHAR(512)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +timestamp + + [TIMESTAMP] + NOT NULL + +value + + [JSONB] task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance_note - -task_instance_note - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] + +task_instance_note + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [VARCHAR(128)] task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance_history - -task_instance_history - -id - - [INTEGER] - NOT NULL - -custom_operator_name - - [VARCHAR(1000)] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -map_index - - [INTEGER] - NOT NULL - -max_tries - - [INTEGER] - -next_kwargs - - [JSON] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -task_id - - [VARCHAR(250)] - NOT NULL - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - NOT NULL - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance_history + +id + + [INTEGER] + NOT NULL + +context_carrier + + [JSONB] + +custom_operator_name + + [VARCHAR(1000)] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +map_index + + [INTEGER] + NOT NULL + +max_tries + + [INTEGER] + +next_kwargs + + [JSONB] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +run_id + + [VARCHAR(250)] + NOT NULL + +span_status + + [VARCHAR(50)] + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +task_id + + [VARCHAR(250)] + NOT NULL + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + NOT NULL + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 @@ -1413,7 +1429,7 @@ kwargs - [JSON] + [JSONB] last_refreshed @@ -1697,89 +1713,101 @@ dag_version--task_instance - -0..N + +0..N {0,1} dag_run - -dag_run + +dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + +clear_number + + [INTEGER] + NOT NULL -id - - [INTEGER] - NOT NULL +conf + + [BYTEA] -backfill_id - - [INTEGER] +context_carrier + + [JSONB] -clear_number - - [INTEGER] - NOT NULL +creating_job_id + + [INTEGER] -conf - - [BYTEA] +dag_id + + [VARCHAR(250)] + NOT NULL -creating_job_id +dag_version_id - [INTEGER] + [UUID] -dag_id - - [VARCHAR(250)] - NOT NULL +data_interval_end + + [TIMESTAMP] -dag_version_id - - [UUID] +data_interval_start + + [TIMESTAMP] -data_interval_end - - [TIMESTAMP] +end_date + + [TIMESTAMP] -data_interval_start - - [TIMESTAMP] +external_trigger + + [BOOLEAN] -end_date - - [TIMESTAMP] +last_scheduling_decision + + [TIMESTAMP] -external_trigger - - [BOOLEAN] +log_template_id + + [INTEGER] -last_scheduling_decision - - [TIMESTAMP] +logical_date + + [TIMESTAMP] + NOT NULL -log_template_id - - [INTEGER] +queued_at + + [TIMESTAMP] -logical_date - - [TIMESTAMP] - NOT NULL +run_id + + [VARCHAR(250)] + NOT NULL -queued_at - - [TIMESTAMP] +run_type + + [VARCHAR(50)] + NOT NULL -run_id - - [VARCHAR(250)] - NOT NULL +scheduled_by_job_id + + [INTEGER] -run_type - - [VARCHAR(50)] - NOT NULL +span_status + + [VARCHAR(50)] start_date @@ -1800,9 +1828,9 @@ dag_version--dag_run - -0..N -{0,1} + +0..N +{0,1} @@ -1850,7 +1878,7 @@ dag_version--dag_code 0..N -1 +1 @@ -1905,536 +1933,536 @@ dag_run--dagrun_asset_event - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 backfill_dag_run - -backfill_dag_run - -id - - [INTEGER] - NOT NULL - -backfill_id - - [INTEGER] - NOT NULL - -dag_run_id - - [INTEGER] - -exception_reason - - [VARCHAR(250)] - -logical_date - - [TIMESTAMP] - NOT NULL - -sort_ordinal - - [INTEGER] - NOT NULL + +backfill_dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + NOT NULL + +dag_run_id + + [INTEGER] + +exception_reason + + [VARCHAR(250)] + +logical_date + + [TIMESTAMP] + NOT NULL + +sort_ordinal + + [INTEGER] + NOT NULL dag_run--backfill_dag_run - -0..N -{0,1} + +0..N +{0,1} dag_run_note - -dag_run_note - -dag_run_id - - [INTEGER] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] + +dag_run_note + +dag_run_id + + [INTEGER] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [VARCHAR(128)] dag_run--dag_run_note - -1 -1 + +1 +1 dag_run--task_reschedule - -0..N -1 + +0..N +1 dag_run--task_reschedule - -0..N -1 + +0..N +1 log_template - -log_template - -id - - [INTEGER] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -elasticsearch_id - - [TEXT] - NOT NULL - -filename - - [TEXT] - NOT NULL + +log_template + +id + + [INTEGER] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +elasticsearch_id + + [TEXT] + NOT NULL + +filename + + [TEXT] + NOT NULL log_template--dag_run - -0..N -{0,1} + +0..N +{0,1} backfill - -backfill - -id - - [INTEGER] - NOT NULL - -completed_at - - [TIMESTAMP] - -created_at - - [TIMESTAMP] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_run_conf - - [JSON] - NOT NULL - -from_date - - [TIMESTAMP] - NOT NULL - -is_paused - - [BOOLEAN] - -max_active_runs - - [INTEGER] - NOT NULL - -reprocess_behavior - - [VARCHAR(250)] - NOT NULL - -to_date - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +backfill + +id + + [INTEGER] + NOT NULL + +completed_at + + [TIMESTAMP] + +created_at + + [TIMESTAMP] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_run_conf + + [JSON] + NOT NULL + +from_date + + [TIMESTAMP] + NOT NULL + +is_paused + + [BOOLEAN] + +max_active_runs + + [INTEGER] + NOT NULL + +reprocess_behavior + + [VARCHAR(250)] + NOT NULL + +to_date + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL backfill--dag_run - -0..N -{0,1} + +0..N +{0,1} backfill--backfill_dag_run - -0..N -1 + +0..N +1 session - -session - -id - - [INTEGER] - NOT NULL - -data - - [BYTEA] - -expiry - - [TIMESTAMP] - -session_id - - [VARCHAR(255)] + +session + +id + + [INTEGER] + NOT NULL + +data + + [BYTEA] + +expiry + + [TIMESTAMP] + +session_id + + [VARCHAR(255)] alembic_version - -alembic_version - -version_num - - [VARCHAR(32)] - NOT NULL + +alembic_version + +version_num + + [VARCHAR(32)] + NOT NULL ab_user - -ab_user - -id - - [INTEGER] - NOT NULL - -active - - [BOOLEAN] - -changed_by_fk - - [INTEGER] - -changed_on - - [TIMESTAMP] - -created_by_fk - - [INTEGER] - -created_on - - [TIMESTAMP] - -email - - [VARCHAR(512)] - NOT NULL - -fail_login_count - - [INTEGER] - -first_name - - [VARCHAR(256)] - NOT NULL - -last_login - - [TIMESTAMP] - -last_name - - [VARCHAR(256)] - NOT NULL - -login_count - - [INTEGER] - -password - - [VARCHAR(256)] - -username - - [VARCHAR(512)] - NOT NULL + +ab_user + +id + + [INTEGER] + NOT NULL + +active + + [BOOLEAN] + +changed_by_fk + + [INTEGER] + +changed_on + + [TIMESTAMP] + +created_by_fk + + [INTEGER] + +created_on + + [TIMESTAMP] + +email + + [VARCHAR(512)] + NOT NULL + +fail_login_count + + [INTEGER] + +first_name + + [VARCHAR(256)] + NOT NULL + +last_login + + [TIMESTAMP] + +last_name + + [VARCHAR(256)] + NOT NULL + +login_count + + [INTEGER] + +password + + [VARCHAR(256)] + +username + + [VARCHAR(512)] + NOT NULL ab_user--ab_user - -0..N -{0,1} + +0..N +{0,1} ab_user--ab_user - -0..N -{0,1} + +0..N +{0,1} ab_user_role - -ab_user_role - -id - - [INTEGER] - NOT NULL - -role_id - - [INTEGER] - -user_id - - [INTEGER] + +ab_user_role + +id + + [INTEGER] + NOT NULL + +role_id + + [INTEGER] + +user_id + + [INTEGER] ab_user--ab_user_role - -0..N -{0,1} + +0..N +{0,1} ab_register_user - -ab_register_user - -id - - [INTEGER] - NOT NULL - -email - - [VARCHAR(512)] - NOT NULL - -first_name - - [VARCHAR(256)] - NOT NULL - -last_name - - [VARCHAR(256)] - NOT NULL - -password - - [VARCHAR(256)] - -registration_date - - [TIMESTAMP] - -registration_hash - - [VARCHAR(256)] - -username - - [VARCHAR(512)] - NOT NULL + +ab_register_user + +id + + [INTEGER] + NOT NULL + +email + + [VARCHAR(512)] + NOT NULL + +first_name + + [VARCHAR(256)] + NOT NULL + +last_name + + [VARCHAR(256)] + NOT NULL + +password + + [VARCHAR(256)] + +registration_date + + [TIMESTAMP] + +registration_hash + + [VARCHAR(256)] + +username + + [VARCHAR(512)] + NOT NULL ab_permission - -ab_permission - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(100)] - NOT NULL + +ab_permission + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(100)] + NOT NULL ab_permission_view - -ab_permission_view - -id - - [INTEGER] - NOT NULL - -permission_id - - [INTEGER] - -view_menu_id - - [INTEGER] + +ab_permission_view + +id + + [INTEGER] + NOT NULL + +permission_id + + [INTEGER] + +view_menu_id + + [INTEGER] ab_permission--ab_permission_view - -0..N -{0,1} + +0..N +{0,1} ab_permission_view_role - -ab_permission_view_role - -id - - [INTEGER] - NOT NULL - -permission_view_id - - [INTEGER] - -role_id - - [INTEGER] + +ab_permission_view_role + +id + + [INTEGER] + NOT NULL + +permission_view_id + + [INTEGER] + +role_id + + [INTEGER] ab_permission_view--ab_permission_view_role - -0..N -{0,1} + +0..N +{0,1} ab_view_menu - -ab_view_menu - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(250)] - NOT NULL + +ab_view_menu + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(250)] + NOT NULL ab_view_menu--ab_permission_view - -0..N -{0,1} + +0..N +{0,1} ab_role - -ab_role - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(64)] - NOT NULL + +ab_role + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(64)] + NOT NULL ab_role--ab_user_role - -0..N -{0,1} + +0..N +{0,1} ab_role--ab_permission_view_role - -0..N -{0,1} + +0..N +{0,1} alembic_version_fab - -alembic_version_fab - -version_num - - [VARCHAR(32)] - NOT NULL + +alembic_version_fab + +version_num + + [VARCHAR(32)] + NOT NULL diff --git a/docs/apache-airflow/migrations-ref.rst b/docs/apache-airflow/migrations-ref.rst index edd166e0bf418..14da68cdc88e3 100644 --- a/docs/apache-airflow/migrations-ref.rst +++ b/docs/apache-airflow/migrations-ref.rst @@ -39,7 +39,9 @@ Here's the list of all the Database Migrations that are executed via when you ru +-------------------------+------------------+-------------------+--------------------------------------------------------------+ | Revision ID | Revises ID | Airflow Version | Description | +=========================+==================+===================+==============================================================+ -| ``038dc8bc6284`` (head) | ``e229247a6cb1`` | ``3.0.0`` | update trigger_timeout column in task_instance table to UTC. | +| ``0eb040b3eb12`` (head) | ``038dc8bc6284`` | ``3.0.0`` | Add new otel span fields. | ++-------------------------+------------------+-------------------+--------------------------------------------------------------+ +| ``038dc8bc6284`` | ``e229247a6cb1`` | ``3.0.0`` | update trigger_timeout column in task_instance table to UTC. | +-------------------------+------------------+-------------------+--------------------------------------------------------------+ | ``e229247a6cb1`` | ``eed27faa34e3`` | ``3.0.0`` | Add DagBundleModel. | +-------------------------+------------------+-------------------+--------------------------------------------------------------+ From 0c5e45d041921cb4f8fb8d1601504b1dd6c4fb17 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Mon, 16 Dec 2024 19:21:55 +0200 Subject: [PATCH 31/52] fix failing tests --- .../versions/0052_3_0_0_add_new_otel_span_fields.py | 4 ++-- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- tests/api_fastapi/common/test_exceptions.py | 6 +++--- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py index 55013fa41cd5d..afb7d03c5ad64 100644 --- a/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py +++ b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py @@ -41,10 +41,10 @@ def upgrade(): """Apply add new otel span fields.""" op.add_column("dag_run", sa.Column("scheduled_by_job_id", sa.Integer, nullable=True)) - op.add_column("dag_run", sa.Column("context_carrier", sa.String(2000), nullable=False)) + op.add_column("dag_run", sa.Column("context_carrier", sa.JSON, nullable=True)) op.add_column("dag_run", sa.Column("span_status", sa.String(2000), nullable=False)) - op.add_column("task_instance", sa.Column("context_carrier", sa.String(2000), nullable=False)) + op.add_column("task_instance", sa.Column("context_carrier", sa.JSON, nullable=True)) op.add_column("task_instance", sa.Column("span_status", sa.String(2000), nullable=False)) diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index 642bfc8a85ff5..a6cd250b1c9cd 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -250b25e09d2b7ef2ef650dff6dd23c15eb7067dd6d924ce83ddcef6464c6d5c5 \ No newline at end of file +df8698effe4faa1460f6673dc8e301ef3f8d16cdf46e8133b16a9450cd5497b2 \ No newline at end of file diff --git a/tests/api_fastapi/common/test_exceptions.py b/tests/api_fastapi/common/test_exceptions.py index e296f2320430c..3304503bec458 100644 --- a/tests/api_fastapi/common/test_exceptions.py +++ b/tests/api_fastapi/common/test_exceptions.py @@ -186,7 +186,7 @@ def test_handle_single_column_unique_constraint_error(self, session, table, expe status_code=status.HTTP_409_CONFLICT, detail={ "reason": "Unique constraint violation", - "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, (SELECT max(log_template.id) AS max_1 \nFROM log_template), ?, ?, ?, ?)", + "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id, scheduled_by_job_id, context_carrier, span_status) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, (SELECT max(log_template.id) AS max_1 \nFROM log_template), ?, ?, ?, ?, ?, ?, ?)", "orig_error": "UNIQUE constraint failed: dag_run.dag_id, dag_run.run_id", }, ), @@ -194,7 +194,7 @@ def test_handle_single_column_unique_constraint_error(self, session, table, expe status_code=status.HTTP_409_CONFLICT, detail={ "reason": "Unique constraint violation", - "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id) VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, (SELECT max(log_template.id) AS max_1 \nFROM log_template), %s, %s, %s, %s)", + "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id, scheduled_by_job_id, context_carrier, span_status) VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, (SELECT max(log_template.id) AS max_1 \nFROM log_template), %s, %s, %s, %s, %s, %s, %s)", "orig_error": "(1062, \"Duplicate entry 'test_dag_id-test_run_id' for key 'dag_run.dag_run_dag_id_run_id_key'\")", }, ), @@ -202,7 +202,7 @@ def test_handle_single_column_unique_constraint_error(self, session, table, expe status_code=status.HTTP_409_CONFLICT, detail={ "reason": "Unique constraint violation", - "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id) VALUES (%(dag_id)s, %(queued_at)s, %(logical_date)s, %(start_date)s, %(end_date)s, %(state)s, %(run_id)s, %(creating_job_id)s, %(external_trigger)s, %(run_type)s, %(triggered_by)s, %(conf)s, %(data_interval_start)s, %(data_interval_end)s, %(last_scheduling_decision)s, (SELECT max(log_template.id) AS max_1 \nFROM log_template), %(updated_at)s, %(clear_number)s, %(backfill_id)s, %(dag_version_id)s) RETURNING dag_run.id", + "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id, scheduled_by_job_id, context_carrier, span_status) VALUES (%(dag_id)s, %(queued_at)s, %(logical_date)s, %(start_date)s, %(end_date)s, %(state)s, %(run_id)s, %(creating_job_id)s, %(external_trigger)s, %(run_type)s, %(triggered_by)s, %(conf)s, %(data_interval_start)s, %(data_interval_end)s, %(last_scheduling_decision)s, (SELECT max(log_template.id) AS max_1 \nFROM log_template), %(updated_at)s, %(clear_number)s, %(backfill_id)s, %(dag_version_id)s, %(scheduled_by_job_id)s, %(context_carrier)s, %(span_status)s) RETURNING dag_run.id", "orig_error": 'duplicate key value violates unique constraint "dag_run_dag_id_run_id_key"\nDETAIL: Key (dag_id, run_id)=(test_dag_id, test_run_id) already exists.\n', }, ), From db481a1024583fdbc287848ec7518bb6a09d3395 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Mon, 16 Dec 2024 20:35:00 +0200 Subject: [PATCH 32/52] fix discrepancies between model and migration file --- .../0052_3_0_0_add_new_otel_span_fields.py | 10 +- airflow/models/dagrun.py | 2 +- airflow/models/taskinstance.py | 2 +- airflow/models/taskinstancehistory.py | 2 +- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- docs/apache-airflow/img/airflow_erd.svg | 257 +++++++++--------- 6 files changed, 140 insertions(+), 135 deletions(-) diff --git a/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py index afb7d03c5ad64..9e6f99b7fa3f8 100644 --- a/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py +++ b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py @@ -30,6 +30,8 @@ import sqlalchemy as sa from alembic import op +from airflow.utils.sqlalchemy import ExtendedJSON + # revision identifiers, used by Alembic. revision = "0eb040b3eb12" down_revision = "038dc8bc6284" @@ -41,11 +43,11 @@ def upgrade(): """Apply add new otel span fields.""" op.add_column("dag_run", sa.Column("scheduled_by_job_id", sa.Integer, nullable=True)) - op.add_column("dag_run", sa.Column("context_carrier", sa.JSON, nullable=True)) - op.add_column("dag_run", sa.Column("span_status", sa.String(2000), nullable=False)) + op.add_column("dag_run", sa.Column("context_carrier", ExtendedJSON, nullable=True)) + op.add_column("dag_run", sa.Column("span_status", sa.String(250), nullable=False)) - op.add_column("task_instance", sa.Column("context_carrier", sa.JSON, nullable=True)) - op.add_column("task_instance", sa.Column("span_status", sa.String(2000), nullable=False)) + op.add_column("task_instance", sa.Column("context_carrier", ExtendedJSON, nullable=True)) + op.add_column("task_instance", sa.Column("span_status", sa.String(250), nullable=False)) def downgrade(): diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 0c980f8166c2d..67e3f7d9d6d64 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -188,7 +188,7 @@ class DagRun(Base, LoggingMixin): scheduled_by_job_id = Column(Integer) # Span context carrier, used for context propagation. context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) - span_status = Column(String(50), default=SpanStatus.NOT_STARTED) + span_status = Column(String(250), default=SpanStatus.NOT_STARTED, nullable=False) # Remove this `if` after upgrading Sphinx-AutoAPI if not TYPE_CHECKING and "BUILDING_AIRFLOW_DOCS" in os.environ: diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index a8551e13aa947..b41bc3821c031 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -1701,7 +1701,7 @@ class TaskInstance(Base, LoggingMixin): updated_at = Column(UtcDateTime, default=timezone.utcnow, onupdate=timezone.utcnow) rendered_map_index = Column(String(250)) context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) - span_status = Column(String(50), default=SpanStatus.NOT_STARTED) + span_status = Column(String(250), default=SpanStatus.NOT_STARTED, nullable=False) external_executor_id = Column(StringID()) diff --git a/airflow/models/taskinstancehistory.py b/airflow/models/taskinstancehistory.py index 2e6ca19abc045..01a930a7f7c34 100644 --- a/airflow/models/taskinstancehistory.py +++ b/airflow/models/taskinstancehistory.py @@ -85,7 +85,7 @@ class TaskInstanceHistory(Base): updated_at = Column(UtcDateTime, default=timezone.utcnow, onupdate=timezone.utcnow) rendered_map_index = Column(String(250)) context_carrier = Column(MutableDict.as_mutable(ExtendedJSON)) - span_status = Column(String(50), default=SpanStatus.NOT_STARTED) + span_status = Column(String(250), default=SpanStatus.NOT_STARTED, nullable=False) external_executor_id = Column(StringID()) trigger_id = Column(Integer) diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index a6cd250b1c9cd..9fa3bdd43e959 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -df8698effe4faa1460f6673dc8e301ef3f8d16cdf46e8133b16a9450cd5497b2 \ No newline at end of file +6c8eb567912ecc86ed1cba779a38e7662031e9384d84991a1a219d28363ae3df \ No newline at end of file diff --git a/docs/apache-airflow/img/airflow_erd.svg b/docs/apache-airflow/img/airflow_erd.svg index 38a5843a5631c..bfcc2acfda9e1 100644 --- a/docs/apache-airflow/img/airflow_erd.svg +++ b/docs/apache-airflow/img/airflow_erd.svg @@ -828,7 +828,8 @@ span_status - [VARCHAR(50)] + [VARCHAR(250)] + NOT NULL start_date @@ -1343,7 +1344,8 @@ span_status - [VARCHAR(50)] + [VARCHAR(250)] + NOT NULL start_date @@ -1720,110 +1722,111 @@ dag_run - -dag_run - -id - - [INTEGER] - NOT NULL - -backfill_id - - [INTEGER] - -clear_number - - [INTEGER] - NOT NULL - -conf - - [BYTEA] - -context_carrier - - [JSONB] - -creating_job_id - - [INTEGER] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -data_interval_end - - [TIMESTAMP] - -data_interval_start - - [TIMESTAMP] - -end_date - - [TIMESTAMP] - -external_trigger - - [BOOLEAN] - -last_scheduling_decision - - [TIMESTAMP] - -log_template_id - - [INTEGER] - -logical_date - - [TIMESTAMP] - NOT NULL - -queued_at - - [TIMESTAMP] - -run_id - - [VARCHAR(250)] - NOT NULL - -run_type - - [VARCHAR(50)] - NOT NULL - -scheduled_by_job_id - - [INTEGER] - -span_status - - [VARCHAR(50)] - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(50)] - -triggered_by - - [VARCHAR(50)] - -updated_at - - [TIMESTAMP] + +dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + +clear_number + + [INTEGER] + NOT NULL + +conf + + [BYTEA] + +context_carrier + + [JSONB] + +creating_job_id + + [INTEGER] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +data_interval_end + + [TIMESTAMP] + +data_interval_start + + [TIMESTAMP] + +end_date + + [TIMESTAMP] + +external_trigger + + [BOOLEAN] + +last_scheduling_decision + + [TIMESTAMP] + +log_template_id + + [INTEGER] + +logical_date + + [TIMESTAMP] + NOT NULL + +queued_at + + [TIMESTAMP] + +run_id + + [VARCHAR(250)] + NOT NULL + +run_type + + [VARCHAR(50)] + NOT NULL + +scheduled_by_job_id + + [INTEGER] + +span_status + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(50)] + +triggered_by + + [VARCHAR(50)] + +updated_at + + [TIMESTAMP] @@ -1933,23 +1936,23 @@ dag_run--dagrun_asset_event - + 0..N -1 +1 dag_run--task_instance - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 @@ -1988,9 +1991,9 @@ dag_run--backfill_dag_run - + 0..N -{0,1} +{0,1} @@ -2024,23 +2027,23 @@ dag_run--dag_run_note - -1 -1 + +1 +1 dag_run--task_reschedule - + 0..N -1 +1 dag_run--task_reschedule - + 0..N -1 +1 @@ -2071,8 +2074,8 @@ log_template--dag_run - -0..N + +0..N {0,1} @@ -2137,8 +2140,8 @@ backfill--dag_run - -0..N + +0..N {0,1} From 6744115cf72f77d9669f49103bceb124c9ac79cf Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 17 Dec 2024 22:17:45 +0200 Subject: [PATCH 33/52] fix include_dag_run on ti refresh_from_db --- airflow/models/taskinstance.py | 18 ++++++++++-------- 1 file changed, 10 insertions(+), 8 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index b41bc3821c031..fa8e41245baec 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -860,13 +860,19 @@ def _refresh_from_db( ) if ti: - inspector = inspect(ti) # Check if the ti is detached or the dag_run relationship isn't loaded. # If the scheduler that started the dag_run has exited (gracefully or forcefully), # there will be changes to the dag_run span context_carrier. # It's best to include the dag_run whenever possible, so that the ti will contain the updates. - include_dag_run = not inspector.detached and "dag_run" not in inspector.unloaded - log.debug("Unloaded: %s", inspector.unloaded) + ti_inspector = inspect(ti) + dr_inspector = inspect(ti.dag_run) + + is_ti_attached = not ti_inspector.detached + is_dr_attached = not dr_inspector.detached + is_dr_loaded = "dag_run" not in ti_inspector.unloaded + + include_dag_run = is_ti_attached and is_dr_attached and is_dr_loaded + _set_ti_attrs(task_instance, ti, include_dag_run=include_dag_run) else: task_instance.state = None @@ -1003,11 +1009,7 @@ def get_triggering_events() -> dict[str, list[AssetEvent]]: # Re-attach it if we get called. nonlocal dag_run if dag_run not in session: - # In case, refresh_from_db has also included the dag_run, - # the object will be considered dirty by the session. - # Trying to merge the dirty dag_run with load=False, will result to an SQLAlchemy error. - # Regular merge, with the default load value. - dag_run = session.merge(dag_run) + dag_run = session.merge(dag_run, load=False) asset_events = dag_run.consumed_asset_events triggering_events: dict[str, list[AssetEvent]] = defaultdict(list) for event in asset_events: From 8434b02826b9b9ee009c4c72b5fa81975e9c2d95 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 18 Dec 2024 15:07:32 +0200 Subject: [PATCH 34/52] fix issue with recreated spans + fix integration tests --- airflow/jobs/scheduler_job_runner.py | 14 +- .../otel/dags/otel_test_dag_with_pause.py | 58 ++--- .../dags/otel_test_dag_with_pause_in_task.py | 156 ++++++++++++++ tests/integration/otel/test_otel.py | 198 ++++++++++++++---- tests/jobs/test_scheduler_job.py | 3 + 5 files changed, 364 insertions(+), 65 deletions(-) create mode 100644 tests/integration/otel/dags/otel_test_dag_with_pause_in_task.py diff --git a/airflow/jobs/scheduler_job_runner.py b/airflow/jobs/scheduler_job_runner.py index f74a65d3cd190..828b60afad606 100644 --- a/airflow/jobs/scheduler_job_runner.py +++ b/airflow/jobs/scheduler_job_runner.py @@ -1145,9 +1145,10 @@ def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session and dag_run.state in State.unfinished_dr_states and dag_run.span_status == SpanStatus.ACTIVE ): + initial_scheduler_id = dag_run.scheduled_by_job_id job: Job = session.scalars( select(Job).where( - Job.id == dag_run.scheduled_by_job_id, + Job.id == initial_scheduler_id, Job.job_type == "SchedulerJob", ) ).one() @@ -1167,13 +1168,16 @@ def _recreate_unhealthy_scheduler_spans_if_needed(self, dag_run: DagRun, session tis = dag_run.get_task_instances(session=session) - # At this point, any tis will have been adopted by the current scheduler. - # If the span_status is ACTIVE but there isn't an entry on the active spans, - # then it was started by the unhealthy scheduler. + # At this point, any tis will have been adopted by the current scheduler, + # and ti.queued_by_job_id will point to the current id. + # Any tis that have been executed by the unhealthy scheduler, will need a new span + # so that it can be associated with the new dag_run span. tis_needing_spans = [ ti for ti in tis - if ti.span_status == SpanStatus.ACTIVE and self.active_spans.get(ti.key) is None + # If it has started and there is a reference on the active_spans dict, + # then it was started by the current scheduler. + if ti.start_date is not None and self.active_spans.get(ti.key) is None ] dr_context = Trace.extract(dag_run.context_carrier) diff --git a/tests/integration/otel/dags/otel_test_dag_with_pause.py b/tests/integration/otel/dags/otel_test_dag_with_pause.py index 8b8b3e5d5b5b2..1c46f7f7cc16a 100644 --- a/tests/integration/otel/dags/otel_test_dag_with_pause.py +++ b/tests/integration/otel/dags/otel_test_dag_with_pause.py @@ -52,28 +52,6 @@ def task1_func(**dag_context): ti = dag_context["ti"] context_carrier = ti.context_carrier - dag_folder = os.path.dirname(os.path.abspath(__file__)) - control_file = os.path.join(dag_folder, "dag_control.txt") - - # Create the file and write 'pause' to it. - with open(control_file, "w") as file: - file.write("pause") - - # Pause execution until the word 'pause' is replaced on the file. - while True: - # If there is an exception, then writing to the file failed. Let it exit. - file_contents = None - with open(control_file) as file: - file_contents = file.read() - - if "pause" in file_contents: - logger.info("Task has been paused.") - continue - else: - logger.info("Resuming task execution.") - # Break the loop and finish with the task execution. - break - otel_task_tracer = otel_tracer.get_otel_tracer_for_task(Trace) tracer_provider = otel_task_tracer.get_otel_tracer_provider() @@ -128,12 +106,39 @@ def task1_func(**dag_context): s4.set_attribute("attr4", "val4") logger.info("From task sub_span4.") + logger.info("Task_1 finished.") + + def paused_task_func(): + logger.info("Starting Paused_task.") + + dag_folder = os.path.dirname(os.path.abspath(__file__)) + control_file = os.path.join(dag_folder, "dag_control.txt") + + # Create the file and write 'pause' to it. + with open(control_file, "w") as file: + file.write("pause") + + # Pause execution until the word 'pause' is replaced on the file. + while True: + # If there is an exception, then writing to the file failed. Let it exit. + file_contents = None + with open(control_file) as file: + file_contents = file.read() + + if "pause" in file_contents: + logger.info("Task has been paused.") + continue + else: + logger.info("Resuming task execution.") + # Break the loop and finish with the task execution. + break + # Cleanup the control file. if os.path.exists(control_file): os.remove(control_file) print("Control file has been cleaned up.") - logger.info("Task_1 finished.") + logger.info("Paused_task finished.") def task2_func(): logger.info("Starting Task_2.") @@ -147,10 +152,15 @@ def task2_func(): python_callable=task1_func, ) + pause = PythonOperator( + task_id="paused_task", + python_callable=paused_task_func, + ) + t2 = PythonOperator( task_id="task_2", python_callable=task2_func, ) # Dependencies. - t1 >> t2 + t1 >> pause >> t2 diff --git a/tests/integration/otel/dags/otel_test_dag_with_pause_in_task.py b/tests/integration/otel/dags/otel_test_dag_with_pause_in_task.py new file mode 100644 index 0000000000000..9119634c0d3fc --- /dev/null +++ b/tests/integration/otel/dags/otel_test_dag_with_pause_in_task.py @@ -0,0 +1,156 @@ +# 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 logging +import os +from datetime import datetime + +from opentelemetry import trace +from sqlalchemy import select + +from airflow import DAG +from airflow.models import TaskInstance +from airflow.providers.standard.operators.python import PythonOperator +from airflow.traces import otel_tracer +from airflow.traces.tracer import Trace +from airflow.utils.session import create_session + +logger = logging.getLogger("airflow.otel_test_dag_with_pause_in_task") + +args = { + "owner": "airflow", + "start_date": datetime(2024, 9, 2), + "retries": 0, +} + +# DAG definition. +with DAG( + "otel_test_dag_with_pause_in_task", + default_args=args, + schedule=None, + catchup=False, +) as dag: + # Tasks. + def task1_func(**dag_context): + logger.info("Starting Task_1.") + + ti = dag_context["ti"] + context_carrier = ti.context_carrier + + dag_folder = os.path.dirname(os.path.abspath(__file__)) + control_file = os.path.join(dag_folder, "dag_control.txt") + + # Create the file and write 'pause' to it. + with open(control_file, "w") as file: + file.write("pause") + + # Pause execution until the word 'pause' is replaced on the file. + while True: + # If there is an exception, then writing to the file failed. Let it exit. + file_contents = None + with open(control_file) as file: + file_contents = file.read() + + if "pause" in file_contents: + logger.info("Task has been paused.") + continue + else: + logger.info("Resuming task execution.") + # Break the loop and finish with the task execution. + break + + otel_task_tracer = otel_tracer.get_otel_tracer_for_task(Trace) + tracer_provider = otel_task_tracer.get_otel_tracer_provider() + + if context_carrier is not None: + logger.info("Found ti.context_carrier: %s.", context_carrier) + logger.info("Extracting the span context from the context_carrier.") + + # If the task takes too long to execute, then the ti should be read from the db + # to make sure that the initial context_carrier is the same. + with create_session() as session: + session_ti: TaskInstance = session.scalars( + select(TaskInstance).where( + TaskInstance.task_id == ti.task_id, + TaskInstance.run_id == ti.run_id, + ) + ).one() + context_carrier = session_ti.context_carrier + + parent_context = Trace.extract(context_carrier) + with otel_task_tracer.start_child_span( + span_name=f"{ti.task_id}_sub_span1", + parent_context=parent_context, + component="dag", + ) as s1: + s1.set_attribute("attr1", "val1") + logger.info("From task sub_span1.") + + with otel_task_tracer.start_child_span(f"{ti.task_id}_sub_span2") as s2: + s2.set_attribute("attr2", "val2") + logger.info("From task sub_span2.") + + tracer = trace.get_tracer("trace_test.tracer", tracer_provider=tracer_provider) + with tracer.start_as_current_span(name=f"{ti.task_id}_sub_span3") as s3: + s3.set_attribute("attr3", "val3") + logger.info("From task sub_span3.") + + with create_session() as session: + session_ti: TaskInstance = session.scalars( + select(TaskInstance).where( + TaskInstance.task_id == ti.task_id, + TaskInstance.run_id == ti.run_id, + ) + ).one() + context_carrier = session_ti.context_carrier + parent_context = Trace.extract(context_carrier) + + with otel_task_tracer.start_child_span( + span_name=f"{ti.task_id}_sub_span4", + parent_context=parent_context, + component="dag", + ) as s4: + s4.set_attribute("attr4", "val4") + logger.info("From task sub_span4.") + + # Cleanup the control file. + if os.path.exists(control_file): + os.remove(control_file) + print("Control file has been cleaned up.") + + logger.info("Task_1 finished.") + + def task2_func(): + logger.info("Starting Task_2.") + for i in range(3): + logger.info("Task_2, iteration '%d'.", i) + logger.info("Task_2 finished.") + + # Task operators. + t1 = PythonOperator( + task_id="task_1", + python_callable=task1_func, + ) + + t2 = PythonOperator( + task_id="task_2", + python_callable=task2_func, + ) + + # Dependencies. + t1 >> t2 diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index c0822c5d2bcba..b515f03266a94 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -156,7 +156,7 @@ def check_spans_with_continuance(output: str, dag: DAG, continuance_for_t1: bool # This is done by comparing the span_id of each root span with the parent_id of each non-root span. parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) - # The span hierarchy for dag 'otel_test_dag_with_pause' is + # The span hierarchy for dag 'otel_test_dag_with_pause_in_task' is # dag span # |_ task_1 span # |_ scheduler_exited span @@ -395,6 +395,121 @@ def check_spans_without_continuance( ) +def check_spans_for_paused_dag( + output: str, dag: DAG, is_recreated: bool = False, check_t1_sub_spans: bool = True +): + recreated_suffix = "_recreated" if is_recreated else "" + + # Get a list of lines from the captured output. + output_lines = output.splitlines() + + # Filter the output, create a json obj for each span and then store them into dictionaries. + # One dictionary with only the root spans, and one with all the captured spans (root and otherwise). + root_span_dict, span_dict = extract_spans_from_output(output_lines) + # Generate a dictionary with parent child relationships. + # This is done by comparing the span_id of each root span with the parent_id of each non-root span. + parent_child_dict = get_parent_child_dict(root_span_dict, span_dict) + + # Any spans generated under a task, are children of the task span. + # The span hierarchy for dag 'otel_test_dag_with_pause' is + # dag span + # |_ task_1 span + # |_ sub_span_1 + # |_ sub_span_2 + # |_ sub_span_3 + # |_ sub_span_4 + # |_ paused_task span + # |_ task_2 span + # + # In case task_1 has finished running and the span is recreated, + # the sub spans are lost and can't be recreated. The span hierarchy will be + # dag span + # |_ task_1 span + # |_ paused_task span + # |_ task_2 span + + dag_id = dag.dag_id + + task_instance_ids = dag.task_ids + task1_id = task_instance_ids[0] + paused_task_id = task_instance_ids[1] + task2_id = task_instance_ids[2] + + # Based on the current tests, only the root span and the task1 span will be recreated. + # TODO: Adjust accordingly, if there are more tests in the future + # that require other spans to be recreated as well. + dag_root_span_name = f"{dag_id}{recreated_suffix}" + + dag_root_span_children_names = [ + f"{task1_id}{recreated_suffix}", + f"{paused_task_id}{recreated_suffix}", + f"{task2_id}", + ] + + task1_span_children_names = [ + f"{task1_id}_sub_span1", + f"{task1_id}_sub_span4", + ] + + # Single element lists. + task1_sub_span1_children_span_names = [f"{task1_id}_sub_span2"] + task1_sub_span2_children_span_names = [f"{task1_id}_sub_span3"] + + assert_span_name_belongs_to_root_span( + root_span_dict=root_span_dict, span_name=dag_root_span_name, should_succeed=True + ) + + # Check direct children of the root span. + assert_parent_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + parent_name=dag_root_span_name, + children_names=dag_root_span_children_names, + ) + + # Use a span name that exists, but it's not a direct child. + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_root_span_name, + child_name=f"{task1_id}_sub_span1", + span_exists=True, + ) + + # Use a span name that doesn't exist at all. + assert_span_not_in_children_spans( + parent_child_dict=parent_child_dict, + root_span_dict=root_span_dict, + span_dict=span_dict, + parent_name=dag_root_span_name, + child_name=f"{task1_id}_non_existent", + span_exists=False, + ) + + if check_t1_sub_spans: + # Check children of the task1 span. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}{recreated_suffix}", + children_names=task1_span_children_names, + ) + + # Check children of task1 sub span1. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span1", + children_names=task1_sub_span1_children_span_names, + ) + + # Check children of task1 sub span2. + assert_parent_children_spans_for_non_root( + span_dict=span_dict, + parent_name=f"{task1_id}_sub_span2", + children_names=task1_sub_span2_children_span_names, + ) + + def print_output_for_dag_tis(dag: DAG): with create_session() as session: tis: list[TaskInstance] = dag.get_task_instances(session=session) @@ -493,9 +608,6 @@ def setup_class(cls): if cls.log_level == "debug": log.setLevel(logging.DEBUG) - db_init_command = ["airflow", "db", "init"] - subprocess.run(db_init_command, check=True, env=os.environ.copy()) - @classmethod def serialize_and_get_dags(cls) -> dict[str, DAG]: log.info("Serializing Dags from directory %s", cls.dag_folder) @@ -503,7 +615,7 @@ def serialize_and_get_dags(cls) -> dict[str, DAG]: dag_bag = DagBag(dag_folder=cls.dag_folder, include_examples=False) dag_ids = dag_bag.dag_ids - assert len(dag_ids) == 2 + assert len(dag_ids) == 3 dag_dict: dict[str, DAG] = {} with create_session() as session: @@ -736,7 +848,7 @@ def test_scheduler_change_in_the_middle_of_first_task_until_the_end( # so that the test can capture their output. celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() - dag_id = "otel_test_dag_with_pause" + dag_id = "otel_test_dag_with_pause_in_task" dag = self.dags[dag_id] run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) @@ -747,8 +859,13 @@ def test_scheduler_change_in_the_middle_of_first_task_until_the_end( while True: try: with open(control_file) as file: - # If it reaches inside the block, then the file exists and the test can read it. - break + file_contents = file.read() + + if "pause" in file_contents: + log.info("Control file exists and the task has been paused.") + break + else: + continue except FileNotFoundError: print("Control file not found. Waiting...") time.sleep(1) @@ -835,7 +952,7 @@ def test_scheduler_exits_gracefully_in_the_middle_of_the_first_task( # so that the test can capture their output. celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() - dag_id = "otel_test_dag_with_pause" + dag_id = "otel_test_dag_with_pause_in_task" dag = self.dags[dag_id] run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) @@ -846,8 +963,13 @@ def test_scheduler_exits_gracefully_in_the_middle_of_the_first_task( while True: try: with open(control_file) as file: - # If it reaches inside the block, then the file exists and the test can read it. - break + file_contents = file.read() + + if "pause" in file_contents: + log.info("Control file exists and the task has been paused.") + break + else: + continue except FileNotFoundError: print("Control file not found. Waiting...") time.sleep(1) @@ -917,7 +1039,7 @@ def test_scheduler_exits_forcefully_in_the_middle_of_the_first_task( # so that the test can capture their output. celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() - dag_id = "otel_test_dag_with_pause" + dag_id = "otel_test_dag_with_pause_in_task" dag = self.dags[dag_id] run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) @@ -928,8 +1050,13 @@ def test_scheduler_exits_forcefully_in_the_middle_of_the_first_task( while True: try: with open(control_file) as file: - # If it reaches inside the block, then the file exists and the test can read it. - break + file_contents = file.read() + + if "pause" in file_contents: + log.info("Control file exists and the task has been paused.") + break + else: + continue except FileNotFoundError: print("Control file not found. Waiting...") time.sleep(1) @@ -991,7 +1118,7 @@ def test_scheduler_exits_forcefully_after_the_first_task_finishes( ): """ The first scheduler will exit forcefully after the first task finishes, - so that it won't have time end any active spans. + so that it won't have time to end any active spans. In this scenario, the sub-spans for the first task will be lost. The only way to retrieve them, would be to re-run the task. """ @@ -1003,33 +1130,28 @@ def test_scheduler_exits_forcefully_after_the_first_task_finishes( # so that the test can capture their output. celery_worker_process, scheduler_process_1 = self.start_worker_and_scheduler1() - dag_id = "otel_test_dag" + dag_id = "otel_test_dag_with_pause" dag = self.dags[dag_id] run_id = unpause_trigger_dag_and_get_run_id(dag_id=dag_id) - with create_session() as session: - tis: list[TaskInstance] = dag.get_task_instances(session=session) - - task_1 = tis[0] + # Control file path. + control_file = os.path.join(self.dag_folder, "dag_control.txt") while True: - with create_session() as session: - ti = ( - session.query(TaskInstance) - .filter( - TaskInstance.task_id == task_1.task_id, - TaskInstance.run_id == task_1.run_id, - ) - .first() - ) - - if ti is None: - continue + try: + with open(control_file) as file: + file_contents = file.read() - # Wait until the task has been finished. - if ti.state in State.finished: - break + if "pause" in file_contents: + log.info("Control file exists and the task has been paused.") + break + else: + continue + except FileNotFoundError: + print("Control file not found. Waiting...") + time.sleep(1) + continue # Since, we are past the loop, then the file exists and the dag has been paused. # Kill scheduler1 and start scheduler2. @@ -1040,6 +1162,10 @@ def test_scheduler_exits_forcefully_after_the_first_task_finishes( dag_id=dag_id, run_id=run_id, state=State.RUNNING, span_status=SpanStatus.ACTIVE ) + # Rewrite the file to unpause the dag. + with open(control_file, "w") as file: + file.write("continue") + time.sleep(15) # The task should be adopted. @@ -1076,7 +1202,7 @@ def test_scheduler_exits_forcefully_after_the_first_task_finishes( if self.use_otel != "true": # Dag run should have succeeded. Test the spans in the output. - check_spans_without_continuance(output=out, dag=dag, is_recreated=True, check_t1_sub_spans=False) + check_spans_for_paused_dag(output=out, dag=dag, is_recreated=True, check_t1_sub_spans=False) def start_worker_and_scheduler1(self): celery_worker_process = subprocess.Popen( diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index e0eada7044440..4563e64604f77 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -2465,6 +2465,7 @@ def test_recreate_unhealthy_scheduler_spans_if_needed(self, ti_state, final_ti_s ti = dr.get_task_instances(session=session)[0] ti.state = ti_state + ti.start_date = timezone.utcnow() ti.span_status = SpanStatus.ACTIVE ti.queued_by_job_id = old_job.id session.merge(ti) @@ -2491,8 +2492,10 @@ def test_recreate_unhealthy_scheduler_spans_if_needed(self, ti_state, final_ti_s if final_ti_span_status == SpanStatus.ACTIVE: assert self.job_runner.active_spans.get(ti.key) is not None + assert len(self.job_runner.active_spans.get_all()) == 2 else: assert self.job_runner.active_spans.get(ti.key) is None + assert len(self.job_runner.active_spans.get_all()) == 1 assert dr.span_status == SpanStatus.ACTIVE assert ti.span_status == final_ti_span_status From b562d87aa9b1b40b75a76360789f56205a3d7fa6 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 18 Dec 2024 15:29:16 +0200 Subject: [PATCH 35/52] trigger er diagram generation --- .../0052_3_0_0_add_new_otel_span_fields.py | 7 +- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- docs/apache-airflow/img/airflow_erd.svg | 2267 +++++++++-------- 3 files changed, 1153 insertions(+), 1123 deletions(-) diff --git a/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py index 9e6f99b7fa3f8..90e5651700f55 100644 --- a/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py +++ b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py @@ -29,8 +29,7 @@ import sqlalchemy as sa from alembic import op - -from airflow.utils.sqlalchemy import ExtendedJSON +from sqlalchemy import JSON # revision identifiers, used by Alembic. revision = "0eb040b3eb12" @@ -43,10 +42,10 @@ def upgrade(): """Apply add new otel span fields.""" op.add_column("dag_run", sa.Column("scheduled_by_job_id", sa.Integer, nullable=True)) - op.add_column("dag_run", sa.Column("context_carrier", ExtendedJSON, nullable=True)) + op.add_column("dag_run", sa.Column("context_carrier", JSON, nullable=True)) op.add_column("dag_run", sa.Column("span_status", sa.String(250), nullable=False)) - op.add_column("task_instance", sa.Column("context_carrier", ExtendedJSON, nullable=True)) + op.add_column("task_instance", sa.Column("context_carrier", JSON, nullable=True)) op.add_column("task_instance", sa.Column("span_status", sa.String(250), nullable=False)) diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index b503e8dfaf91a..8db43702be941 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -ba10504bc54d15b2faca37ae9db172848a498e471bbf332e031715f728158ff8 \ No newline at end of file +2029d7879c29a6e7b5d1e20c992a1f7afa23a20e900a92b71fa70f3ae0950ade \ No newline at end of file diff --git a/docs/apache-airflow/img/airflow_erd.svg b/docs/apache-airflow/img/airflow_erd.svg index 9f19f0f920a5c..7ba6d63c90481 100644 --- a/docs/apache-airflow/img/airflow_erd.svg +++ b/docs/apache-airflow/img/airflow_erd.svg @@ -4,11 +4,11 @@ - - + + %3 - + log @@ -105,8 +105,8 @@ callback_data - [JSON] - NOT NULL + [JSONB] + NOT NULL callback_type @@ -604,74 +604,74 @@ asset_event - -asset_event - -id - - [INTEGER] - NOT NULL - -asset_id - - [INTEGER] - NOT NULL - -extra - - [JSON] - NOT NULL - -source_dag_id - - [VARCHAR(250)] - -source_map_index - - [INTEGER] - -source_run_id - - [VARCHAR(250)] - -source_task_id - - [VARCHAR(250)] - -timestamp - - [TIMESTAMP] - NOT NULL + +asset_event + +id + + [INTEGER] + NOT NULL + +asset_id + + [INTEGER] + NOT NULL + +extra + + [JSON] + NOT NULL + +source_dag_id + + [VARCHAR(250)] + +source_map_index + + [INTEGER] + +source_run_id + + [VARCHAR(250)] + +source_task_id + + [VARCHAR(250)] + +timestamp + + [TIMESTAMP] + NOT NULL asset_event--asset_alias_asset_event - -0..N -1 + +0..N +1 dagrun_asset_event - -dagrun_asset_event - -dag_run_id - - [INTEGER] - NOT NULL - -event_id - - [INTEGER] - NOT NULL + +dagrun_asset_event + +dag_run_id + + [INTEGER] + NOT NULL + +event_id + + [INTEGER] + NOT NULL asset_event--dagrun_asset_event - -0..N -1 + +0..N +1 @@ -713,687 +713,705 @@ task_instance - -task_instance - -id - - [UUID] - NOT NULL - -custom_operator_name - - [VARCHAR(1000)] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -last_heartbeat_at - - [TIMESTAMP] - -map_index - - [INTEGER] - NOT NULL - -max_tries - - [INTEGER] - -next_kwargs - - [JSON] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -task_id - - [VARCHAR(250)] - NOT NULL - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance + +id + + [UUID] + NOT NULL + +context_carrier + + [JSONB] + +custom_operator_name + + [VARCHAR(1000)] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +last_heartbeat_at + + [TIMESTAMP] + +map_index + + [INTEGER] + NOT NULL + +max_tries + + [INTEGER] + +next_kwargs + + [JSONB] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +run_id + + [VARCHAR(250)] + NOT NULL + +span_status + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +task_id + + [VARCHAR(250)] + NOT NULL + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] trigger--task_instance - -0..N + +0..N {0,1} task_reschedule - -task_reschedule - -id - - [INTEGER] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -duration - - [INTEGER] - NOT NULL - -end_date - - [TIMESTAMP] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -reschedule_date - - [TIMESTAMP] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -try_number - - [INTEGER] - NOT NULL + +task_reschedule + +id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +duration + + [INTEGER] + NOT NULL + +end_date + + [TIMESTAMP] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +reschedule_date + + [TIMESTAMP] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +try_number + + [INTEGER] + NOT NULL task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 rendered_task_instance_fields - -rendered_task_instance_fields - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -k8s_pod_yaml - - [JSON] - -rendered_fields - - [JSON] - NOT NULL + +rendered_task_instance_fields + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +k8s_pod_yaml + + [JSON] + +rendered_fields + + [JSON] + NOT NULL task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_map - -task_map - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -keys - - [JSON] - -length - - [INTEGER] - NOT NULL + +task_map + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +keys + + [JSONB] + +length + + [INTEGER] + NOT NULL task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 xcom - -xcom - -dag_run_id - - [INTEGER] - NOT NULL - -key - - [VARCHAR(512)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -timestamp - - [TIMESTAMP] - NOT NULL - -value - - [JSONB] + +xcom + +dag_run_id + + [INTEGER] + NOT NULL + +key + + [VARCHAR(512)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +timestamp + + [TIMESTAMP] + NOT NULL + +value + + [JSONB] task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance_note - -task_instance_note - -dag_id - - [VARCHAR(250)] - NOT NULL - -map_index - - [INTEGER] - NOT NULL - -run_id - - [VARCHAR(250)] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] + +task_instance_note + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL + +run_id + + [VARCHAR(250)] + NOT NULL + +task_id + + [VARCHAR(250)] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [VARCHAR(128)] task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance_history - -task_instance_history - -id - - [INTEGER] - NOT NULL - -custom_operator_name - - [VARCHAR(1000)] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -duration - - [DOUBLE_PRECISION] - -end_date - - [TIMESTAMP] - -executor - - [VARCHAR(1000)] - -executor_config - - [BYTEA] - -external_executor_id - - [VARCHAR(250)] - -hostname - - [VARCHAR(1000)] - -map_index - - [INTEGER] - NOT NULL - -max_tries - - [INTEGER] - -next_kwargs - - [JSON] - -next_method - - [VARCHAR(1000)] - -operator - - [VARCHAR(1000)] - -pid - - [INTEGER] - -pool - - [VARCHAR(256)] - NOT NULL - -pool_slots - - [INTEGER] - NOT NULL - -priority_weight - - [INTEGER] - -queue - - [VARCHAR(256)] - -queued_by_job_id - - [INTEGER] - -queued_dttm - - [TIMESTAMP] - -rendered_map_index - - [VARCHAR(250)] - -run_id - - [VARCHAR(250)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(20)] - -task_display_name - - [VARCHAR(2000)] - -task_id - - [VARCHAR(250)] - NOT NULL - -trigger_id - - [INTEGER] - -trigger_timeout - - [TIMESTAMP] - -try_number - - [INTEGER] - NOT NULL - -unixname - - [VARCHAR(1000)] - -updated_at - - [TIMESTAMP] + +task_instance_history + +id + + [INTEGER] + NOT NULL + +context_carrier + + [JSONB] + +custom_operator_name + + [VARCHAR(1000)] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +duration + + [DOUBLE_PRECISION] + +end_date + + [TIMESTAMP] + +executor + + [VARCHAR(1000)] + +executor_config + + [BYTEA] + +external_executor_id + + [VARCHAR(250)] + +hostname + + [VARCHAR(1000)] + +map_index + + [INTEGER] + NOT NULL + +max_tries + + [INTEGER] + +next_kwargs + + [JSONB] + +next_method + + [VARCHAR(1000)] + +operator + + [VARCHAR(1000)] + +pid + + [INTEGER] + +pool + + [VARCHAR(256)] + NOT NULL + +pool_slots + + [INTEGER] + NOT NULL + +priority_weight + + [INTEGER] + +queue + + [VARCHAR(256)] + +queued_by_job_id + + [INTEGER] + +queued_dttm + + [TIMESTAMP] + +rendered_map_index + + [VARCHAR(250)] + +run_id + + [VARCHAR(250)] + NOT NULL + +span_status + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(20)] + +task_display_name + + [VARCHAR(2000)] + +task_id + + [VARCHAR(250)] + NOT NULL + +trigger_id + + [INTEGER] + +trigger_timeout + + [TIMESTAMP] + +try_number + + [INTEGER] + NOT NULL + +unixname + + [VARCHAR(1000)] + +updated_at + + [TIMESTAMP] task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 @@ -1683,112 +1701,125 @@ dag_version--task_instance - -0..N + +0..N {0,1} dag_run - -dag_run - -id - - [INTEGER] - NOT NULL - -backfill_id - - [INTEGER] - -clear_number - - [INTEGER] - NOT NULL - -conf - - [BYTEA] - -creating_job_id - - [INTEGER] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -data_interval_end - - [TIMESTAMP] - -data_interval_start - - [TIMESTAMP] - -end_date - - [TIMESTAMP] - -external_trigger - - [BOOLEAN] - -last_scheduling_decision - - [TIMESTAMP] - -log_template_id - - [INTEGER] - -logical_date - - [TIMESTAMP] - NOT NULL - -queued_at - - [TIMESTAMP] - -run_id - - [VARCHAR(250)] - NOT NULL - -run_type - - [VARCHAR(50)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(50)] - -triggered_by - - [VARCHAR(50)] - -updated_at - - [TIMESTAMP] + +dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + +clear_number + + [INTEGER] + NOT NULL + +conf + + [BYTEA] + +context_carrier + + [JSONB] + +creating_job_id + + [INTEGER] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +data_interval_end + + [TIMESTAMP] + +data_interval_start + + [TIMESTAMP] + +end_date + + [TIMESTAMP] + +external_trigger + + [BOOLEAN] + +last_scheduling_decision + + [TIMESTAMP] + +log_template_id + + [INTEGER] + +logical_date + + [TIMESTAMP] + NOT NULL + +queued_at + + [TIMESTAMP] + +run_id + + [VARCHAR(250)] + NOT NULL + +run_type + + [VARCHAR(50)] + NOT NULL + +scheduled_by_job_id + + [INTEGER] + +span_status + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(50)] + +triggered_by + + [VARCHAR(50)] + +updated_at + + [TIMESTAMP] dag_version--dag_run - -0..N -{0,1} + +0..N +{0,1} @@ -1836,7 +1867,7 @@ dag_version--dag_code 0..N -1 +1 @@ -1891,536 +1922,536 @@ dag_run--dagrun_asset_event - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 backfill_dag_run - -backfill_dag_run - -id - - [INTEGER] - NOT NULL - -backfill_id - - [INTEGER] - NOT NULL - -dag_run_id - - [INTEGER] - -exception_reason - - [VARCHAR(250)] - -logical_date - - [TIMESTAMP] - NOT NULL - -sort_ordinal - - [INTEGER] - NOT NULL + +backfill_dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + NOT NULL + +dag_run_id + + [INTEGER] + +exception_reason + + [VARCHAR(250)] + +logical_date + + [TIMESTAMP] + NOT NULL + +sort_ordinal + + [INTEGER] + NOT NULL dag_run--backfill_dag_run - -0..N -{0,1} + +0..N +{0,1} dag_run_note - -dag_run_note - -dag_run_id - - [INTEGER] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] + +dag_run_note + +dag_run_id + + [INTEGER] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [VARCHAR(128)] dag_run--dag_run_note - -1 -1 + +1 +1 dag_run--task_reschedule - -0..N -1 + +0..N +1 dag_run--task_reschedule - -0..N -1 + +0..N +1 log_template - -log_template - -id - - [INTEGER] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -elasticsearch_id - - [TEXT] - NOT NULL - -filename - - [TEXT] - NOT NULL + +log_template + +id + + [INTEGER] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +elasticsearch_id + + [TEXT] + NOT NULL + +filename + + [TEXT] + NOT NULL log_template--dag_run - -0..N -{0,1} + +0..N +{0,1} backfill - -backfill - -id - - [INTEGER] - NOT NULL - -completed_at - - [TIMESTAMP] - -created_at - - [TIMESTAMP] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_run_conf - - [JSON] - NOT NULL - -from_date - - [TIMESTAMP] - NOT NULL - -is_paused - - [BOOLEAN] - -max_active_runs - - [INTEGER] - NOT NULL - -reprocess_behavior - - [VARCHAR(250)] - NOT NULL - -to_date - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL + +backfill + +id + + [INTEGER] + NOT NULL + +completed_at + + [TIMESTAMP] + +created_at + + [TIMESTAMP] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_run_conf + + [JSON] + NOT NULL + +from_date + + [TIMESTAMP] + NOT NULL + +is_paused + + [BOOLEAN] + +max_active_runs + + [INTEGER] + NOT NULL + +reprocess_behavior + + [VARCHAR(250)] + NOT NULL + +to_date + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL backfill--dag_run - -0..N -{0,1} + +0..N +{0,1} backfill--backfill_dag_run - -0..N -1 + +0..N +1 session - -session - -id - - [INTEGER] - NOT NULL - -data - - [BYTEA] - -expiry - - [TIMESTAMP] - -session_id - - [VARCHAR(255)] + +session + +id + + [INTEGER] + NOT NULL + +data + + [BYTEA] + +expiry + + [TIMESTAMP] + +session_id + + [VARCHAR(255)] alembic_version - -alembic_version - -version_num - - [VARCHAR(32)] - NOT NULL + +alembic_version + +version_num + + [VARCHAR(32)] + NOT NULL ab_user - -ab_user - -id - - [INTEGER] - NOT NULL - -active - - [BOOLEAN] - -changed_by_fk - - [INTEGER] - -changed_on - - [TIMESTAMP] - -created_by_fk - - [INTEGER] - -created_on - - [TIMESTAMP] - -email - - [VARCHAR(512)] - NOT NULL - -fail_login_count - - [INTEGER] - -first_name - - [VARCHAR(256)] - NOT NULL - -last_login - - [TIMESTAMP] - -last_name - - [VARCHAR(256)] - NOT NULL - -login_count - - [INTEGER] - -password - - [VARCHAR(256)] - -username - - [VARCHAR(512)] - NOT NULL + +ab_user + +id + + [INTEGER] + NOT NULL + +active + + [BOOLEAN] + +changed_by_fk + + [INTEGER] + +changed_on + + [TIMESTAMP] + +created_by_fk + + [INTEGER] + +created_on + + [TIMESTAMP] + +email + + [VARCHAR(512)] + NOT NULL + +fail_login_count + + [INTEGER] + +first_name + + [VARCHAR(256)] + NOT NULL + +last_login + + [TIMESTAMP] + +last_name + + [VARCHAR(256)] + NOT NULL + +login_count + + [INTEGER] + +password + + [VARCHAR(256)] + +username + + [VARCHAR(512)] + NOT NULL ab_user--ab_user - -0..N -{0,1} + +0..N +{0,1} ab_user--ab_user - -0..N -{0,1} + +0..N +{0,1} ab_user_role - -ab_user_role - -id - - [INTEGER] - NOT NULL - -role_id - - [INTEGER] - -user_id - - [INTEGER] + +ab_user_role + +id + + [INTEGER] + NOT NULL + +role_id + + [INTEGER] + +user_id + + [INTEGER] ab_user--ab_user_role - -0..N -{0,1} + +0..N +{0,1} ab_register_user - -ab_register_user - -id - - [INTEGER] - NOT NULL - -email - - [VARCHAR(512)] - NOT NULL - -first_name - - [VARCHAR(256)] - NOT NULL - -last_name - - [VARCHAR(256)] - NOT NULL - -password - - [VARCHAR(256)] - -registration_date - - [TIMESTAMP] - -registration_hash - - [VARCHAR(256)] - -username - - [VARCHAR(512)] - NOT NULL + +ab_register_user + +id + + [INTEGER] + NOT NULL + +email + + [VARCHAR(512)] + NOT NULL + +first_name + + [VARCHAR(256)] + NOT NULL + +last_name + + [VARCHAR(256)] + NOT NULL + +password + + [VARCHAR(256)] + +registration_date + + [TIMESTAMP] + +registration_hash + + [VARCHAR(256)] + +username + + [VARCHAR(512)] + NOT NULL ab_permission - -ab_permission - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(100)] - NOT NULL + +ab_permission + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(100)] + NOT NULL ab_permission_view - -ab_permission_view - -id - - [INTEGER] - NOT NULL - -permission_id - - [INTEGER] - -view_menu_id - - [INTEGER] + +ab_permission_view + +id + + [INTEGER] + NOT NULL + +permission_id + + [INTEGER] + +view_menu_id + + [INTEGER] ab_permission--ab_permission_view - -0..N -{0,1} + +0..N +{0,1} ab_permission_view_role - -ab_permission_view_role - -id - - [INTEGER] - NOT NULL - -permission_view_id - - [INTEGER] - -role_id - - [INTEGER] + +ab_permission_view_role + +id + + [INTEGER] + NOT NULL + +permission_view_id + + [INTEGER] + +role_id + + [INTEGER] ab_permission_view--ab_permission_view_role - -0..N -{0,1} + +0..N +{0,1} ab_view_menu - -ab_view_menu - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(250)] - NOT NULL + +ab_view_menu + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(250)] + NOT NULL ab_view_menu--ab_permission_view - -0..N -{0,1} + +0..N +{0,1} ab_role - -ab_role - -id - - [INTEGER] - NOT NULL - -name - - [VARCHAR(64)] - NOT NULL + +ab_role + +id + + [INTEGER] + NOT NULL + +name + + [VARCHAR(64)] + NOT NULL ab_role--ab_user_role - -0..N -{0,1} + +0..N +{0,1} ab_role--ab_permission_view_role - -0..N -{0,1} + +0..N +{0,1} alembic_version_fab - -alembic_version_fab - -version_num - - [VARCHAR(32)] - NOT NULL + +alembic_version_fab + +version_num + + [VARCHAR(32)] + NOT NULL From 1d19534218cc51b7c21dc1f938c5d7d5d43fd2ac Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 18 Dec 2024 15:31:18 +0200 Subject: [PATCH 36/52] trigger er diagram generation --- .../versions/0052_3_0_0_add_new_otel_span_fields.py | 7 ++++--- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py index 90e5651700f55..9e6f99b7fa3f8 100644 --- a/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py +++ b/airflow/migrations/versions/0052_3_0_0_add_new_otel_span_fields.py @@ -29,7 +29,8 @@ import sqlalchemy as sa from alembic import op -from sqlalchemy import JSON + +from airflow.utils.sqlalchemy import ExtendedJSON # revision identifiers, used by Alembic. revision = "0eb040b3eb12" @@ -42,10 +43,10 @@ def upgrade(): """Apply add new otel span fields.""" op.add_column("dag_run", sa.Column("scheduled_by_job_id", sa.Integer, nullable=True)) - op.add_column("dag_run", sa.Column("context_carrier", JSON, nullable=True)) + op.add_column("dag_run", sa.Column("context_carrier", ExtendedJSON, nullable=True)) op.add_column("dag_run", sa.Column("span_status", sa.String(250), nullable=False)) - op.add_column("task_instance", sa.Column("context_carrier", JSON, nullable=True)) + op.add_column("task_instance", sa.Column("context_carrier", ExtendedJSON, nullable=True)) op.add_column("task_instance", sa.Column("span_status", sa.String(250), nullable=False)) diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index 8db43702be941..3ae95e16ca00e 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -2029d7879c29a6e7b5d1e20c992a1f7afa23a20e900a92b71fa70f3ae0950ade \ No newline at end of file +ae23f340a23b3dd8b997f5893d77c7fcc05a1622c208dcda014e379380f11100 \ No newline at end of file From 67e85afc95c5ca427760e5b3f8b455c3843a54d3 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 18 Dec 2024 15:43:22 +0200 Subject: [PATCH 37/52] refactor dagrun to improve readability --- airflow/models/dagrun.py | 198 ++++++++++++++++++++------------------- 1 file changed, 101 insertions(+), 97 deletions(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 67e3f7d9d6d64..324056b898c35 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -901,6 +901,104 @@ def set_dagrun_span_attrs(self, span: Span | EmptySpan, dagv: DagVersion): span.add_event(name="airflow.dag_run.ended", timestamp=datetime_to_nano(self.end_date)) span.set_attributes(attributes) + def start_dr_spans_if_needed(self, tis: list[TI]): + # If there is no value in active_spans, then the span hasn't already been started. + if self.active_spans is not None and self.active_spans.get(self.run_id) is None: + if self.span_status == SpanStatus.NOT_STARTED or self.span_status == SpanStatus.NEEDS_CONTINUANCE: + dr_span = None + continue_ti_spans = False + if self.span_status == SpanStatus.NOT_STARTED: + dr_span = Trace.start_root_span( + span_name=f"{self.dag_id}", + component="dag", + start_time=self.queued_at, # This is later converted to nano. + start_as_current=False, + ) + elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: + # Use the existing context_carrier to set the initial dag_run span as the parent. + parent_context = Trace.extract(self.context_carrier) + with Trace.start_child_span( + span_name="new_scheduler", parent_context=parent_context + ) as s: + s.set_attribute("trace_status", "continued") + + dr_span = Trace.start_child_span( + span_name=f"{self.dag_id}_continued", + parent_context=parent_context, + component="dag", + # No start time + start_as_current=False, + ) + # After this span is started, the context_carrier will be replaced by the new one. + # New task span will use this span as the parent. + continue_ti_spans = True + carrier = Trace.inject() + self.context_carrier = carrier + self.span_status = SpanStatus.ACTIVE + # Set the span in a synchronized dictionary, so that the variable can be used to end the span. + self.active_spans.set(self.run_id, dr_span) + self.log.debug( + "DagRun span has been started and the injected context_carrier is: %s", + self.context_carrier, + ) + # Start TI spans that also need continuance. + if continue_ti_spans: + new_dagrun_context = Trace.extract(self.context_carrier) + for ti in tis: + if ti.span_status == SpanStatus.NEEDS_CONTINUANCE: + ti_span = Trace.start_child_span( + span_name=f"{ti.task_id}_continued", + parent_context=new_dagrun_context, + start_as_current=False, + ) + ti_carrier = Trace.inject() + ti.context_carrier = ti_carrier + ti.span_status = SpanStatus.ACTIVE + self.active_spans.set(ti.key, ti_span) + else: + self.log.info( + "Found span_status '%s', while updating state for dag_run '%s'", + self.span_status, + self.run_id, + ) + + def end_dr_span_if_needed(self, dagv: DagVersion): + if self.active_spans is not None: + active_span = self.active_spans.get(self.run_id) + if active_span is not None: + self.log.debug( + "Found active span with span_id: %s, for dag_id: %s, run_id: %s, state: %s", + active_span.get_span_context().span_id, + self.dag_id, + self.run_id, + self.state, + ) + + self.set_dagrun_span_attrs(span=active_span, dagv=dagv) + active_span.end(end_time=datetime_to_nano(self.end_date)) + # Remove the span from the dict. + self.active_spans.delete(self.run_id) + self.span_status = SpanStatus.ENDED + else: + if self.span_status == SpanStatus.ACTIVE: + # Another scheduler has started the span. + # Update the DB SpanStatus to notify the owner to end it. + self.span_status = SpanStatus.SHOULD_END + elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: + # This is a corner case where the scheduler exited gracefully + # while the dag_run was almost done. + # Since it reached this point, the dag has finished but there has been no time + # to create a new span for the current scheduler. + # There is no need for more spans, update the status on the db. + self.span_status = SpanStatus.ENDED + else: + self.log.debug( + "No active span has been found for dag_id: %s, run_id: %s, state: %s", + self.dag_id, + self.run_id, + self.state, + ) + @provide_session def update_state( self, session: Session = NEW_SESSION, execute_callbacks: bool = True @@ -1039,68 +1137,8 @@ def recalculate(self) -> _UnfinishedStates: # finally, if the leaves aren't done, the dag is still running else: - # If there is no value in active_spans, then the span hasn't already been started. - if self.active_spans is not None and self.active_spans.get(self.run_id) is None: - if ( - self.span_status == SpanStatus.NOT_STARTED - or self.span_status == SpanStatus.NEEDS_CONTINUANCE - ): - dr_span = None - continue_ti_spans = False - if self.span_status == SpanStatus.NOT_STARTED: - dr_span = Trace.start_root_span( - span_name=f"{self.dag_id}", - component="dag", - start_time=self.queued_at, # This is later converted to nano. - start_as_current=False, - ) - elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: - # Use the existing context_carrier to set the initial dag_run span as the parent. - parent_context = Trace.extract(self.context_carrier) - with Trace.start_child_span( - span_name="new_scheduler", parent_context=parent_context - ) as s: - s.set_attribute("trace_status", "continued") - - dr_span = Trace.start_child_span( - span_name=f"{self.dag_id}_continued", - parent_context=parent_context, - component="dag", - # No start time - start_as_current=False, - ) - # After this span is started, the context_carrier will be replaced by the new one. - # New task span will use this span as the parent. - continue_ti_spans = True - carrier = Trace.inject() - self.context_carrier = carrier - self.span_status = SpanStatus.ACTIVE - # Set the span in a synchronized dictionary, so that the variable can be used to end the span. - self.active_spans.set(self.run_id, dr_span) - self.log.debug( - "DagRun span has been started and the injected context_carrier is: %s", - self.context_carrier, - ) - # Start TI spans that also need continuance. - if continue_ti_spans: - new_dagrun_context = Trace.extract(self.context_carrier) - for ti in tis: - if ti.span_status == SpanStatus.NEEDS_CONTINUANCE: - ti_span = Trace.start_child_span( - span_name=f"{ti.task_id}_continued", - parent_context=new_dagrun_context, - start_as_current=False, - ) - ti_carrier = Trace.inject() - ti.context_carrier = ti_carrier - ti.span_status = SpanStatus.ACTIVE - self.active_spans.set(ti.key, ti_span) - else: - self.log.info( - "Found span_status '%s', while updating state for dag_run '%s'", - self.span_status, - self.run_id, - ) + # It might need to start TI spans as well. + self.start_dr_spans_if_needed(tis=tis) self.set_state(DagRunState.RUNNING) @@ -1132,41 +1170,7 @@ def recalculate(self) -> _UnfinishedStates: dagv.version if dagv else None, ) - if self.active_spans is not None: - active_span = self.active_spans.get(self.run_id) - if active_span is not None: - self.log.debug( - "Found active span with span_id: %s, for dag_id: %s, run_id: %s, state: %s", - active_span.get_span_context().span_id, - self.dag_id, - self.run_id, - self.state, - ) - - self.set_dagrun_span_attrs(span=active_span, dagv=dagv) - active_span.end(end_time=datetime_to_nano(self.end_date)) - # Remove the span from the dict. - self.active_spans.delete(self.run_id) - self.span_status = SpanStatus.ENDED - else: - if self.span_status == SpanStatus.ACTIVE: - # Another scheduler has started the span. - # Update the DB SpanStatus to notify the owner to end it. - self.span_status = SpanStatus.SHOULD_END - elif self.span_status == SpanStatus.NEEDS_CONTINUANCE: - # This is a corner case where the scheduler exited gracefully - # while the dag_run was almost done. - # Since it reached this point, the dag has finished but there has been no time - # to create a new span for the current scheduler. - # There is no need for more spans, update the status on the db. - self.span_status = SpanStatus.ENDED - else: - self.log.debug( - "No active span has been found for dag_id: %s, run_id: %s, state: %s", - self.dag_id, - self.run_id, - self.state, - ) + self.end_dr_span_if_needed(dagv=dagv) session.flush() From 82c159b366facc7ceea6804777086529d3771ab8 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 18 Dec 2024 18:24:56 +0200 Subject: [PATCH 38/52] unit tests for dag_run span changes --- tests/models/test_dagrun.py | 159 ++++++++++++++++++++++++++++++++++++ 1 file changed, 159 insertions(+) diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index 53cf0b0f672cc..a8f221fe59b62 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -33,6 +33,7 @@ from airflow.exceptions import AirflowException from airflow.models.baseoperator import BaseOperator from airflow.models.dag import DAG, DagModel +from airflow.models.dag_version import DagVersion from airflow.models.dagrun import DagRun, DagRunNote from airflow.models.taskinstance import TaskInstance, TaskInstanceNote, clear_task_instances from airflow.models.taskmap import TaskMap @@ -44,7 +45,9 @@ from airflow.stats import Stats from airflow.triggers.base import StartTriggerArgs from airflow.utils import timezone +from airflow.utils.span_status import SpanStatus from airflow.utils.state import DagRunState, State, TaskInstanceState +from airflow.utils.thread_safe_dict import ThreadSafeDict from airflow.utils.trigger_rule import TriggerRule from airflow.utils.types import DagRunType @@ -518,6 +521,162 @@ def test_on_success_callback_when_task_skipped(self, session): assert dag_run.state == DagRunState.SUCCESS mock_on_success.assert_called_once() + def test_start_dr_spans_if_needed_new_span(self, session): + dag = DAG( + dag_id="test_start_dr_spans_if_needed_new_span", + schedule=datetime.timedelta(days=1), + start_date=datetime.datetime(2017, 1, 1), + ) + DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session) + + dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) + dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) + dag_task1.set_downstream(dag_task2) + + initial_task_states = { + "test_task1": TaskInstanceState.QUEUED, + "test_task2": TaskInstanceState.QUEUED, + } + + # Scheduler uses Serialized DAG -- so use that instead of the Actual DAG + dag = SerializedDAG.from_dict(SerializedDAG.to_dict(dag)) + + dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session) + + active_spans = ThreadSafeDict() + dag_run.set_active_spans(active_spans) + + tis = dag_run.get_task_instances() + + assert dag_run.active_spans is not None + assert dag_run.active_spans.get(dag_run.run_id) is None + assert dag_run.span_status == SpanStatus.NOT_STARTED + + dag_run.start_dr_spans_if_needed(tis=tis) + + assert dag_run.span_status == SpanStatus.ACTIVE + assert dag_run.active_spans.get(dag_run.run_id) is not None + + def test_start_dr_spans_if_needed_span_with_continuance(self, session): + dag = DAG( + dag_id="test_start_dr_spans_if_needed_span_with_continuance", + schedule=datetime.timedelta(days=1), + start_date=datetime.datetime(2017, 1, 1), + ) + DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session) + + dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) + dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) + dag_task1.set_downstream(dag_task2) + + initial_task_states = { + "test_task1": TaskInstanceState.RUNNING, + "test_task2": TaskInstanceState.QUEUED, + } + + # Scheduler uses Serialized DAG -- so use that instead of the Actual DAG + dag = SerializedDAG.from_dict(SerializedDAG.to_dict(dag)) + + dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session) + + active_spans = ThreadSafeDict() + dag_run.set_active_spans(active_spans) + + dag_run.span_status = SpanStatus.NEEDS_CONTINUANCE + + tis = dag_run.get_task_instances() + + first_ti = tis[0] + first_ti.span_status = SpanStatus.NEEDS_CONTINUANCE + + assert dag_run.active_spans is not None + assert dag_run.active_spans.get(dag_run.run_id) is None + assert dag_run.active_spans.get(first_ti.key) is None + assert dag_run.span_status == SpanStatus.NEEDS_CONTINUANCE + assert first_ti.span_status == SpanStatus.NEEDS_CONTINUANCE + + dag_run.start_dr_spans_if_needed(tis=tis) + + assert dag_run.span_status == SpanStatus.ACTIVE + assert first_ti.span_status == SpanStatus.ACTIVE + assert dag_run.active_spans.get(dag_run.run_id) is not None + assert dag_run.active_spans.get(first_ti.key) is not None + + def test_end_dr_span_if_needed(self, session): + dag = DAG( + dag_id="test_end_dr_span_if_needed", + schedule=datetime.timedelta(days=1), + start_date=datetime.datetime(2017, 1, 1), + ) + DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session) + + dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) + dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) + dag_task1.set_downstream(dag_task2) + + initial_task_states = { + "test_task1": TaskInstanceState.SUCCESS, + "test_task2": TaskInstanceState.SUCCESS, + } + + # Scheduler uses Serialized DAG -- so use that instead of the Actual DAG + dag = SerializedDAG.from_dict(SerializedDAG.to_dict(dag)) + + dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session) + + active_spans = ThreadSafeDict() + dag_run.set_active_spans(active_spans) + + from airflow.traces.tracer import Trace + + dr_span = Trace.start_root_span(span_name="test_span", start_as_current=False) + + active_spans.set(dag_run.run_id, dr_span) + + assert dag_run.active_spans is not None + assert dag_run.active_spans.get(dag_run.run_id) is not None + + dag_version = DagVersion.get_latest_version(dag.dag_id) + dag_run.end_dr_span_if_needed(dagv=dag_version) + + assert dag_run.span_status == SpanStatus.ENDED + assert dag_run.active_spans.get(dag_run.run_id) is None + + def test_end_dr_span_if_needed_with_span_from_another_scheduler(self, session): + dag = DAG( + dag_id="test_end_dr_span_if_needed_with_span_from_another_scheduler", + schedule=datetime.timedelta(days=1), + start_date=datetime.datetime(2017, 1, 1), + ) + DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session) + + dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) + dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) + dag_task1.set_downstream(dag_task2) + + initial_task_states = { + "test_task1": TaskInstanceState.SUCCESS, + "test_task2": TaskInstanceState.SUCCESS, + } + + # Scheduler uses Serialized DAG -- so use that instead of the Actual DAG + dag = SerializedDAG.from_dict(SerializedDAG.to_dict(dag)) + + dag_run = self.create_dag_run(dag=dag, task_states=initial_task_states, session=session) + + active_spans = ThreadSafeDict() + dag_run.set_active_spans(active_spans) + + dag_run.span_status = SpanStatus.ACTIVE + + assert dag_run.active_spans is not None + assert dag_run.active_spans.get(dag_run.run_id) is None + + dag_version = DagVersion.get_latest_version(dag.dag_id) + dag_run.end_dr_span_if_needed(dagv=dag_version) + + assert dag_run.span_status == SpanStatus.SHOULD_END + def test_dagrun_update_state_with_handle_callback_success(self, session): def on_success_callable(context): assert context["dag_run"].dag_id == "test_dagrun_update_state_with_handle_callback_success" From 537dec0900ef3d21d2fe91c2795f3ba0d0752b73 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 18 Dec 2024 19:08:17 +0200 Subject: [PATCH 39/52] fix refresh issue with loading dag_run when task_instance isn't bound to the session --- airflow/models/taskinstance.py | 27 +++++++++++++++++++-------- 1 file changed, 19 insertions(+), 8 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index d203916579578..492497281fe95 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -864,14 +864,25 @@ def _refresh_from_db( # If the scheduler that started the dag_run has exited (gracefully or forcefully), # there will be changes to the dag_run span context_carrier. # It's best to include the dag_run whenever possible, so that the ti will contain the updates. - ti_inspector = inspect(ti) - dr_inspector = inspect(ti.dag_run) - - is_ti_attached = not ti_inspector.detached - is_dr_attached = not dr_inspector.detached - is_dr_loaded = "dag_run" not in ti_inspector.unloaded - - include_dag_run = is_ti_attached and is_dr_attached and is_dr_loaded + task_instance_inspector = inspect(task_instance) + is_task_instance_bound_to_session = task_instance_inspector.session is not None + + # If the check is false, then it will try load the dag_run relationship from the task_instance + # and it will fail with this error: + # + # sqlalchemy.orm.exc.DetachedInstanceError: Parent instance + # is not bound to a Session; lazy load operation of attribute 'dag_run' cannot proceed + if is_task_instance_bound_to_session: + ti_inspector = inspect(ti) + dr_inspector = inspect(ti.dag_run) + + is_ti_attached = not ti_inspector.detached + is_dr_attached = not dr_inspector.detached + is_dr_loaded = "dag_run" not in ti_inspector.unloaded + + include_dag_run = is_ti_attached and is_dr_attached and is_dr_loaded + else: + include_dag_run = False _set_ti_attrs(task_instance, ti, include_dag_run=include_dag_run) else: From 11d5a6cac4b62b4dcbe80195f8df640b1fafb7f7 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 31 Dec 2024 17:12:58 +0200 Subject: [PATCH 40/52] fix test_scheduler_job failures after merge --- tests/jobs/test_scheduler_job.py | 3 --- 1 file changed, 3 deletions(-) diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index f8c66201dc3c8..fc308f500334c 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -2433,7 +2433,6 @@ def test_recreate_unhealthy_scheduler_spans_if_needed(self, ti_state, final_ti_s dag_id="test_recreate_unhealthy_scheduler_spans_if_needed", start_date=DEFAULT_DATE, max_active_runs=1, - processor_subdir=TEST_DAG_FOLDER, dagrun_timeout=datetime.timedelta(seconds=60), ): EmptyOperator(task_id="dummy") @@ -2504,7 +2503,6 @@ def test_end_spans_of_externally_ended_ops(self, dag_maker): dag_id="test_end_spans_of_externally_ended_ops", start_date=DEFAULT_DATE, max_active_runs=1, - processor_subdir=TEST_DAG_FOLDER, dagrun_timeout=datetime.timedelta(seconds=60), ): EmptyOperator(task_id="dummy") @@ -2563,7 +2561,6 @@ def test_end_active_spans(self, state, final_span_status, dag_maker): dag_id="test_end_active_spans", start_date=DEFAULT_DATE, max_active_runs=1, - processor_subdir=TEST_DAG_FOLDER, dagrun_timeout=datetime.timedelta(seconds=60), ): EmptyOperator(task_id="dummy") From 09ed2e91e383ccd949e86e0ef2eede34c960ab61 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 31 Dec 2024 18:37:05 +0200 Subject: [PATCH 41/52] fix test_dagrun failures after merge --- tests/models/test_dagrun.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index 91dc596cc7850..ec9b64ce9ebc8 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -527,7 +527,7 @@ def test_start_dr_spans_if_needed_new_span(self, session): schedule=datetime.timedelta(days=1), start_date=datetime.datetime(2017, 1, 1), ) - DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session) + DAG.bulk_write_to_db(dags=[dag], session=session) dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) @@ -563,7 +563,7 @@ def test_start_dr_spans_if_needed_span_with_continuance(self, session): schedule=datetime.timedelta(days=1), start_date=datetime.datetime(2017, 1, 1), ) - DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session) + DAG.bulk_write_to_db(dags=[dag], session=session) dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) @@ -608,7 +608,7 @@ def test_end_dr_span_if_needed(self, session): schedule=datetime.timedelta(days=1), start_date=datetime.datetime(2017, 1, 1), ) - DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session) + DAG.bulk_write_to_db(dags=[dag], session=session) dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) @@ -648,7 +648,7 @@ def test_end_dr_span_if_needed_with_span_from_another_scheduler(self, session): schedule=datetime.timedelta(days=1), start_date=datetime.datetime(2017, 1, 1), ) - DAG.bulk_write_to_db(dags=[dag], processor_subdir="/tmp/test", session=session) + DAG.bulk_write_to_db(dags=[dag], session=session) dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) From c5a71b6229877be59cbaa52fdcbeb45d8cdad2a6 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 15 Jan 2025 15:46:33 +0200 Subject: [PATCH 42/52] fix test failures after merge --- tests/models/test_dagrun.py | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index 01865e41e22b6..cd44b3cc93a3d 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -521,13 +521,13 @@ def test_on_success_callback_when_task_skipped(self, session): assert dag_run.state == DagRunState.SUCCESS mock_on_success.assert_called_once() - def test_start_dr_spans_if_needed_new_span(self, session): + def test_start_dr_spans_if_needed_new_span(self, testing_dag_bundle, session): dag = DAG( dag_id="test_start_dr_spans_if_needed_new_span", schedule=datetime.timedelta(days=1), start_date=datetime.datetime(2017, 1, 1), ) - DAG.bulk_write_to_db(dags=[dag], session=session) + DAG.bulk_write_to_db("testing", None, dags=[dag], session=session) dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) @@ -557,13 +557,13 @@ def test_start_dr_spans_if_needed_new_span(self, session): assert dag_run.span_status == SpanStatus.ACTIVE assert dag_run.active_spans.get(dag_run.run_id) is not None - def test_start_dr_spans_if_needed_span_with_continuance(self, session): + def test_start_dr_spans_if_needed_span_with_continuance(self, testing_dag_bundle, session): dag = DAG( dag_id="test_start_dr_spans_if_needed_span_with_continuance", schedule=datetime.timedelta(days=1), start_date=datetime.datetime(2017, 1, 1), ) - DAG.bulk_write_to_db(dags=[dag], session=session) + DAG.bulk_write_to_db("testing", None, dags=[dag], session=session) dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) @@ -602,13 +602,13 @@ def test_start_dr_spans_if_needed_span_with_continuance(self, session): assert dag_run.active_spans.get(dag_run.run_id) is not None assert dag_run.active_spans.get(first_ti.key) is not None - def test_end_dr_span_if_needed(self, session): + def test_end_dr_span_if_needed(self, testing_dag_bundle, session): dag = DAG( dag_id="test_end_dr_span_if_needed", schedule=datetime.timedelta(days=1), start_date=datetime.datetime(2017, 1, 1), ) - DAG.bulk_write_to_db(dags=[dag], session=session) + DAG.bulk_write_to_db("testing", None, dags=[dag], session=session) dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) @@ -642,13 +642,13 @@ def test_end_dr_span_if_needed(self, session): assert dag_run.span_status == SpanStatus.ENDED assert dag_run.active_spans.get(dag_run.run_id) is None - def test_end_dr_span_if_needed_with_span_from_another_scheduler(self, session): + def test_end_dr_span_if_needed_with_span_from_another_scheduler(self, testing_dag_bundle, session): dag = DAG( dag_id="test_end_dr_span_if_needed_with_span_from_another_scheduler", schedule=datetime.timedelta(days=1), start_date=datetime.datetime(2017, 1, 1), ) - DAG.bulk_write_to_db(dags=[dag], session=session) + DAG.bulk_write_to_db("testing", None, dags=[dag], session=session) dag_task1 = EmptyOperator(task_id="test_task1", dag=dag) dag_task2 = EmptyOperator(task_id="test_task2", dag=dag) From 6d5d3615c30ee0118db6e73f34cfa6cbb438d774 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 15 Jan 2025 17:55:10 +0200 Subject: [PATCH 43/52] fix test_exceptions.py --- tests/api_fastapi/common/test_exceptions.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/api_fastapi/common/test_exceptions.py b/tests/api_fastapi/common/test_exceptions.py index 14a9e579407b5..0ea8463e6baca 100644 --- a/tests/api_fastapi/common/test_exceptions.py +++ b/tests/api_fastapi/common/test_exceptions.py @@ -186,7 +186,7 @@ def test_handle_single_column_unique_constraint_error(self, session, table, expe status_code=status.HTTP_409_CONFLICT, detail={ "reason": "Unique constraint violation", - "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id, bundle_version, scheduled_by_job_id, context_carrier, span_status) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, (SELECT max(log_template.id) AS max_1 \nFROM log_template), ?, ?, ?, ?, ?, ?, ?)", + "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id, bundle_version, scheduled_by_job_id, context_carrier, span_status) VALUES (?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, (SELECT max(log_template.id) AS max_1 \nFROM log_template), ?, ?, ?, ?, ?, ?, ?, ?)", "orig_error": "UNIQUE constraint failed: dag_run.dag_id, dag_run.run_id", }, ), @@ -194,7 +194,7 @@ def test_handle_single_column_unique_constraint_error(self, session, table, expe status_code=status.HTTP_409_CONFLICT, detail={ "reason": "Unique constraint violation", - "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id, bundle_version, scheduled_by_job_id, context_carrier, span_status) VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, (SELECT max(log_template.id) AS max_1 \nFROM log_template), %s, %s, %s, %s, %s, %s, %s)", + "statement": "INSERT INTO dag_run (dag_id, queued_at, logical_date, start_date, end_date, state, run_id, creating_job_id, external_trigger, run_type, triggered_by, conf, data_interval_start, data_interval_end, last_scheduling_decision, log_template_id, updated_at, clear_number, backfill_id, dag_version_id, bundle_version, scheduled_by_job_id, context_carrier, span_status) VALUES (%s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, %s, (SELECT max(log_template.id) AS max_1 \nFROM log_template), %s, %s, %s, %s, %s, %s, %s, %s)", "orig_error": "(1062, \"Duplicate entry 'test_dag_id-test_run_id' for key 'dag_run.dag_run_dag_id_run_id_key'\")", }, ), From cb96bf76ae58e853212a8602bab9c4737755714d Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Fri, 17 Jan 2025 13:57:45 +0200 Subject: [PATCH 44/52] trigger ER diagram creation --- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- docs/apache-airflow/img/airflow_erd.svg | 1293 ++++++++++---------- 2 files changed, 663 insertions(+), 632 deletions(-) diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index 5626cf9708b0b..87757d1b6d9ba 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -cb858681fdc7a596db20c1c5dbf93812fd011a6df1e0b5322a49a51c8476bb93 \ No newline at end of file +bc0cc8716c482f9d2e17204ab586c7127d3152a91276b068f2bb102a7d156fa2 \ No newline at end of file diff --git a/docs/apache-airflow/img/airflow_erd.svg b/docs/apache-airflow/img/airflow_erd.svg index 3fa6369924caa..ada7ea06cd653 100644 --- a/docs/apache-airflow/img/airflow_erd.svg +++ b/docs/apache-airflow/img/airflow_erd.svg @@ -121,8 +121,8 @@ callback_data - [JSON] - NOT NULL + [JSONB] + NOT NULL callback_type @@ -649,24 +649,24 @@ dagrun_asset_event - -dagrun_asset_event - -dag_run_id - - [INTEGER] - NOT NULL - -event_id - - [INTEGER] - NOT NULL + +dagrun_asset_event + +dag_run_id + + [INTEGER] + NOT NULL + +event_id + + [INTEGER] + NOT NULL asset_event--dagrun_asset_event - -0..N + +0..N 1 @@ -709,114 +709,123 @@ task_instance - -task_instance + +task_instance + +id + + [UUID] + NOT NULL + +context_carrier + + [JSONB] -id - - [UUID] - NOT NULL +custom_operator_name + + [VARCHAR(1000)] -custom_operator_name - - [VARCHAR(1000)] +dag_id + + [VARCHAR(250)] + NOT NULL -dag_id - - [VARCHAR(250)] - NOT NULL +dag_version_id + + [UUID] -dag_version_id - - [UUID] +duration + + [DOUBLE_PRECISION] -duration - - [DOUBLE_PRECISION] +end_date + + [TIMESTAMP] -end_date - - [TIMESTAMP] +executor + + [VARCHAR(1000)] -executor - - [VARCHAR(1000)] +executor_config + + [BYTEA] -executor_config - - [BYTEA] +external_executor_id + + [VARCHAR(250)] -external_executor_id - - [VARCHAR(250)] +hostname + + [VARCHAR(1000)] -hostname - - [VARCHAR(1000)] +last_heartbeat_at + + [TIMESTAMP] -last_heartbeat_at - - [TIMESTAMP] +map_index + + [INTEGER] + NOT NULL -map_index - - [INTEGER] - NOT NULL +max_tries + + [INTEGER] -max_tries - - [INTEGER] +next_kwargs + + [JSONB] -next_kwargs - - [JSON] +next_method + + [VARCHAR(1000)] -next_method - - [VARCHAR(1000)] +operator + + [VARCHAR(1000)] -operator - - [VARCHAR(1000)] +pid + + [INTEGER] -pid - - [INTEGER] +pool + + [VARCHAR(256)] + NOT NULL -pool - - [VARCHAR(256)] - NOT NULL +pool_slots + + [INTEGER] + NOT NULL -pool_slots - - [INTEGER] - NOT NULL +priority_weight + + [INTEGER] -priority_weight - - [INTEGER] +queue + + [VARCHAR(256)] -queue - - [VARCHAR(256)] +queued_by_job_id + + [INTEGER] -queued_by_job_id - - [INTEGER] +queued_dttm + + [TIMESTAMP] -queued_dttm - - [TIMESTAMP] +rendered_map_index + + [VARCHAR(250)] -rendered_map_index - - [VARCHAR(250)] +run_id + + [VARCHAR(250)] + NOT NULL -run_id - - [VARCHAR(250)] - NOT NULL +span_status + + [VARCHAR(250)] + NOT NULL start_date @@ -858,93 +867,93 @@ trigger--task_instance - -0..N + +0..N {0,1} task_reschedule - -task_reschedule + +task_reschedule + +id + + [INTEGER] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL -id - - [INTEGER] - NOT NULL +duration + + [INTEGER] + NOT NULL -dag_id - - [VARCHAR(250)] - NOT NULL +end_date + + [TIMESTAMP] + NOT NULL -duration - - [INTEGER] - NOT NULL +map_index + + [INTEGER] + NOT NULL -end_date - - [TIMESTAMP] - NOT NULL +reschedule_date + + [TIMESTAMP] + NOT NULL -map_index - - [INTEGER] - NOT NULL +run_id + + [VARCHAR(250)] + NOT NULL -reschedule_date - - [TIMESTAMP] - NOT NULL +start_date + + [TIMESTAMP] + NOT NULL -run_id - - [VARCHAR(250)] - NOT NULL +task_id + + [VARCHAR(250)] + NOT NULL -start_date - - [TIMESTAMP] - NOT NULL - -task_id - - [VARCHAR(250)] - NOT NULL - -try_number - - [INTEGER] - NOT NULL +try_number + + [INTEGER] + NOT NULL task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 task_instance--task_reschedule - -0..N -1 + +0..N +1 @@ -984,30 +993,30 @@ task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 task_instance--rendered_task_instance_fields - -0..N -1 + +0..N +1 @@ -1037,7 +1046,7 @@ keys - [JSON] + [JSONB] length @@ -1047,30 +1056,30 @@ task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 task_instance--task_map - -0..N -1 + +0..N +1 @@ -1120,210 +1129,219 @@ task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance--xcom - -0..N -1 + +0..N +1 task_instance_note - -task_instance_note + +task_instance_note + +dag_id + + [VARCHAR(250)] + NOT NULL + +map_index + + [INTEGER] + NOT NULL -dag_id - - [VARCHAR(250)] - NOT NULL +run_id + + [VARCHAR(250)] + NOT NULL -map_index - - [INTEGER] - NOT NULL +task_id + + [VARCHAR(250)] + NOT NULL -run_id - - [VARCHAR(250)] - NOT NULL +content + + [VARCHAR(1000)] -task_id - - [VARCHAR(250)] - NOT NULL +created_at + + [TIMESTAMP] + NOT NULL -content - - [VARCHAR(1000)] +updated_at + + [TIMESTAMP] + NOT NULL -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] +user_id + + [VARCHAR(128)] task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance--task_instance_note - -0..N -1 + +0..N +1 task_instance_history - -task_instance_history + +task_instance_history + +id + + [INTEGER] + NOT NULL + +context_carrier + + [JSONB] -id - - [INTEGER] - NOT NULL +custom_operator_name + + [VARCHAR(1000)] -custom_operator_name - - [VARCHAR(1000)] +dag_id + + [VARCHAR(250)] + NOT NULL -dag_id - - [VARCHAR(250)] - NOT NULL +dag_version_id + + [UUID] -dag_version_id - - [UUID] +duration + + [DOUBLE_PRECISION] -duration - - [DOUBLE_PRECISION] +end_date + + [TIMESTAMP] -end_date - - [TIMESTAMP] +executor + + [VARCHAR(1000)] -executor - - [VARCHAR(1000)] +executor_config + + [BYTEA] -executor_config - - [BYTEA] +external_executor_id + + [VARCHAR(250)] -external_executor_id - - [VARCHAR(250)] +hostname + + [VARCHAR(1000)] -hostname - - [VARCHAR(1000)] +map_index + + [INTEGER] + NOT NULL -map_index - - [INTEGER] - NOT NULL +max_tries + + [INTEGER] -max_tries - - [INTEGER] +next_kwargs + + [JSONB] -next_kwargs - - [JSON] +next_method + + [VARCHAR(1000)] -next_method - - [VARCHAR(1000)] +operator + + [VARCHAR(1000)] -operator - - [VARCHAR(1000)] +pid + + [INTEGER] -pid - - [INTEGER] +pool + + [VARCHAR(256)] + NOT NULL -pool - - [VARCHAR(256)] - NOT NULL +pool_slots + + [INTEGER] + NOT NULL -pool_slots - - [INTEGER] - NOT NULL +priority_weight + + [INTEGER] -priority_weight - - [INTEGER] +queue + + [VARCHAR(256)] -queue - - [VARCHAR(256)] +queued_by_job_id + + [INTEGER] -queued_by_job_id - - [INTEGER] +queued_dttm + + [TIMESTAMP] -queued_dttm - - [TIMESTAMP] +rendered_map_index + + [VARCHAR(250)] -rendered_map_index - - [VARCHAR(250)] +run_id + + [VARCHAR(250)] + NOT NULL -run_id - - [VARCHAR(250)] - NOT NULL +span_status + + [VARCHAR(250)] + NOT NULL start_date @@ -1366,30 +1384,30 @@ task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 task_instance--task_instance_history - -0..N -1 + +0..N +1 @@ -1731,369 +1749,382 @@ deadline - -deadline - -id - - [UUID] - NOT NULL - -callback - - [VARCHAR(500)] - NOT NULL - -callback_kwargs - - [JSON] - -dag_id - - [VARCHAR(250)] - -dagrun_id - - [INTEGER] - -deadline - - [TIMESTAMP] - NOT NULL + +deadline + +id + + [UUID] + NOT NULL + +callback + + [VARCHAR(500)] + NOT NULL + +callback_kwargs + + [JSON] + +dag_id + + [VARCHAR(250)] + +dagrun_id + + [INTEGER] + +deadline + + [TIMESTAMP] + NOT NULL dag--deadline - -0..N + +0..N {0,1} dag_version--task_instance - -0..N -{0,1} + +0..N +{0,1} dag_run - -dag_run - -id - - [INTEGER] - NOT NULL - -backfill_id - - [INTEGER] - -bundle_version - - [VARCHAR(250)] - -clear_number - - [INTEGER] - NOT NULL - -conf - - [JSONB] - -creating_job_id - - [INTEGER] - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - -data_interval_end - - [TIMESTAMP] - -data_interval_start - - [TIMESTAMP] - -end_date - - [TIMESTAMP] - -external_trigger - - [BOOLEAN] - -last_scheduling_decision - - [TIMESTAMP] - -log_template_id - - [INTEGER] - -logical_date - - [TIMESTAMP] - NOT NULL - -queued_at - - [TIMESTAMP] - -run_id - - [VARCHAR(250)] - NOT NULL - -run_type - - [VARCHAR(50)] - NOT NULL - -start_date - - [TIMESTAMP] - -state - - [VARCHAR(50)] - -triggered_by - - [VARCHAR(50)] - -updated_at - - [TIMESTAMP] + +dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + +bundle_version + + [VARCHAR(250)] + +clear_number + + [INTEGER] + NOT NULL + +conf + + [JSONB] + +context_carrier + + [JSONB] + +creating_job_id + + [INTEGER] + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + +data_interval_end + + [TIMESTAMP] + +data_interval_start + + [TIMESTAMP] + +end_date + + [TIMESTAMP] + +external_trigger + + [BOOLEAN] + +last_scheduling_decision + + [TIMESTAMP] + +log_template_id + + [INTEGER] + +logical_date + + [TIMESTAMP] + NOT NULL + +queued_at + + [TIMESTAMP] + +run_id + + [VARCHAR(250)] + NOT NULL + +run_type + + [VARCHAR(50)] + NOT NULL + +scheduled_by_job_id + + [INTEGER] + +span_status + + [VARCHAR(250)] + NOT NULL + +start_date + + [TIMESTAMP] + +state + + [VARCHAR(50)] + +triggered_by + + [VARCHAR(50)] + +updated_at + + [TIMESTAMP] dag_version--dag_run - -0..N -{0,1} + +0..N +{0,1} dag_code - -dag_code - -id - - [UUID] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - NOT NULL - -fileloc - - [VARCHAR(2000)] - NOT NULL - -last_updated - - [TIMESTAMP] - NOT NULL - -source_code - - [TEXT] - NOT NULL - -source_code_hash - - [VARCHAR(32)] - NOT NULL + +dag_code + +id + + [UUID] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + NOT NULL + +fileloc + + [VARCHAR(2000)] + NOT NULL + +last_updated + + [TIMESTAMP] + NOT NULL + +source_code + + [TEXT] + NOT NULL + +source_code_hash + + [VARCHAR(32)] + NOT NULL dag_version--dag_code - -0..N -1 + +0..N +1 serialized_dag - -serialized_dag - -id - - [UUID] - NOT NULL - -created_at - - [TIMESTAMP] - NOT NULL - -dag_hash - - [VARCHAR(32)] - NOT NULL - -dag_id - - [VARCHAR(250)] - NOT NULL - -dag_version_id - - [UUID] - NOT NULL - -data - - [JSON] - -data_compressed - - [BYTEA] + +serialized_dag + +id + + [UUID] + NOT NULL + +created_at + + [TIMESTAMP] + NOT NULL + +dag_hash + + [VARCHAR(32)] + NOT NULL + +dag_id + + [VARCHAR(250)] + NOT NULL + +dag_version_id + + [UUID] + NOT NULL + +data + + [JSON] + +data_compressed + + [BYTEA] dag_version--serialized_dag - -0..N -1 + +0..N +1 dag_run--dagrun_asset_event - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 dag_run--task_instance - -0..N -1 + +0..N +1 dag_run--deadline - -0..N -{0,1} + +0..N +{0,1} backfill_dag_run - -backfill_dag_run - -id - - [INTEGER] - NOT NULL - -backfill_id - - [INTEGER] - NOT NULL - -dag_run_id - - [INTEGER] - -exception_reason - - [VARCHAR(250)] - -logical_date - - [TIMESTAMP] - NOT NULL - -sort_ordinal - - [INTEGER] - NOT NULL + +backfill_dag_run + +id + + [INTEGER] + NOT NULL + +backfill_id + + [INTEGER] + NOT NULL + +dag_run_id + + [INTEGER] + +exception_reason + + [VARCHAR(250)] + +logical_date + + [TIMESTAMP] + NOT NULL + +sort_ordinal + + [INTEGER] + NOT NULL dag_run--backfill_dag_run - -0..N -{0,1} + +0..N +{0,1} dag_run_note - -dag_run_note - -dag_run_id - - [INTEGER] - NOT NULL - -content - - [VARCHAR(1000)] - -created_at - - [TIMESTAMP] - NOT NULL - -updated_at - - [TIMESTAMP] - NOT NULL - -user_id - - [VARCHAR(128)] + +dag_run_note + +dag_run_id + + [INTEGER] + NOT NULL + +content + + [VARCHAR(1000)] + +created_at + + [TIMESTAMP] + NOT NULL + +updated_at + + [TIMESTAMP] + NOT NULL + +user_id + + [VARCHAR(128)] dag_run--dag_run_note - -1 -1 + +1 +1 dag_run--task_reschedule - -0..N -1 + +0..N +1 dag_run--task_reschedule - -0..N -1 + +0..N +1 @@ -2124,9 +2155,9 @@ log_template--dag_run - -0..N -{0,1} + +0..N +{0,1} @@ -2190,16 +2221,16 @@ backfill--dag_run - -0..N -{0,1} + +0..N +{0,1} backfill--backfill_dag_run - -0..N -1 + +0..N +1 From 86b3cc4c1d7800d1bd0ee236c4ccfbf52f8cfbeb Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Fri, 17 Jan 2025 13:58:07 +0200 Subject: [PATCH 45/52] trigger ER diagram creation --- .../migrations/versions/0056_3_0_0_add_new_otel_span_fields.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/migrations/versions/0056_3_0_0_add_new_otel_span_fields.py b/airflow/migrations/versions/0056_3_0_0_add_new_otel_span_fields.py index e8190c3cf6d0e..802e7d55e745f 100644 --- a/airflow/migrations/versions/0056_3_0_0_add_new_otel_span_fields.py +++ b/airflow/migrations/versions/0056_3_0_0_add_new_otel_span_fields.py @@ -42,7 +42,7 @@ def upgrade(): """Apply add new otel span fields.""" - op.add_column("dag_run", sa.Column("scheduled_by_job_id", sa.Integer, nullable=True)) + op.add_column("dag_run", sa.Column("scheduled_by_job_id1", sa.Integer, nullable=True)) op.add_column("dag_run", sa.Column("context_carrier", ExtendedJSON, nullable=True)) op.add_column("dag_run", sa.Column("span_status", sa.String(250), nullable=False)) From 4004ca5b8cc9730ff401af4d6d43e8f9a45b4435 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Fri, 17 Jan 2025 13:59:35 +0200 Subject: [PATCH 46/52] trigger ER diagram creation --- .../migrations/versions/0056_3_0_0_add_new_otel_span_fields.py | 2 +- docs/apache-airflow/img/airflow_erd.sha256 | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/migrations/versions/0056_3_0_0_add_new_otel_span_fields.py b/airflow/migrations/versions/0056_3_0_0_add_new_otel_span_fields.py index 802e7d55e745f..e8190c3cf6d0e 100644 --- a/airflow/migrations/versions/0056_3_0_0_add_new_otel_span_fields.py +++ b/airflow/migrations/versions/0056_3_0_0_add_new_otel_span_fields.py @@ -42,7 +42,7 @@ def upgrade(): """Apply add new otel span fields.""" - op.add_column("dag_run", sa.Column("scheduled_by_job_id1", sa.Integer, nullable=True)) + op.add_column("dag_run", sa.Column("scheduled_by_job_id", sa.Integer, nullable=True)) op.add_column("dag_run", sa.Column("context_carrier", ExtendedJSON, nullable=True)) op.add_column("dag_run", sa.Column("span_status", sa.String(250), nullable=False)) diff --git a/docs/apache-airflow/img/airflow_erd.sha256 b/docs/apache-airflow/img/airflow_erd.sha256 index 87757d1b6d9ba..a736de2b0791e 100644 --- a/docs/apache-airflow/img/airflow_erd.sha256 +++ b/docs/apache-airflow/img/airflow_erd.sha256 @@ -1 +1 @@ -bc0cc8716c482f9d2e17204ab586c7127d3152a91276b068f2bb102a7d156fa2 \ No newline at end of file +2cc5f17792e8dbbb270482d8d902bcb279a6eaf6b15c14aa443aadfdfdf53b65 \ No newline at end of file From d86a7bf9159bf544d2765f3f91e55ef7859fa569 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Tue, 11 Feb 2025 17:29:50 +0200 Subject: [PATCH 47/52] fix failures after merge with main --- airflow/executors/base_executor.py | 57 ++++++++++++++--------------- tests/integration/otel/test_otel.py | 13 ++++++- 2 files changed, 39 insertions(+), 31 deletions(-) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index 04763ef15b4a3..732e5cde35b7c 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -365,35 +365,6 @@ def trigger_tasks(self, open_slots: int) -> None: for _ in range(min((open_slots, len(self.queued_tasks)))): key, item = sorted_queue.pop(0) - (command, _, queue, ti) = item - - # If it's None, then the span for the current TaskInstanceKey hasn't been started. - if self.active_spans is not None and self.active_spans.get(key) is None: - from airflow.models.taskinstance import SimpleTaskInstance - - if isinstance(ti, SimpleTaskInstance): - parent_context = Trace.extract(ti.parent_context_carrier) - else: - parent_context = Trace.extract(ti.dag_run.context_carrier) - # Start a new span using the context from the parent. - # Attributes will be set once the task has finished so that all - # values will be available (end_time, duration, etc.). - span = Trace.start_child_span( - span_name=f"{ti.task_id}", - parent_context=parent_context, - component="task", - start_time=ti.queued_dttm, - start_as_current=False, - ) - self.active_spans.set(key, span) - # Inject the current context into the carrier. - carrier = Trace.inject() - # The carrier needs to be set on the ti, but it can't happen here because db calls are expensive. - # So set the carrier as an argument to the command. - # The command execution will set it on the ti, and it will be propagated to the task itself. - command = list(command) - command.append("--carrier") - command.append(json.dumps(carrier)) # If a task makes it here but is still understood by the executor # to be running, it generally means that the task has been killed @@ -437,6 +408,34 @@ def trigger_tasks(self, open_slots: int) -> None: if hasattr(self, "_process_workloads"): workloads.append(item) else: + (command, _, queue, ti) = item + # If it's None, then the span for the current TaskInstanceKey hasn't been started. + if self.active_spans is not None and self.active_spans.get(key) is None: + from airflow.models.taskinstance import SimpleTaskInstance + + if isinstance(ti, SimpleTaskInstance): + parent_context = Trace.extract(ti.parent_context_carrier) + else: + parent_context = Trace.extract(ti.dag_run.context_carrier) + # Start a new span using the context from the parent. + # Attributes will be set once the task has finished so that all + # values will be available (end_time, duration, etc.). + span = Trace.start_child_span( + span_name=f"{ti.task_id}", + parent_context=parent_context, + component="task", + start_time=ti.queued_dttm, + start_as_current=False, + ) + self.active_spans.set(key, span) + # Inject the current context into the carrier. + carrier = Trace.inject() + # The carrier needs to be set on the ti, but it can't happen here because db calls are expensive. + # So set the carrier as an argument to the command. + # The command execution will set it on the ti, and it will be propagated to the task itself. + command = list(command) + command.append("--carrier") + command.append(json.dumps(carrier)) task_tuples.append((key, command, queue, getattr(ti, "executor_config", None))) if task_tuples: diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index b515f03266a94..e427f039ac720 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -44,6 +44,7 @@ extract_spans_from_output, get_parent_child_dict, ) +from tests_common.test_utils.version_compat import AIRFLOW_V_3_0_PLUS log = logging.getLogger("integration.otel.test_otel") @@ -626,9 +627,17 @@ def serialize_and_get_dags(cls) -> dict[str, DAG]: assert dag is not None, f"DAG with ID {dag_id} not found." # Sync the DAG to the database. - dag.sync_to_db(session=session) + if AIRFLOW_V_3_0_PLUS: + from airflow.models.dagbundle import DagBundleModel + + if session.query(DagBundleModel).filter(DagBundleModel.name == "testing").count() == 0: + session.add(DagBundleModel(name="testing")) + session.commit() + dag.bulk_write_to_db("testing", None, [dag], session) + else: + dag.sync_to_db(session=session) # Manually serialize the dag and write it to the db to avoid a db error. - SerializedDagModel.write_dag(dag, session=session) + SerializedDagModel.write_dag(dag, bundle_name="testing", session=session) session.commit() From 4d74c933d913e5bcabb587be3ee28e86781383c3 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sun, 9 Mar 2025 14:48:15 +0200 Subject: [PATCH 48/52] fix mypy error --- tests/integration/otel/test_otel.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index e427f039ac720..235c8b1cf2948 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -526,7 +526,9 @@ def print_ti_output(ti: TaskInstance): if task_log_reader.supports_read: metadata: dict[str, Any] = {} logs, metadata = task_log_reader.read_log_chunks(ti, ti.try_number, metadata) - if ti.hostname in dict(logs[0]): + log_entry = logs[0] + assert isinstance(log_entry, dict), f"Expected dict but got: {type(log_entry)}" + if ti.hostname in dict(log_entry): output = ( str(dict(logs[0])[ti.hostname]) .replace("\\n", "\n") From f7af6f9eb9e1eeb3ce82ff677e897069d700b58b Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sun, 9 Mar 2025 14:50:03 +0200 Subject: [PATCH 49/52] fix more mypy errors --- tests/integration/otel/test_otel.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/tests/integration/otel/test_otel.py b/tests/integration/otel/test_otel.py index 235c8b1cf2948..30be12a12bc2c 100644 --- a/tests/integration/otel/test_otel.py +++ b/tests/integration/otel/test_otel.py @@ -536,8 +536,10 @@ def print_ti_output(ti: TaskInstance): ) while metadata["end_of_log"] is False: logs, metadata = task_log_reader.read_log_chunks(ti, ti.try_number - 1, metadata) - if ti.hostname in dict(logs[0]): - output = output + str(dict(logs[0])[ti.hostname]).replace("\\n", "\n") + log_entry = logs[0] + assert isinstance(log_entry, dict), f"Expected dict but got: {type(log_entry)}" + if ti.hostname in dict(log_entry): + output = output + str(dict(log_entry)[ti.hostname]).replace("\\n", "\n") # Logging the output is enough for capfd to capture it. log.info(format(output)) From ed1f566e11711819ee6f40925f140fc2bd204256 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sun, 9 Mar 2025 14:52:28 +0200 Subject: [PATCH 50/52] remove leftover dagrun field external_trigger --- airflow/models/dagrun.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 5624155ed27fd..bd65f3611dae5 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -950,7 +950,6 @@ def set_dagrun_span_attrs(self, span: Span | EmptySpan): (self.end_date - self.start_date).total_seconds() if self.start_date and self.end_date else 0 ), "airflow.dag_run.state": str(self._state), - "airflow.dag_run.external_trigger": str(self.external_trigger), "airflow.dag_run.run_type": str(self.run_type), "airflow.dag_run.data_interval_start": str(self.data_interval_start), "airflow.dag_run.data_interval_end": str(self.data_interval_end), From 4a99dd7502d938e5c61f1e68374a61d313119eb5 Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Sun, 9 Mar 2025 15:32:24 +0200 Subject: [PATCH 51/52] fix test failures after merge --- tests/jobs/test_scheduler_job.py | 6 +++--- tests/models/test_dagrun.py | 6 ++---- 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index e8e6fe55d7e12..7923e6dbc26cd 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -2355,7 +2355,7 @@ def test_recreate_unhealthy_scheduler_spans_if_needed(self, ti_state, final_ti_s self.job_runner.active_spans = ThreadSafeDict() assert len(self.job_runner.active_spans.get_all()) == 0 - dr = dag_maker.create_dagrun(external_trigger=True) + dr = dag_maker.create_dagrun() dr.state = State.RUNNING dr.span_status = SpanStatus.ACTIVE dr.scheduled_by_job_id = old_job.id @@ -2416,7 +2416,7 @@ def test_end_spans_of_externally_ended_ops(self, dag_maker): self.job_runner.active_spans = ThreadSafeDict() assert len(self.job_runner.active_spans.get_all()) == 0 - dr = dag_maker.create_dagrun(external_trigger=True) + dr = dag_maker.create_dagrun() dr.state = State.SUCCESS dr.span_status = SpanStatus.SHOULD_END @@ -2474,7 +2474,7 @@ def test_end_active_spans(self, state, final_span_status, dag_maker): self.job_runner.active_spans = ThreadSafeDict() assert len(self.job_runner.active_spans.get_all()) == 0 - dr = dag_maker.create_dagrun(external_trigger=True) + dr = dag_maker.create_dagrun() dr.state = state dr.span_status = SpanStatus.ACTIVE diff --git a/tests/models/test_dagrun.py b/tests/models/test_dagrun.py index ac5ba0035a4b3..78d94ac714c02 100644 --- a/tests/models/test_dagrun.py +++ b/tests/models/test_dagrun.py @@ -582,8 +582,7 @@ def test_end_dr_span_if_needed(self, testing_dag_bundle, dag_maker, session): assert dag_run.active_spans is not None assert dag_run.active_spans.get(dag_run.run_id) is not None - dag_version = DagVersion.get_latest_version(dag.dag_id) - dag_run.end_dr_span_if_needed(dagv=dag_version) + dag_run.end_dr_span_if_needed() assert dag_run.span_status == SpanStatus.ENDED assert dag_run.active_spans.get(dag_run.run_id) is None @@ -621,8 +620,7 @@ def test_end_dr_span_if_needed_with_span_from_another_scheduler( assert dag_run.active_spans is not None assert dag_run.active_spans.get(dag_run.run_id) is None - dag_version = DagVersion.get_latest_version(dag.dag_id) - dag_run.end_dr_span_if_needed(dagv=dag_version) + dag_run.end_dr_span_if_needed() assert dag_run.span_status == SpanStatus.SHOULD_END From 8b90db62eae27e96282eaf924ff3b69f1d43f90a Mon Sep 17 00:00:00 2001 From: Christos Bisias Date: Wed, 12 Mar 2025 20:46:25 +0200 Subject: [PATCH 52/52] set the dr context on the workload ti --- airflow/executors/base_executor.py | 40 +++++++++++++++++++----------- airflow/executors/workloads.py | 5 ++++ 2 files changed, 31 insertions(+), 14 deletions(-) diff --git a/airflow/executors/base_executor.py b/airflow/executors/base_executor.py index 395bbdae98fe3..fc5e41b9157bb 100644 --- a/airflow/executors/base_executor.py +++ b/airflow/executors/base_executor.py @@ -359,7 +359,7 @@ def trigger_tasks(self, open_slots: int) -> None: """ sorted_queue = self.order_queued_tasks_by_priority() task_tuples = [] - workloads = [] + workload_list = [] for _ in range(min((open_slots, len(self.queued_tasks)))): key, item = sorted_queue.pop(0) @@ -403,43 +403,55 @@ def trigger_tasks(self, open_slots: int) -> None: # TODO: TaskSDK: Compat, remove when KubeExecutor is fully moved over to TaskSDK too. # TODO: TaskSDK: We need to minimum version requirements on executors with Airflow 3. # How/where do we do that? Executor loader? - if hasattr(self, "_process_workloads"): - workloads.append(item) - else: - (command, _, queue, ti) = item + from airflow.executors import workloads + + carrier: dict = {} + if isinstance(item, workloads.TaskInstance) and hasattr(item, "ti"): + ti = item.ti + # If it's None, then the span for the current TaskInstanceKey hasn't been started. if self.active_spans is not None and self.active_spans.get(key) is None: from airflow.models.taskinstance import SimpleTaskInstance if isinstance(ti, SimpleTaskInstance): parent_context = Trace.extract(ti.parent_context_carrier) + elif isinstance(ti, workloads.TaskInstance): + parent_context = Trace.extract(ti.parent_context_carrier) else: parent_context = Trace.extract(ti.dag_run.context_carrier) # Start a new span using the context from the parent. # Attributes will be set once the task has finished so that all # values will be available (end_time, duration, etc.). + from airflow.utils import timezone + span = Trace.start_child_span( span_name=f"{ti.task_id}", parent_context=parent_context, component="task", - start_time=ti.queued_dttm, + start_time=timezone.utcnow(), start_as_current=False, ) self.active_spans.set(key, span) # Inject the current context into the carrier. carrier = Trace.inject() - # The carrier needs to be set on the ti, but it can't happen here because db calls are expensive. - # So set the carrier as an argument to the command. - # The command execution will set it on the ti, and it will be propagated to the task itself. - command = list(command) - command.append("--carrier") - command.append(json.dumps(carrier)) + ti.context_carrier = carrier + + if hasattr(self, "_process_workloads"): + workload_list.append(item) + else: + (command, _, queue, ti) = item + # The carrier needs to be set on the ti, but it can't happen here because db calls are expensive. + # So set the carrier as an argument to the command. + # The command execution will set it on the ti, and it will be propagated to the task itself. + command = list(command) + command.append("--carrier") + command.append(json.dumps(carrier)) task_tuples.append((key, command, queue, getattr(ti, "executor_config", None))) if task_tuples: self._process_tasks(task_tuples) - elif workloads: - self._process_workloads(workloads) # type: ignore[attr-defined] + elif workload_list: + self._process_workloads(workload_list) # type: ignore[attr-defined] @add_span def _process_tasks(self, task_tuples: list[TaskTuple]) -> None: diff --git a/airflow/executors/workloads.py b/airflow/executors/workloads.py index 00ca97405a04c..b203cc7e82122 100644 --- a/airflow/executors/workloads.py +++ b/airflow/executors/workloads.py @@ -63,6 +63,10 @@ class TaskInstance(BaseModel): priority_weight: int executor_config: dict | None = None + parent_context_carrier: dict | None = None + context_carrier: dict | None = None + queued_dttm: datetime | None = None + # TODO: Task-SDK: Can we replace TastInstanceKey with just the uuid across the codebase? @property def key(self) -> TaskInstanceKey: @@ -99,6 +103,7 @@ def make(cls, ti: TIModel, dag_rel_path: Path | None = None) -> ExecuteTask: from airflow.utils.helpers import log_filename_template_renderer ser_ti = TaskInstance.model_validate(ti, from_attributes=True) + ser_ti.parent_context_carrier = ti.dag_run.context_carrier bundle_info = BundleInfo( name=ti.dag_model.bundle_name, version=ti.dag_run.bundle_version,