From cade71efbd37071ad5256564124079f1e46c20a4 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 09:48:40 -0700 Subject: [PATCH 01/53] Refactor tests re sensor reschedule mode and try_number It's a little clearer what's going on when we're not in a loop or behind an `assert...` helper function. Additionally I add asserts immediately after calling `clear`. This will make it easier to review changes when we attempt to fix try_number shenanigans. --- tests/sensors/test_base.py | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/tests/sensors/test_base.py b/tests/sensors/test_base.py index a41e6836838d5..fce98de418dc1 100644 --- a/tests/sensors/test_base.py +++ b/tests/sensors/test_base.py @@ -297,7 +297,17 @@ def _get_tis(): date1 = timezone.utcnow() time_machine.move_to(date1, tick=False) + sensor_ti, dummy_ti = _get_tis() + assert dummy_ti.state == State.NONE + assert sensor_ti.state == State.NONE + + # prepare for first run + sensor_ti.state = State.SCHEDULED + + session.commit() + self._run(sensor, session=session) + sensor_ti, dummy_ti = _get_tis() assert sensor_ti.state == State.UP_FOR_RESCHEDULE assert dummy_ti.state == State.NONE @@ -359,6 +369,8 @@ def _get_tis(): # first poke returns False and task is re-scheduled date1 = timezone.utcnow() time_machine.move_to(date1, tick=False) + sensor_ti, dummy_ti = _get_tis() + session.commit() self._run(sensor) sensor_ti, dummy_ti = _get_tis() assert sensor_ti.state == State.UP_FOR_RESCHEDULE From c04e0f52658788797f280ee5f02969a154116df7 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 10:00:54 -0700 Subject: [PATCH 02/53] fix --- tests/sensors/test_base.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/sensors/test_base.py b/tests/sensors/test_base.py index fce98de418dc1..f48a89f68bf6d 100644 --- a/tests/sensors/test_base.py +++ b/tests/sensors/test_base.py @@ -301,11 +301,6 @@ def _get_tis(): assert dummy_ti.state == State.NONE assert sensor_ti.state == State.NONE - # prepare for first run - sensor_ti.state = State.SCHEDULED - - session.commit() - self._run(sensor, session=session) sensor_ti, dummy_ti = _get_tis() From 31e2c2137ebf3366b5534a5a20f9442c8b7b9411 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 10:01:30 -0700 Subject: [PATCH 03/53] fix --- tests/sensors/test_base.py | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tests/sensors/test_base.py b/tests/sensors/test_base.py index f48a89f68bf6d..330caa6fee931 100644 --- a/tests/sensors/test_base.py +++ b/tests/sensors/test_base.py @@ -297,12 +297,7 @@ def _get_tis(): date1 = timezone.utcnow() time_machine.move_to(date1, tick=False) - sensor_ti, dummy_ti = _get_tis() - assert dummy_ti.state == State.NONE - assert sensor_ti.state == State.NONE - self._run(sensor, session=session) - sensor_ti, dummy_ti = _get_tis() assert sensor_ti.state == State.UP_FOR_RESCHEDULE assert dummy_ti.state == State.NONE From ad2ccd0066235356bfa2b868f6d536c8f83b9bca Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 10:03:38 -0700 Subject: [PATCH 04/53] fix --- tests/sensors/test_base.py | 2 -- 1 file changed, 2 deletions(-) diff --git a/tests/sensors/test_base.py b/tests/sensors/test_base.py index 330caa6fee931..a41e6836838d5 100644 --- a/tests/sensors/test_base.py +++ b/tests/sensors/test_base.py @@ -359,8 +359,6 @@ def _get_tis(): # first poke returns False and task is re-scheduled date1 = timezone.utcnow() time_machine.move_to(date1, tick=False) - sensor_ti, dummy_ti = _get_tis() - session.commit() self._run(sensor) sensor_ti, dummy_ti = _get_tis() assert sensor_ti.state == State.UP_FOR_RESCHEDULE From 47ae9a62b6fadd85b9feb1c902da1fffc7519d1c Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 30 Apr 2024 14:16:01 -0700 Subject: [PATCH 05/53] Remove try_number shenanigans Previously, there was a lot of bad stuff happening around try_number. We incremented it when task started running. And because of that, we had this logic to return "_try_number + 1" when task *not* running. And, since it was naively incremented when task starts running -- i.e. without regard to why it is running -- we *decremented* it when deferring or exiting on a reschedule. What I do here is try to remove *all* of that stuff: * no more private _try_number attr * no more getter logic * no more decrementing * no more incrementing as part of task execution Now what we do is increment only when the task is set to *scheduled* and only when it's not coming out of deferral or "up_for_reschedule". --- airflow/api/common/mark_tasks.py | 4 - .../schemas/task_instance_schema.py | 2 +- .../decreasing_priority_weight_strategy.py | 2 +- airflow/models/dag.py | 1 + airflow/models/dagrun.py | 29 +++-- airflow/models/taskinstance.py | 111 ++---------------- .../amazon/aws/executors/ecs/ecs_executor.py | 2 +- .../elasticsearch/log/es_task_handler.py | 2 +- airflow/serialization/serialized_objects.py | 4 + airflow/utils/log/file_task_handler.py | 8 +- airflow/www/utils.py | 2 +- airflow/www/views.py | 8 +- tests/jobs/test_backfill_job.py | 19 ++- tests/models/test_taskinstance.py | 67 +++++------ tests/plugins/priority_weight_strategy.py | 2 +- tests/test_utils/mock_executor.py | 2 +- 16 files changed, 83 insertions(+), 182 deletions(-) diff --git a/airflow/api/common/mark_tasks.py b/airflow/api/common/mark_tasks.py index a9d8325e5ba64..76f279b02e2e0 100644 --- a/airflow/api/common/mark_tasks.py +++ b/airflow/api/common/mark_tasks.py @@ -158,10 +158,6 @@ def set_state( qry_sub_dag = all_subdag_tasks_query(sub_dag_run_ids, session, state, confirmed_dates) tis_altered += session.scalars(qry_sub_dag.with_for_update()).all() for task_instance in tis_altered: - # The try_number was decremented when setting to up_for_reschedule and deferred. - # Increment it back when changing the state again - if task_instance.state in (TaskInstanceState.DEFERRED, TaskInstanceState.UP_FOR_RESCHEDULE): - task_instance._try_number += 1 task_instance.set_state(state, session=session) session.flush() else: diff --git a/airflow/api_connexion/schemas/task_instance_schema.py b/airflow/api_connexion/schemas/task_instance_schema.py index 4777b8bd4c577..f4ea4bdddf72b 100644 --- a/airflow/api_connexion/schemas/task_instance_schema.py +++ b/airflow/api_connexion/schemas/task_instance_schema.py @@ -53,7 +53,7 @@ class Meta: end_date = auto_field() duration = auto_field() state = TaskInstanceStateField() - _try_number = auto_field(data_key="try_number") + try_number = auto_field() max_tries = auto_field() task_display_name = fields.String(attribute="task_display_name", dump_only=True) hostname = auto_field() diff --git a/airflow/example_dags/plugins/decreasing_priority_weight_strategy.py b/airflow/example_dags/plugins/decreasing_priority_weight_strategy.py index 44e6bad432a60..3335b7da4ee77 100644 --- a/airflow/example_dags/plugins/decreasing_priority_weight_strategy.py +++ b/airflow/example_dags/plugins/decreasing_priority_weight_strategy.py @@ -30,7 +30,7 @@ class DecreasingPriorityStrategy(PriorityWeightStrategy): """A priority weight strategy that decreases the priority weight with each attempt of the DAG task.""" def get_weight(self, ti: TaskInstance): - return max(3 - ti._try_number + 1, 1) + return max(3 - ti.try_number + 1, 1) class DecreasingPriorityWeightStrategyPlugin(AirflowPlugin): diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 9db0de45a036d..05c7ca3d13af2 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -2949,6 +2949,7 @@ def add_logger_if_needed(ti: TaskInstance): schedulable_tis, _ = dr.update_state(session=session) for s in schedulable_tis: s.state = TaskInstanceState.SCHEDULED + # s.try_number += 1 # todo: should we increase try number? session.commit() # triggerer may mark tasks scheduled so we read from DB all_tis = set(dr.get_task_instances(session=session)) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 117dd59e288ba..ae7f555a270a4 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -45,7 +45,7 @@ from sqlalchemy.exc import IntegrityError from sqlalchemy.ext.associationproxy import association_proxy from sqlalchemy.orm import declared_attr, joinedload, relationship, synonym, validates -from sqlalchemy.sql.expression import false, select, true +from sqlalchemy.sql.expression import case, false, select, true from airflow import settings from airflow.api_internal.internal_api_call import internal_api_call @@ -1077,7 +1077,6 @@ def _expand_mapped_task_if_needed(ti: TI) -> Iterable[TI] | None: if tis_filter is not None: fresh_tis = session.scalars(select(TI).where(tis_filter)).all() changed_tis = any(ti.state != old_states[ti.key] for ti in fresh_tis) - return ready_tis, changed_tis, expansion_happened def _are_premature_tis( @@ -1545,7 +1544,6 @@ def schedule_tis( and not ti.task.on_success_callback and not ti.task.outlets ): - ti._try_number += 1 ti.defer_task( defer=TaskDeferred(trigger=ti.task.start_trigger, method_name=ti.task.next_method), session=session, @@ -1560,16 +1558,27 @@ def schedule_tis( schedulable_ti_ids, max_tis_per_query or len(schedulable_ti_ids) ) for schedulable_ti_ids_chunk in schedulable_ti_ids_chunks: - count += session.execute( + schedulable_cond = ( + TI.dag_id == self.dag_id, + TI.run_id == self.run_id, + tuple_in_condition((TI.task_id, TI.map_index), schedulable_ti_ids_chunk), + ) + update_stmt = ( update(TI) - .where( - TI.dag_id == self.dag_id, - TI.run_id == self.run_id, - tuple_in_condition((TI.task_id, TI.map_index), schedulable_ti_ids_chunk), + .where(*schedulable_cond) + .values( + state=TaskInstanceState.SCHEDULED, + try_number=case( + ( + or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE), + TI.try_number + 1, + ), + else_=TI.try_number, + ), ) - .values(state=TaskInstanceState.SCHEDULED) .execution_options(synchronize_session=False) - ).rowcount + ) + count += session.execute(update_stmt).rowcount # Tasks using EmptyOperator should not be executed, mark them as success if dummy_ti_ids: diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 1a9d1e00362ef..df654f96bb4b4 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -288,7 +288,7 @@ def clear_task_instances( # task are not found since database records could be # outdated. We make max_tries the maximum value of its # original max_tries or the last attempted try number. - ti.max_tries = max(ti.max_tries, ti.prev_attempted_tries) + ti.max_tries = max(ti.max_tries, ti.try_number) ti.state = None ti.external_executor_id = None ti.clear_next_method_args() @@ -539,7 +539,7 @@ def _refresh_from_db( task_instance.end_date = ti.end_date task_instance.duration = ti.duration task_instance.state = ti.state - task_instance.try_number = _get_private_try_number(task_instance=ti) + task_instance.try_number = ti.try_number task_instance.max_tries = ti.max_tries task_instance.hostname = ti.hostname task_instance.unixname = ti.unixname @@ -928,53 +928,6 @@ def _handle_failure( TaskInstance.save_to_db(failure_context["ti"], session) -def _get_try_number(*, task_instance: TaskInstance): - """ - Return the try number that a task number will be when it is actually run. - - If the TaskInstance is currently running, this will match the column in the - database, in all other cases this will be incremented. - - This is designed so that task logs end up in the right file. - - :param task_instance: the task instance - - :meta private: - """ - if task_instance.state == TaskInstanceState.RUNNING: - return task_instance._try_number - return task_instance._try_number + 1 - - -def _get_private_try_number(*, task_instance: TaskInstance | TaskInstancePydantic): - """ - Opposite of _get_try_number. - - Given the value returned by try_number, return the value of _try_number that - should produce the same result. - This is needed for setting _try_number on TaskInstance from the value on PydanticTaskInstance, which has no private attrs. - - :param task_instance: the task instance - - :meta private: - """ - if task_instance.state == TaskInstanceState.RUNNING: - return task_instance.try_number - return task_instance.try_number - 1 - - -def _set_try_number(*, task_instance: TaskInstance | TaskInstancePydantic, value: int) -> None: - """ - Set a task try number. - - :param task_instance: the task instance - :param value: the try number - - :meta private: - """ - task_instance._try_number = value # type: ignore[union-attr] - - def _refresh_from_task( *, task_instance: TaskInstance | TaskInstancePydantic, task: Operator, pool_override: str | None = None ) -> None: @@ -1020,6 +973,8 @@ def _record_task_map_for_downstreams( :meta private: """ + # todo: when sering task then be smarter than just setting dag to None. use sentinel + # to indicate it was attached and when re-setting .dag, skip the extra stuff if next(task.iter_mapped_dependants(), None) is None: # No mapped dependants, no need to validate. return # TODO: We don't push TaskMap for mapped task instances because it's not @@ -1164,13 +1119,10 @@ def _get_email_subject_content( 'Mark success: Link
' ) - # This function is called after changing the state from RUNNING, - # so we need to subtract 1 from self.try_number here. - current_try_number = task_instance.try_number - 1 additional_context: dict[str, Any] = { "exception": exception, "exception_html": exception_html, - "try_number": current_try_number, + "try_number": task_instance.try_number, "max_tries": task_instance.max_tries, } @@ -1343,7 +1295,7 @@ class TaskInstance(Base, LoggingMixin): end_date = Column(UtcDateTime) duration = Column(Float) state = Column(String(20)) - _try_number = Column("try_number", Integer, default=0) + try_number = Column(Integer, default=0) max_tries = Column(Integer, server_default=text("-1")) hostname = Column(String(1000)) unixname = Column(String(1000)) @@ -1527,7 +1479,7 @@ def insert_mapping(run_id: str, task: Operator, map_index: int) -> dict[str, Any "dag_id": task.dag_id, "task_id": task.task_id, "run_id": run_id, - "_try_number": 0, + "try_number": 0, "hostname": "", "unixname": getuser(), "queue": task.queue, @@ -1549,39 +1501,6 @@ def init_on_load(self) -> None: """Initialize the attributes that aren't stored in the DB.""" self.test_mode = False # can be changed when calling 'run' - @hybrid_property - def try_number(self): - """ - Return the try number that a task number will be when it is actually run. - - If the TaskInstance is currently running, this will match the column in the - database, in all other cases this will be incremented. - - This is designed so that task logs end up in the right file. - """ - return _get_try_number(task_instance=self) - - @try_number.expression - def try_number(cls): - """ - Return the expression to be used by SQLAlchemy when filtering on try_number. - - This is required because the override in the get_try_number function causes - try_number values to be off by one when listing tasks in the UI. - - :meta private: - """ - return cls._try_number - - @try_number.setter - def try_number(self, value: int) -> None: - """ - Set a task try number. - - :param value: the try number - """ - _set_try_number(task_instance=self, value=value) - @property def prev_attempted_tries(self) -> int: """ @@ -1591,11 +1510,11 @@ def prev_attempted_tries(self) -> int: Using `try_number` throws off the counts for non-running tasks. Also useful in error logging contexts to get the try number for the last try that was attempted. """ - return self._try_number + return self.try_number @property def next_try_number(self) -> int: - return self._try_number + 1 + return self.try_number + 1 @property def operator_name(self) -> str | None: @@ -2336,6 +2255,7 @@ def _check_and_change_state_before_execution( # If the task continues after being deferred (next_method is set), use the original start_date ti.start_date = ti.start_date if ti.next_method else timezone.utcnow() if ti.state == TaskInstanceState.UP_FOR_RESCHEDULE: + log.info("in state up_for_reschedule: task_id=%s", ti.task_id) tr_start_date = session.scalar( TR.stmt_for_task_instance(ti, descending=False).with_only_columns(TR.start_date).limit(1) ) @@ -2372,7 +2292,6 @@ def _check_and_change_state_before_execution( cls.logger().info("Resuming after deferral") else: cls.logger().info("Starting attempt %s of %s", ti.try_number, ti.max_tries + 1) - ti._try_number += 1 if not test_mode: session.add(Log(TaskInstanceState.RUNNING.value, ti)) @@ -2791,9 +2710,6 @@ def defer_task(self, session: Session, defer: TaskDeferred) -> None: self.next_method = defer.method_name self.next_kwargs = defer.kwargs or {} - # Decrement try number so the next one is the same try - self._try_number -= 1 - # Calculate timeout too if it was passed if defer.timeout is not None: self.trigger_timeout = timezone.utcnow() + defer.timeout @@ -2910,7 +2826,7 @@ def _handle_reschedule( self.task_id, self.dag_id, self.run_id, - self._try_number, + self.try_number, actual_start_date, self.end_date, reschedule_exception.reschedule_date, @@ -2921,10 +2837,6 @@ def _handle_reschedule( # set state self.state = TaskInstanceState.UP_FOR_RESCHEDULE - # Decrement try_number so subsequent runs will use the same try number and write - # to same log file. - self._try_number -= 1 - self.clear_next_method_args() session.merge(self) @@ -3040,7 +2952,6 @@ def fetch_handle_failure_context( # e.g. we could make refresh_from_db return a TI and replace ti with that raise RuntimeError("Expected TaskInstance here. Further AIP-44 work required.") # We increase the try_number to fail the task if it fails to start after sometime - ti._try_number += 1 ti.state = State.UP_FOR_RETRY email_for_state = operator.attrgetter("email_on_retry") callbacks = task.on_retry_callback if task else None diff --git a/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py b/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py index c5e7e3d6b46c3..a62058a4235a2 100644 --- a/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py +++ b/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py @@ -527,7 +527,7 @@ def try_adopt_task_instances(self, tis: Sequence[TaskInstance]) -> Sequence[Task ti.queue, ti.command_as_list(), ti.executor_config, - ti.prev_attempted_tries, + ti.try_number, # prev attempted tries ) adopted_tis.append(ti) diff --git a/airflow/providers/elasticsearch/log/es_task_handler.py b/airflow/providers/elasticsearch/log/es_task_handler.py index cfc7b49e9edd7..82cc887553d65 100644 --- a/airflow/providers/elasticsearch/log/es_task_handler.py +++ b/airflow/providers/elasticsearch/log/es_task_handler.py @@ -108,7 +108,7 @@ def _ensure_ti(ti: TaskInstanceKey | TaskInstance, session) -> TaskInstance: .one_or_none() ) if isinstance(val, TaskInstance): - val._try_number = ti.try_number + val.try_number = ti.try_number return val else: raise AirflowException(f"Could not find TaskInstance for {ti}") diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py index 669204376694e..80b256e797589 100644 --- a/airflow/serialization/serialized_objects.py +++ b/airflow/serialization/serialized_objects.py @@ -551,8 +551,12 @@ def serialize( elif isinstance(var, Resources): return var.to_dict() elif isinstance(var, MappedOperator): + # todo: try to determine should_expand here + # todo: try to determine fail_stop here return cls._encode(SerializedBaseOperator.serialize_mapped_operator(var), type_=DAT.OP) elif isinstance(var, BaseOperator): + # todo: try to determine should_expand here + # todo: try to determine fail_stop here return cls._encode(SerializedBaseOperator.serialize_operator(var), type_=DAT.OP) elif isinstance(var, cls._datetime_types): return cls._encode(var.timestamp(), type_=DAT.DATETIME) diff --git a/airflow/utils/log/file_task_handler.py b/airflow/utils/log/file_task_handler.py index 2a1dfd25f6410..043f59d52e895 100644 --- a/airflow/utils/log/file_task_handler.py +++ b/airflow/utils/log/file_task_handler.py @@ -145,8 +145,7 @@ def _ensure_ti(ti: TaskInstanceKey | TaskInstance | TaskInstancePydantic, sessio Will raise exception if no TI is found in the database. """ - from airflow.models.taskinstance import TaskInstance, _get_private_try_number - from airflow.serialization.pydantic.taskinstance import TaskInstancePydantic + from airflow.models.taskinstance import TaskInstance if isinstance(ti, TaskInstance): return ti @@ -162,10 +161,7 @@ def _ensure_ti(ti: TaskInstanceKey | TaskInstance | TaskInstancePydantic, sessio ) if not val: raise AirflowException(f"Could not find TaskInstance for {ti}") - if isinstance(ti, TaskInstancePydantic): - val.try_number = _get_private_try_number(task_instance=ti) - else: # TaskInstanceKey - val.try_number = ti.try_number + val.try_number = ti.try_number return val diff --git a/airflow/www/utils.py b/airflow/www/utils.py index 513b453006fd1..4235ab597f9ea 100644 --- a/airflow/www/utils.py +++ b/airflow/www/utils.py @@ -147,7 +147,7 @@ def get_mapped_summary(parent_instance, task_instances): "start_date": group_start_date, "end_date": group_end_date, "mapped_states": mapped_states, - "try_number": get_try_count(parent_instance._try_number, parent_instance.state), + "try_number": get_try_count(parent_instance.try_number, parent_instance.state), "execution_date": parent_instance.execution_date, } diff --git a/airflow/www/views.py b/airflow/www/views.py index 682a70c8ea0fe..cce04d2a02f6c 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -314,7 +314,7 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session) -> TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, - TaskInstance._try_number, + TaskInstance.try_number, func.min(TaskInstanceNote.content).label("note"), func.count(func.coalesce(TaskInstance.state, sqla.literal("no_status"))).label("state_count"), func.min(TaskInstance.queued_dttm).label("queued_dttm"), @@ -326,7 +326,7 @@ def dag_to_grid(dag: DagModel, dag_runs: Sequence[DagRun], session: Session) -> TaskInstance.dag_id == dag.dag_id, TaskInstance.run_id.in_([dag_run.run_id for dag_run in dag_runs]), ) - .group_by(TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, TaskInstance._try_number) + .group_by(TaskInstance.task_id, TaskInstance.run_id, TaskInstance.state, TaskInstance.try_number) .order_by(TaskInstance.task_id, TaskInstance.run_id) ) @@ -409,7 +409,7 @@ def set_overall_state(record): "queued_dttm": task_instance.queued_dttm, "start_date": task_instance.start_date, "end_date": task_instance.end_date, - "try_number": wwwutils.get_try_count(task_instance._try_number, task_instance.state), + "try_number": wwwutils.get_try_count(task_instance.try_number, task_instance.state), "note": task_instance.note, } for task_instance in grouped_tis[item.task_id] @@ -1687,7 +1687,7 @@ def log(self, session: Session = NEW_SESSION): num_logs = 0 if ti is not None: - num_logs = wwwutils.get_try_count(ti._try_number, ti.state) + num_logs = wwwutils.get_try_count(ti.try_number, ti.state) logs = [""] * num_logs root = request.args.get("root", "") return self.render_template( diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index 108ee9700383c..740ffa675cd68 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -1525,7 +1525,7 @@ def test_update_counters(self, dag_maker, session): # match what's in the in-memory ti_status.running map. This is the same # for skipped, failed and retry states. ti_status.running[ti.key] = ti # Task is queued and marked as running - ti._try_number += 1 # Try number is increased during ti.run() + ti.try_number += 1 # Try number is increased during ti.run() ti.set_state(State.SUCCESS, session) # Task finishes with success state job_runner._update_counters(ti_status=ti_status, session=session) # Update counters assert len(ti_status.running) == 0 @@ -1538,7 +1538,7 @@ def test_update_counters(self, dag_maker, session): # Test for success when DB try_number is off from in-memory expectations ti_status.running[ti.key] = ti - ti._try_number += 2 + ti.try_number += 2 ti.set_state(State.SUCCESS, session) job_runner._update_counters(ti_status=ti_status, session=session) assert len(ti_status.running) == 0 @@ -1551,7 +1551,7 @@ def test_update_counters(self, dag_maker, session): # Test for skipped ti_status.running[ti.key] = ti - ti._try_number += 1 + ti.try_number += 1 ti.set_state(State.SKIPPED, session) job_runner._update_counters(ti_status=ti_status, session=session) assert len(ti_status.running) == 0 @@ -1564,7 +1564,7 @@ def test_update_counters(self, dag_maker, session): # Test for failed ti_status.running[ti.key] = ti - ti._try_number += 1 + ti.try_number += 1 ti.set_state(State.FAILED, session) job_runner._update_counters(ti_status=ti_status, session=session) assert len(ti_status.running) == 0 @@ -1577,7 +1577,7 @@ def test_update_counters(self, dag_maker, session): # Test for retry ti_status.running[ti.key] = ti - ti._try_number += 1 + ti.try_number += 1 ti.set_state(State.UP_FOR_RETRY, session) job_runner._update_counters(ti_status=ti_status, session=session) assert len(ti_status.running) == 0 @@ -1595,9 +1595,6 @@ def test_update_counters(self, dag_maker, session): # and DB representation of the task try_number the _same_, which is unlike # the above cases. But this is okay because the in-memory key is used. ti_status.running[ti.key] = ti # Task queued and marked as running - # Note: Both the increase and decrease are kept here for context - ti._try_number += 1 # Try number is increased during ti.run() - ti._try_number -= 1 # Task is being rescheduled, decrement try_number ti.set_state(State.UP_FOR_RESCHEDULE, session) # Task finishes with reschedule state job_runner._update_counters(ti_status=ti_status, session=session) assert len(ti_status.running) == 0 @@ -1610,9 +1607,9 @@ def test_update_counters(self, dag_maker, session): # test for none ti.set_state(State.NONE, session) - # Setting ti._try_number = 0 brings us to ti.try_number==1 + # Setting ti.try_number = 0 brings us to ti.try_number==1 # so that the in-memory key access will work fine - ti._try_number = 0 + ti.try_number = 0 assert ti.try_number == 1 # see ti.try_number property in taskinstance module session.merge(ti) session.commit() @@ -2096,7 +2093,7 @@ def test_backfill_disable_retry(self, dag_maker, disable_retry, try_number, exce run_job(job=job, execute_callable=job_runner._execute) ti = dag_run.get_task_instance(task_id=task1.task_id) - assert ti._try_number == try_number + assert ti.try_number == try_number dag_run.refresh_from_db() diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index 11d833a21c502..e0639fc9241e4 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -29,7 +29,7 @@ from traceback import format_exception from typing import cast from unittest import mock -from unittest.mock import MagicMock, call, mock_open, patch +from unittest.mock import call, mock_open, patch from uuid import uuid4 import pendulum @@ -66,8 +66,6 @@ TaskInstance, TaskInstance as TI, TaskInstanceNote, - _get_private_try_number, - _get_try_number, _run_finished_callback, ) from airflow.models.taskmap import TaskMap @@ -668,13 +666,13 @@ def run_with_error(ti): # first run -- up for retry run_with_error(ti) assert ti.state == State.UP_FOR_RETRY - assert ti._try_number == 1 + assert ti.try_number == 1 assert ti.try_number == 2 # second run -- fail run_with_error(ti) assert ti.state == State.FAILED - assert ti._try_number == 2 + assert ti.try_number == 2 assert ti.try_number == 3 # Clear the TI state since you can't run a task with a FAILED state without @@ -684,14 +682,14 @@ def run_with_error(ti): # third run -- up for retry run_with_error(ti) assert ti.state == State.UP_FOR_RETRY - assert ti._try_number == 3 + assert ti.try_number == 3 assert ti.try_number == 4 # fourth run -- fail run_with_error(ti) ti.refresh_from_db() assert ti.state == State.FAILED - assert ti._try_number == 4 + assert ti.try_number == 4 assert ti.try_number == 5 assert RenderedTaskInstanceFields.get_templated_fields(ti) == expected_rendered_ti_fields @@ -783,7 +781,7 @@ def func(): ti = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances[0] ti.task = task - assert ti._try_number == 0 + assert ti.try_number == 0 assert ti.try_number == 1 def run_ti_and_assert( @@ -803,7 +801,7 @@ def run_ti_and_assert( raise ti.refresh_from_db() assert ti.state == expected_state - assert ti._try_number == expected_try_number + assert ti.try_number == expected_try_number assert ti.try_number == expected_try_number + 1 assert ti.start_date == expected_start_date assert ti.end_date == expected_end_date @@ -837,7 +835,7 @@ def run_ti_and_assert( dag.clear() ti.refresh_from_db() assert ti.state == State.NONE - assert ti._try_number == 1 + assert ti.try_number == 1 # Run again after clearing with reschedules and a retry. # The retry increments the try number, and for that try no reschedule is expected. @@ -880,7 +878,7 @@ def func(): ti = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances[0] ti.task = task - assert ti._try_number == 0 + assert ti.try_number == 0 assert ti.try_number == 1 def run_ti_and_assert( @@ -901,7 +899,7 @@ def run_ti_and_assert( raise ti.refresh_from_db() assert ti.state == expected_state - assert ti._try_number == expected_try_number + assert ti.try_number == expected_try_number assert ti.try_number == expected_try_number + 1 assert ti.start_date == expected_start_date assert ti.end_date == expected_end_date @@ -935,7 +933,7 @@ def run_ti_and_assert( dag.clear() ti.refresh_from_db() assert ti.state == State.NONE - assert ti._try_number == 1 + assert ti.try_number == 1 # Run again after clearing with reschedules and a retry. # The retry increments the try number, and for that try no reschedule is expected. @@ -978,7 +976,7 @@ def func(): ).expand(poke_interval=[0]) ti = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances[0] ti.task = task - assert ti._try_number == 0 + assert ti.try_number == 0 assert ti.try_number == 1 def run_ti_and_assert( @@ -999,7 +997,7 @@ def run_ti_and_assert( raise ti.refresh_from_db() assert ti.state == expected_state - assert ti._try_number == expected_try_number + assert ti.try_number == expected_try_number assert ti.try_number == expected_try_number + 1 assert ti.start_date == expected_start_date assert ti.end_date == expected_end_date @@ -1015,7 +1013,7 @@ def run_ti_and_assert( dag.clear() ti.refresh_from_db() assert ti.state == State.NONE - assert ti._try_number == 0 + assert ti.try_number == 0 # Check that reschedules for ti have also been cleared. assert not task_reschedules_for_ti(ti) @@ -1045,7 +1043,7 @@ def func(): ) ti = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances[0] ti.task = task - assert ti._try_number == 0 + assert ti.try_number == 0 assert ti.try_number == 1 def run_ti_and_assert( @@ -1065,7 +1063,7 @@ def run_ti_and_assert( raise ti.refresh_from_db() assert ti.state == expected_state - assert ti._try_number == expected_try_number + assert ti.try_number == expected_try_number assert ti.try_number == expected_try_number + 1 assert ti.start_date == expected_start_date assert ti.end_date == expected_end_date @@ -1081,7 +1079,7 @@ def run_ti_and_assert( dag.clear() ti.refresh_from_db() assert ti.state == State.NONE - assert ti._try_number == 0 + assert ti.try_number == 0 # Check that reschedules for ti have also been cleared. assert not task_reschedules_for_ti(ti) @@ -1803,12 +1801,12 @@ def test_check_and_change_state_before_execution(self, create_task_instance): serialized_dag = SerializedDagModel.get(ti.task.dag.dag_id).dag ti_from_deserialized_task = TI(task=serialized_dag.get_task(ti.task_id), run_id=ti.run_id) - assert ti_from_deserialized_task._try_number == 0 + assert ti_from_deserialized_task.try_number == 0 assert ti_from_deserialized_task.check_and_change_state_before_execution() # State should be running, and try_number column should be incremented assert ti_from_deserialized_task.external_executor_id == expected_external_executor_id assert ti_from_deserialized_task.state == State.RUNNING - assert ti_from_deserialized_task._try_number == 1 + assert ti_from_deserialized_task.try_number == 1 def test_check_and_change_state_before_execution_provided_id_overrides(self, create_task_instance): expected_external_executor_id = "banana" @@ -1822,14 +1820,14 @@ def test_check_and_change_state_before_execution_provided_id_overrides(self, cre serialized_dag = SerializedDagModel.get(ti.task.dag.dag_id).dag ti_from_deserialized_task = TI(task=serialized_dag.get_task(ti.task_id), run_id=ti.run_id) - assert ti_from_deserialized_task._try_number == 0 + assert ti_from_deserialized_task.try_number == 0 assert ti_from_deserialized_task.check_and_change_state_before_execution( external_executor_id=expected_external_executor_id ) # State should be running, and try_number column should be incremented assert ti_from_deserialized_task.external_executor_id == expected_external_executor_id assert ti_from_deserialized_task.state == State.RUNNING - assert ti_from_deserialized_task._try_number == 1 + assert ti_from_deserialized_task.try_number == 1 def test_check_and_change_state_before_execution_with_exec_id(self, create_task_instance): expected_external_executor_id = "minions" @@ -1840,14 +1838,14 @@ def test_check_and_change_state_before_execution_with_exec_id(self, create_task_ serialized_dag = SerializedDagModel.get(ti.task.dag.dag_id).dag ti_from_deserialized_task = TI(task=serialized_dag.get_task(ti.task_id), run_id=ti.run_id) - assert ti_from_deserialized_task._try_number == 0 + assert ti_from_deserialized_task.try_number == 0 assert ti_from_deserialized_task.check_and_change_state_before_execution( external_executor_id=expected_external_executor_id ) # State should be running, and try_number column should be incremented assert ti_from_deserialized_task.external_executor_id == expected_external_executor_id assert ti_from_deserialized_task.state == State.RUNNING - assert ti_from_deserialized_task._try_number == 1 + assert ti_from_deserialized_task.try_number == 1 def test_check_and_change_state_before_execution_dep_not_met(self, create_task_instance): ti = create_task_instance(dag_id="test_check_and_change_state_before_execution") @@ -3031,8 +3029,8 @@ def test_handle_failure_updates_queued_task_try_number(self, dag_maker): assert ti.try_number == 1 ti.handle_failure("test queued ti", test_mode=True) assert ti.state == State.UP_FOR_RETRY - # Assert that 'ti._try_number' is bumped from 0 to 1. This is the last/current try - assert ti._try_number == 1 + # Assert that 'ti.try_number' is bumped from 0 to 1. This is the last/current try + assert ti.try_number == 1 # Check 'ti.try_number' is bumped to 2. This is try_number for next run assert ti.try_number == 2 @@ -3060,8 +3058,8 @@ def test_handle_failure_no_task(self, Stats_incr, dag_maker): ti.handle_failure("test queued ti", test_mode=False) assert ti.state == State.UP_FOR_RETRY - # Assert that 'ti._try_number' is bumped from 0 to 1. This is the last/current try - assert ti._try_number == 1 + # Assert that 'ti.try_number' is bumped from 0 to 1. This is the last/current try + assert ti.try_number == 1 # Check 'ti.try_number' is bumped to 2. This is try_number for next run assert ti.try_number == 2 @@ -4693,14 +4691,3 @@ def test__refresh_from_db_should_not_increment_try_number(dag_maker, session): assert ti.try_number == 1 # stays 1 ti.refresh_from_db() assert ti.try_number == 1 # stays 1 - - -@pytest.mark.parametrize("state", list(TaskInstanceState)) -def test_get_private_try_number(state: str): - mock_ti = MagicMock() - mock_ti.state = state - private_try_number = 2 - mock_ti._try_number = private_try_number - mock_ti.try_number = _get_try_number(task_instance=mock_ti) - delattr(mock_ti, "_try_number") - assert _get_private_try_number(task_instance=mock_ti) == private_try_number diff --git a/tests/plugins/priority_weight_strategy.py b/tests/plugins/priority_weight_strategy.py index c56ae7364adb3..a20553635f844 100644 --- a/tests/plugins/priority_weight_strategy.py +++ b/tests/plugins/priority_weight_strategy.py @@ -44,7 +44,7 @@ class DecreasingPriorityStrategy(PriorityWeightStrategy): """A priority weight strategy that decreases the priority weight with each attempt.""" def get_weight(self, ti: TaskInstance): - return max(3 - ti._try_number + 1, 1) + return max(3 - ti.try_number + 1, 1) class TestPriorityWeightStrategyPlugin(AirflowPlugin): diff --git a/tests/test_utils/mock_executor.py b/tests/test_utils/mock_executor.py index ba555fbcd9cb3..17845b0861399 100644 --- a/tests/test_utils/mock_executor.py +++ b/tests/test_utils/mock_executor.py @@ -71,7 +71,7 @@ def sort_by(item): sorted_queue = sorted(self.queued_tasks.items(), key=sort_by) for key, (_, _, _, ti) in sorted_queue[:open_slots]: self.queued_tasks.pop(key) - ti._try_number += 1 + ti.try_number += 1 state = self.mock_task_results[key] ti.set_state(state, session=session) self.change_state(key, state) From 32bb153e5ea27b6cd99bdaf012f7b5764da9aeeb Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 08:34:41 -0700 Subject: [PATCH 06/53] increment try number when running dag test --- airflow/models/dag.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/models/dag.py b/airflow/models/dag.py index 05c7ca3d13af2..a6e5930a3e48c 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -2949,7 +2949,7 @@ def add_logger_if_needed(ti: TaskInstance): schedulable_tis, _ = dr.update_state(session=session) for s in schedulable_tis: s.state = TaskInstanceState.SCHEDULED - # s.try_number += 1 # todo: should we increase try number? + s.try_number += 1 session.commit() # triggerer may mark tasks scheduled so we read from DB all_tis = set(dr.get_task_instances(session=session)) From c4cb129db065778e4ec01cd6f0960a2926255257 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 13:12:30 -0700 Subject: [PATCH 07/53] fix tests --- airflow/models/taskinstance.py | 4 +-- airflow/sensors/base.py | 4 +-- tests/sensors/test_base.py | 54 +++++++++++++++++++++++++++------- 3 files changed, 47 insertions(+), 15 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index df654f96bb4b4..e4fa41d717866 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -281,8 +281,7 @@ def clear_task_instances( ti.refresh_from_task(task) if TYPE_CHECKING: assert ti.task - task_retries = task.retries - ti.max_tries = ti.try_number + task_retries - 1 + ti.max_tries = ti.try_number + task.retries else: # Ignore errors when updating max_tries if the DAG or # task are not found since database records could be @@ -2255,7 +2254,6 @@ def _check_and_change_state_before_execution( # If the task continues after being deferred (next_method is set), use the original start_date ti.start_date = ti.start_date if ti.next_method else timezone.utcnow() if ti.state == TaskInstanceState.UP_FOR_RESCHEDULE: - log.info("in state up_for_reschedule: task_id=%s", ti.task_id) tr_start_date = session.scalar( TR.stmt_for_task_instance(ti, descending=False).with_only_columns(TR.start_date).limit(1) ) diff --git a/airflow/sensors/base.py b/airflow/sensors/base.py index 8e13279babb2d..5f49d0096e44b 100644 --- a/airflow/sensors/base.py +++ b/airflow/sensors/base.py @@ -241,11 +241,11 @@ def execute(self, context: Context) -> Any: started_at: datetime.datetime | float if self.reschedule: - # If reschedule, use the start date of the first try (first try can be either the very - # first execution of the task, or the first execution after the task was cleared.) ti = context["ti"] max_tries: int = ti.max_tries or 0 retries: int = self.retries or 0 + # If reschedule, use the start date of the first try (first try can be either the very + # first execution of the task, or the first execution after the task was cleared.) first_try_number = max_tries - retries + 1 start_date = _orig_start_date( dag_id=ti.dag_id, diff --git a/tests/sensors/test_base.py b/tests/sensors/test_base.py index a41e6836838d5..dbf62883a50bb 100644 --- a/tests/sensors/test_base.py +++ b/tests/sensors/test_base.py @@ -297,7 +297,17 @@ def _get_tis(): date1 = timezone.utcnow() time_machine.move_to(date1, tick=False) + sensor_ti, dummy_ti = _get_tis() + assert dummy_ti.state == State.NONE + assert sensor_ti.state == State.NONE + + # ordinarily the scheduler does this + sensor_ti.state = State.SCHEDULED + sensor_ti.try_number += 1 # first TI run + session.commit() + self._run(sensor, session=session) + sensor_ti, dummy_ti = _get_tis() assert sensor_ti.state == State.UP_FOR_RESCHEDULE assert dummy_ti.state == State.NONE @@ -326,6 +336,9 @@ def _get_tis(): # first poke returns False and task is re-scheduled date1 = timezone.utcnow() time_machine.move_to(date1, tick=False) + sensor_ti, dummy_ti = _get_tis() + sensor_ti.try_number += 1 # second TI run + session.commit() self._run(sensor) sensor_ti, dummy_ti = _get_tis() assert sensor_ti.state == State.UP_FOR_RESCHEDULE @@ -359,6 +372,9 @@ def _get_tis(): # first poke returns False and task is re-scheduled date1 = timezone.utcnow() time_machine.move_to(date1, tick=False) + sensor_ti, dummy_ti = _get_tis() + sensor_ti.try_number += 1 # first TI run + session.commit() self._run(sensor) sensor_ti, dummy_ti = _get_tis() assert sensor_ti.state == State.UP_FOR_RESCHEDULE @@ -382,9 +398,12 @@ def _get_tis(): # Task is cleared sensor.clear() sensor_ti, dummy_ti = _get_tis() - assert sensor_ti.try_number == 2 + assert sensor_ti.try_number == 1 assert sensor_ti.max_tries == 2 + sensor_ti, dummy_ti = _get_tis() + sensor_ti.try_number += 1 # second TI run + session.commit() # third poke returns False and task is rescheduled again date3 = date1 + timedelta(seconds=sensor.poke_interval) * 2 + sensor.retry_delay time_machine.coordinates.shift(sensor.poke_interval + sensor.retry_delay.total_seconds()) @@ -687,9 +706,15 @@ def _get_sensor_ti(): tis = dr.get_task_instances(session=session) return next(x for x in tis if x.task_id == SENSOR_OP) + def _increment_try_number(): + sensor_ti = _get_sensor_ti() + sensor_ti.try_number += 1 + session.commit() + # first poke returns False and task is re-scheduled date1 = timezone.utcnow() time_machine.move_to(date1, tick=False) + _increment_try_number() # first TI run self._run(sensor) sensor_ti = _get_sensor_ti() assert sensor_ti.try_number == 1 @@ -701,12 +726,13 @@ def _get_sensor_ti(): with pytest.raises(RuntimeError): self._run(sensor) sensor_ti = _get_sensor_ti() - assert sensor_ti.try_number == 2 + assert sensor_ti.try_number == 1 assert sensor_ti.max_tries == 2 assert sensor_ti.state == State.UP_FOR_RETRY # third poke returns False and task is rescheduled again time_machine.coordinates.shift(sensor.retry_delay + timedelta(seconds=1)) + _increment_try_number() # second TI run self._run(sensor) sensor_ti = _get_sensor_ti() assert sensor_ti.try_number == 2 @@ -718,19 +744,22 @@ def _get_sensor_ti(): with pytest.raises(AirflowSensorTimeout): self._run(sensor) sensor_ti = _get_sensor_ti() - assert sensor_ti.try_number == 3 + assert sensor_ti.try_number == 2 assert sensor_ti.max_tries == 2 assert sensor_ti.state == State.FAILED # Clear the failed sensor sensor.clear() sensor_ti = _get_sensor_ti() - assert sensor_ti.try_number == 3 + # clearing does not change the try_number + assert sensor_ti.try_number == 2 + # but it does change the max_tries assert sensor_ti.max_tries == 4 assert sensor_ti.state is None time_machine.coordinates.shift(20) + _increment_try_number() # third TI run for _ in range(3): time_machine.coordinates.shift(sensor.poke_interval) self._run(sensor) @@ -744,7 +773,7 @@ def _get_sensor_ti(): with pytest.raises(AirflowSensorTimeout): self._run(sensor) sensor_ti = _get_sensor_ti() - assert sensor_ti.try_number == 4 + assert sensor_ti.try_number == 3 assert sensor_ti.max_tries == 4 assert sensor_ti.state == State.FAILED @@ -794,13 +823,16 @@ def _get_sensor_ti(): # first poke returns False and task is re-scheduled date1 = timezone.utcnow() time_machine.move_to(date1, tick=False) + sensor_ti = _get_sensor_ti() + sensor_ti.try_number += 1 # first TI run self._run(sensor) + sensor_ti = _get_sensor_ti() assert sensor_ti.try_number == 1 assert sensor_ti.max_tries == 2 assert sensor_ti.state == State.UP_FOR_RESCHEDULE - # second poke raises RuntimeError and task instance is re-scheduled again + # second poke raises reschedule exception and task instance is re-scheduled again time_machine.coordinates.shift(sensor.poke_interval) self._run(sensor) sensor_ti = _get_sensor_ti() @@ -821,19 +853,21 @@ def _get_sensor_ti(): with pytest.raises(AirflowSensorTimeout): self._run(sensor) sensor_ti = _get_sensor_ti() - assert sensor_ti.try_number == 2 + assert sensor_ti.try_number == 1 assert sensor_ti.max_tries == 2 assert sensor_ti.state == State.FAILED # Clear the failed sensor sensor.clear() sensor_ti = _get_sensor_ti() - assert sensor_ti.try_number == 2 + assert sensor_ti.try_number == 1 assert sensor_ti.max_tries == 3 assert sensor_ti.state == State.NONE time_machine.coordinates.shift(20) + sensor_ti.try_number += 1 # second TI run + session.commit() for _ in range(3): time_machine.coordinates.shift(sensor.poke_interval) self._run(sensor) @@ -847,7 +881,7 @@ def _get_sensor_ti(): with pytest.raises(AirflowSensorTimeout): self._run(sensor) sensor_ti = _get_sensor_ti() - assert sensor_ti.try_number == 3 + assert sensor_ti.try_number == 2 assert sensor_ti.max_tries == 3 assert sensor_ti.state == State.FAILED @@ -983,7 +1017,7 @@ class TestAsyncSensor: (False, AirflowException), ], ) - def test_fail_after_resuming_deffered_sensor(self, soft_fail, expected_exception): + def test_fail_after_resuming_deferred_sensor(self, soft_fail, expected_exception): async_sensor = DummyAsyncSensor(task_id="dummy_async_sensor", soft_fail=soft_fail) ti = TaskInstance(task=async_sensor) ti.next_method = "execute_complete" From e02aa1bbbd3472e79d25e20edb50ec9b5d9e769f Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 13:19:51 -0700 Subject: [PATCH 08/53] restore try_number increment when defer immed --- airflow/models/dagrun.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index ae7f555a270a4..2c3b028a4dbe4 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -1544,6 +1544,7 @@ def schedule_tis( and not ti.task.on_success_callback and not ti.task.outlets ): + ti.try_number += 1 ti.defer_task( defer=TaskDeferred(trigger=ti.task.start_trigger, method_name=ti.task.next_method), session=session, From 566f9cda67a92c568537607200fadd051ab688af Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 13:20:48 -0700 Subject: [PATCH 09/53] nothing --- airflow/models/dagrun.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 2c3b028a4dbe4..595ff2260c114 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -1077,6 +1077,7 @@ def _expand_mapped_task_if_needed(ti: TI) -> Iterable[TI] | None: if tis_filter is not None: fresh_tis = session.scalars(select(TI).where(tis_filter)).all() changed_tis = any(ti.state != old_states[ti.key] for ti in fresh_tis) + return ready_tis, changed_tis, expansion_happened def _are_premature_tis( From 8bad786be7db011a1949be02a291d2973a0459a7 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 13:26:48 -0700 Subject: [PATCH 10/53] reduce diff --- airflow/models/dagrun.py | 16 +++++++--------- 1 file changed, 7 insertions(+), 9 deletions(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 595ff2260c114..4dcc4d1289469 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -1560,14 +1560,13 @@ def schedule_tis( schedulable_ti_ids, max_tis_per_query or len(schedulable_ti_ids) ) for schedulable_ti_ids_chunk in schedulable_ti_ids_chunks: - schedulable_cond = ( - TI.dag_id == self.dag_id, - TI.run_id == self.run_id, - tuple_in_condition((TI.task_id, TI.map_index), schedulable_ti_ids_chunk), - ) - update_stmt = ( + count += session.execute( update(TI) - .where(*schedulable_cond) + .where( + TI.dag_id == self.dag_id, + TI.run_id == self.run_id, + tuple_in_condition((TI.task_id, TI.map_index), schedulable_ti_ids_chunk), + ) .values( state=TaskInstanceState.SCHEDULED, try_number=case( @@ -1579,8 +1578,7 @@ def schedule_tis( ), ) .execution_options(synchronize_session=False) - ) - count += session.execute(update_stmt).rowcount + ).rowcount # Tasks using EmptyOperator should not be executed, mark them as success if dummy_ti_ids: From 7491f3d4b723b469cb5c25603118ec056837d79d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 13:30:02 -0700 Subject: [PATCH 11/53] correct logic --- airflow/models/dagrun.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/models/dagrun.py b/airflow/models/dagrun.py index 4dcc4d1289469..84076ee70c53e 100644 --- a/airflow/models/dagrun.py +++ b/airflow/models/dagrun.py @@ -1545,7 +1545,8 @@ def schedule_tis( and not ti.task.on_success_callback and not ti.task.outlets ): - ti.try_number += 1 + if ti.state != TaskInstanceState.UP_FOR_RESCHEDULE: + ti.try_number += 1 ti.defer_task( defer=TaskDeferred(trigger=ti.task.start_trigger, method_name=ti.task.next_method), session=session, From d467cef865dc6d3be351315abbbecc2ca45a3be8 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 13:34:57 -0700 Subject: [PATCH 12/53] remove todos --- airflow/models/taskinstance.py | 2 -- airflow/serialization/serialized_objects.py | 4 ---- 2 files changed, 6 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index e4fa41d717866..ad22f297a2d61 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -972,8 +972,6 @@ def _record_task_map_for_downstreams( :meta private: """ - # todo: when sering task then be smarter than just setting dag to None. use sentinel - # to indicate it was attached and when re-setting .dag, skip the extra stuff if next(task.iter_mapped_dependants(), None) is None: # No mapped dependants, no need to validate. return # TODO: We don't push TaskMap for mapped task instances because it's not diff --git a/airflow/serialization/serialized_objects.py b/airflow/serialization/serialized_objects.py index 80b256e797589..669204376694e 100644 --- a/airflow/serialization/serialized_objects.py +++ b/airflow/serialization/serialized_objects.py @@ -551,12 +551,8 @@ def serialize( elif isinstance(var, Resources): return var.to_dict() elif isinstance(var, MappedOperator): - # todo: try to determine should_expand here - # todo: try to determine fail_stop here return cls._encode(SerializedBaseOperator.serialize_mapped_operator(var), type_=DAT.OP) elif isinstance(var, BaseOperator): - # todo: try to determine should_expand here - # todo: try to determine fail_stop here return cls._encode(SerializedBaseOperator.serialize_operator(var), type_=DAT.OP) elif isinstance(var, cls._datetime_types): return cls._encode(var.timestamp(), type_=DAT.DATETIME) From a0dcb55c09114d17bdc9502e38ec5a9eafabcfb1 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 14:30:50 -0700 Subject: [PATCH 13/53] fix tests --- tests/cli/commands/test_task_command.py | 31 +++++++++++++++++++------ 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/tests/cli/commands/test_task_command.py b/tests/cli/commands/test_task_command.py index e63f2c3f7e93a..1e7c29cca20db 100644 --- a/tests/cli/commands/test_task_command.py +++ b/tests/cli/commands/test_task_command.py @@ -46,7 +46,7 @@ from airflow.operators.bash import BashOperator from airflow.utils import timezone from airflow.utils.session import create_session -from airflow.utils.state import State +from airflow.utils.state import State, TaskInstanceState from airflow.utils.types import DagRunType from tests.test_utils.config import conf_vars from tests.test_utils.db import clear_db_pools, clear_db_runs @@ -651,6 +651,12 @@ def test_parentdag_downstream_clear(self): task_command.task_clear(args) +def _set_state_and_try_num(ti, session): + ti.state = TaskInstanceState.QUEUED + ti.try_number += 1 + session.commit() + + class TestLogsfromTaskRunCommand: def setup_method(self) -> None: self.dag_id = "test_logging_dag" @@ -668,7 +674,7 @@ def setup_method(self) -> None: dag = DagBag().get_dag(self.dag_id) data_interval = dag.timetable.infer_manual_data_interval(run_after=self.execution_date) - dag.create_dagrun( + self.dr = dag.create_dagrun( run_id=self.run_id, execution_date=self.execution_date, data_interval=data_interval, @@ -676,6 +682,9 @@ def setup_method(self) -> None: state=State.RUNNING, run_type=DagRunType.MANUAL, ) + self.tis = self.dr.get_task_instances() + assert len(self.tis) == 1 + self.ti = self.tis[0] root = self.root_logger = logging.getLogger() self.root_handlers = root.handlers.copy() @@ -757,7 +766,7 @@ def test_external_executor_id_present_for_process_run_task(self, mock_local_job) @pytest.mark.parametrize( "is_k8s, is_container_exec", [("true", "true"), ("true", ""), ("", "true"), ("", "")] ) - def test_logging_with_run_task_stdout_k8s_executor_pod(self, is_k8s, is_container_exec): + def test_logging_with_run_task_stdout_k8s_executor_pod(self, is_k8s, is_container_exec, session): """ When running task --local as k8s executor pod, all logging should make it to stdout. Otherwise, all logging after "running TI" is redirected to logs (and the actual log @@ -770,6 +779,9 @@ def test_logging_with_run_task_stdout_k8s_executor_pod(self, is_k8s, is_containe """ import subprocess + ti = self.dr.get_task_instances(session=session)[0] + _set_state_and_try_num(ti, session) # so that try_number is correct + with mock.patch.dict( "os.environ", AIRFLOW_IS_K8S_EXECUTOR_POD=is_k8s, @@ -807,7 +819,9 @@ def test_logging_with_run_task_stdout_k8s_executor_pod(self, is_k8s, is_containe assert len(lines) == 1 @pytest.mark.skipif(not hasattr(os, "fork"), reason="Forking not available") - def test_logging_with_run_task(self): + def test_logging_with_run_task(self, session): + ti = self.dr.get_task_instances(session=session)[0] + _set_state_and_try_num(ti, session) with conf_vars({("core", "dags_folder"): self.dag_path}): task_command.task_run(self.parser.parse_args(self.task_args)) @@ -852,7 +866,10 @@ def test_run_task_with_pool(self): session.commit() @mock.patch("airflow.task.task_runner.standard_task_runner.CAN_FORK", False) - def test_logging_with_run_task_subprocess(self): + def test_logging_with_run_task_subprocess(self, session): + ti = self.dr.get_task_instances(session=session)[0] + _set_state_and_try_num(ti, session) + with conf_vars({("core", "dags_folder"): self.dag_path}): task_command.task_run(self.parser.parse_args(self.task_args)) @@ -874,14 +891,14 @@ def test_logging_with_run_task_subprocess(self): f"task_id={self.task_id}, run_id={self.run_id}, execution_date=20170101T000000" in logs ) - def test_log_file_template_with_run_task(self): + def test_log_file_template_with_run_task(self, session): """Verify that the taskinstance has the right context for log_filename_template""" with conf_vars({("core", "dags_folder"): self.dag_path}): # increment the try_number of the task to be run with create_session() as session: ti = session.query(TaskInstance).filter_by(run_id=self.run_id).first() - ti.try_number = 1 + ti.try_number = 2 log_file_path = os.path.join(os.path.dirname(self.ti_log_file_path), "attempt=2.log") From 4f94368ad06c4e149b196cadc8ce8066c44df3c5 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 15:13:49 -0700 Subject: [PATCH 14/53] fix tests --- airflow/www/views.py | 2 +- tests/www/views/test_views_log.py | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/airflow/www/views.py b/airflow/www/views.py index cce04d2a02f6c..625995854df06 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -1788,7 +1788,7 @@ def task(self, session: Session = NEW_SESSION): warnings.simplefilter("ignore", RemovedInAirflow3Warning) all_ti_attrs = ( # fetching the value of _try_number to be shown under name try_number in UI - (name, getattr(ti, "_try_number" if name == "try_number" else name)) + (name, getattr(ti, name)) for name in dir(ti) if not name.startswith("_") and name not in ti_attrs_to_skip ) diff --git a/tests/www/views/test_views_log.py b/tests/www/views/test_views_log.py index 3d3248f1108b2..e32eb6654ba7a 100644 --- a/tests/www/views/test_views_log.py +++ b/tests/www/views/test_views_log.py @@ -185,7 +185,7 @@ def create_expected_log_file(log_path, tis): handler = FileTaskHandler(log_path) def create_expected_log_file(try_number): - ti.try_number = try_number - 1 + ti.try_number = 1 handler.set_context(ti) handler.emit(logging.makeLogRecord({"msg": "Log for testing."})) handler.flush() @@ -271,8 +271,9 @@ def test_get_logs_with_metadata_as_download_file(log_admin_client, create_expect in content_disposition ) assert 200 == response.status_code - assert "Log for testing." in response.data.decode("utf-8") - assert "localhost\n" in response.data.decode("utf-8") + content = response.data.decode("utf-8") + assert "Log for testing." in content + assert "localhost\n" in content DIFFERENT_LOG_FILENAME = "{{ ti.dag_id }}/{{ ti.run_id }}/{{ ti.task_id }}/{{ try_number }}.log" From 867a8bacab3a592bb7cbbab43aa92df02d2d1095 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 1 May 2024 17:54:15 -0700 Subject: [PATCH 15/53] fix tests --- tests/www/views/test_views_tasks.py | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/tests/www/views/test_views_tasks.py b/tests/www/views/test_views_tasks.py index 2b893c40f671d..233542ea1e3a2 100644 --- a/tests/www/views/test_views_tasks.py +++ b/tests/www/views/test_views_tasks.py @@ -1089,7 +1089,7 @@ def test_task_instances(admin_client): "task_id": "also_run_this", "trigger_id": None, "trigger_timeout": None, - "try_number": 1, + "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, @@ -1124,7 +1124,7 @@ def test_task_instances(admin_client): "task_id": "run_after_loop", "trigger_id": None, "trigger_timeout": None, - "try_number": 1, + "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, @@ -1159,7 +1159,7 @@ def test_task_instances(admin_client): "task_id": "run_this_last", "trigger_id": None, "trigger_timeout": None, - "try_number": 1, + "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, @@ -1194,7 +1194,7 @@ def test_task_instances(admin_client): "task_id": "runme_0", "trigger_id": None, "trigger_timeout": None, - "try_number": 1, + "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, @@ -1229,7 +1229,7 @@ def test_task_instances(admin_client): "task_id": "runme_1", "trigger_id": None, "trigger_timeout": None, - "try_number": 1, + "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, @@ -1264,7 +1264,7 @@ def test_task_instances(admin_client): "task_id": "runme_2", "trigger_id": None, "trigger_timeout": None, - "try_number": 1, + "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, @@ -1299,7 +1299,7 @@ def test_task_instances(admin_client): "task_id": "this_will_skip", "trigger_id": None, "trigger_timeout": None, - "try_number": 1, + "try_number": 0, "unixname": getuser(), "updated_at": DEFAULT_DATE.isoformat(), }, From 8a40c533e00f05668ea85dfa0b9749842ff70fab Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 May 2024 12:02:34 -0700 Subject: [PATCH 16/53] batch of test fixes --- tests/models/test_taskinstance.py | 73 +++++++++++++++++-------------- 1 file changed, 40 insertions(+), 33 deletions(-) diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index e0639fc9241e4..da42d74dd3874 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -782,7 +782,11 @@ def func(): ti = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances[0] ti.task = task assert ti.try_number == 0 - assert ti.try_number == 1 + + date1 = timezone.utcnow() + date2 = date1 + datetime.timedelta(minutes=1) + date3 = date2 + datetime.timedelta(minutes=1) + date4 = date3 + datetime.timedelta(minutes=1) def run_ti_and_assert( run_date, @@ -794,30 +798,29 @@ def run_ti_and_assert( expected_task_reschedule_count, ): with time_machine.travel(run_date, tick=False): + exc = None try: ti.run() - except AirflowException: + except AirflowException as e: + exc = e if not fail: raise + if exc and not fail: + raise RuntimeError("expected to fail") ti.refresh_from_db() assert ti.state == expected_state assert ti.try_number == expected_try_number - assert ti.try_number == expected_try_number + 1 assert ti.start_date == expected_start_date assert ti.end_date == expected_end_date assert ti.duration == expected_duration assert len(task_reschedules_for_ti(ti)) == expected_task_reschedule_count - date1 = timezone.utcnow() - date2 = date1 + datetime.timedelta(minutes=1) - date3 = date2 + datetime.timedelta(minutes=1) - date4 = date3 + datetime.timedelta(minutes=1) - # Run with multiple reschedules. # During reschedule the try number remains the same, but each reschedule is recorded. # The start date is expected to remain the initial date, hence the duration increases. - # When finished the try number is incremented and there is no reschedule expected - # for this try. + # When there's a new try (task run following something other than a reschedule), then + # the scheduler will increment the try_number. We do that inline here since + # we're not using the scheduler. done, fail = False, False run_ti_and_assert(date1, date1, date1, 0, State.UP_FOR_RESCHEDULE, 0, 1) @@ -829,29 +832,37 @@ def run_ti_and_assert( run_ti_and_assert(date3, date1, date3, 120, State.UP_FOR_RESCHEDULE, 0, 3) done, fail = True, False - run_ti_and_assert(date4, date1, date4, 180, State.SUCCESS, 1, 0) + run_ti_and_assert(date4, date1, date4, 180, State.SUCCESS, 0, 3) # Clear the task instance. dag.clear() ti.refresh_from_db() assert ti.state == State.NONE - assert ti.try_number == 1 + assert ti.try_number == 0 + + # We will run it again with reschedules and a retry. + + # We increment the try number because that's what the scheduler would do + with create_session() as session: + session.get(TaskInstance, ti.key.primary).try_number += 1 - # Run again after clearing with reschedules and a retry. - # The retry increments the try number, and for that try no reschedule is expected. # After the retry the start date is reset, hence the duration is also reset. done, fail = False, False run_ti_and_assert(date1, date1, date1, 0, State.UP_FOR_RESCHEDULE, 1, 1) done, fail = False, True - run_ti_and_assert(date2, date1, date2, 60, State.UP_FOR_RETRY, 2, 0) + run_ti_and_assert(date2, date1, date2, 60, State.UP_FOR_RETRY, 1, 1) + + # scheduler would create a new try here + with create_session() as session: + session.get(TaskInstance, ti.key.primary).try_number += 1 done, fail = False, False run_ti_and_assert(date3, date3, date3, 0, State.UP_FOR_RESCHEDULE, 2, 1) done, fail = True, False - run_ti_and_assert(date4, date3, date4, 60, State.SUCCESS, 3, 0) + run_ti_and_assert(date4, date3, date4, 60, State.SUCCESS, 2, 1) def test_mapped_reschedule_handling(self, dag_maker, task_reschedules_for_ti): """ @@ -1044,7 +1055,6 @@ def func(): ti = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances[0] ti.task = task assert ti.try_number == 0 - assert ti.try_number == 1 def run_ti_and_assert( run_date, @@ -1064,7 +1074,6 @@ def run_ti_and_assert( ti.refresh_from_db() assert ti.state == expected_state assert ti.try_number == expected_try_number - assert ti.try_number == expected_try_number + 1 assert ti.start_date == expected_start_date assert ti.end_date == expected_end_date assert ti.duration == expected_duration @@ -1806,7 +1815,7 @@ def test_check_and_change_state_before_execution(self, create_task_instance): # State should be running, and try_number column should be incremented assert ti_from_deserialized_task.external_executor_id == expected_external_executor_id assert ti_from_deserialized_task.state == State.RUNNING - assert ti_from_deserialized_task.try_number == 1 + assert ti_from_deserialized_task.try_number == 0 def test_check_and_change_state_before_execution_provided_id_overrides(self, create_task_instance): expected_external_executor_id = "banana" @@ -1827,7 +1836,7 @@ def test_check_and_change_state_before_execution_provided_id_overrides(self, cre # State should be running, and try_number column should be incremented assert ti_from_deserialized_task.external_executor_id == expected_external_executor_id assert ti_from_deserialized_task.state == State.RUNNING - assert ti_from_deserialized_task.try_number == 1 + assert ti_from_deserialized_task.try_number == 0 def test_check_and_change_state_before_execution_with_exec_id(self, create_task_instance): expected_external_executor_id = "minions" @@ -1842,10 +1851,10 @@ def test_check_and_change_state_before_execution_with_exec_id(self, create_task_ assert ti_from_deserialized_task.check_and_change_state_before_execution( external_executor_id=expected_external_executor_id ) - # State should be running, and try_number column should be incremented + # State should be running, and try_number column should be unchanged assert ti_from_deserialized_task.external_executor_id == expected_external_executor_id assert ti_from_deserialized_task.state == State.RUNNING - assert ti_from_deserialized_task.try_number == 1 + assert ti_from_deserialized_task.try_number == 0 def test_check_and_change_state_before_execution_dep_not_met(self, create_task_instance): ti = create_task_instance(dag_id="test_check_and_change_state_before_execution") @@ -1893,12 +1902,14 @@ def test_try_number(self, create_task_instance): Test the try_number accessor behaves in various running states """ ti = create_task_instance(dag_id="test_check_and_change_state_before_execution") - assert 1 == ti.try_number + # TI starts at 0. It's only incremented by the scheduler. + assert ti.try_number == 0 ti.try_number = 2 + assert ti.try_number == 2 ti.state = State.RUNNING - assert 2 == ti.try_number + assert ti.try_number == 2 # unaffected by state ti.state = State.SUCCESS - assert 3 == ti.try_number + assert ti.try_number == 2 # unaffected by state def test_get_num_running_task_instances(self, create_task_instance): session = settings.Session() @@ -2045,7 +2056,7 @@ def test_email_alert(self, mock_send_email, dag_maker, use_native_obj): assert email == "to" assert "test_email_alert" in title assert "test_email_alert" in body - assert "Try 1" in body + assert "Try 0" in body @conf_vars( { @@ -2128,7 +2139,7 @@ def test_email_alert(x): (email, title, body), _ = mock_send_email.call_args assert email == "to" assert title == f"Airflow alert: " - assert body.startswith("Try 1") + assert body.startswith("Try 0") # try number only incremented by the scheduler assert "test_email_alert" in body tf = ( @@ -3029,10 +3040,8 @@ def test_handle_failure_updates_queued_task_try_number(self, dag_maker): assert ti.try_number == 1 ti.handle_failure("test queued ti", test_mode=True) assert ti.state == State.UP_FOR_RETRY - # Assert that 'ti.try_number' is bumped from 0 to 1. This is the last/current try + # try_number remains at 1 assert ti.try_number == 1 - # Check 'ti.try_number' is bumped to 2. This is try_number for next run - assert ti.try_number == 2 @patch.object(Stats, "incr") def test_handle_failure_no_task(self, Stats_incr, dag_maker): @@ -3058,10 +3067,8 @@ def test_handle_failure_no_task(self, Stats_incr, dag_maker): ti.handle_failure("test queued ti", test_mode=False) assert ti.state == State.UP_FOR_RETRY - # Assert that 'ti.try_number' is bumped from 0 to 1. This is the last/current try + # try_number remains at 1 assert ti.try_number == 1 - # Check 'ti.try_number' is bumped to 2. This is try_number for next run - assert ti.try_number == 2 Stats_incr.assert_any_call("ti_failures", tags=expected_stats_tags) Stats_incr.assert_any_call("operator_failures_EmptyOperator", tags=expected_stats_tags) From c5184b522f864a336c927b4702d1148a73b61a08 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 May 2024 15:05:20 -0700 Subject: [PATCH 17/53] fix backoff logic --- airflow/models/taskinstance.py | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index ad22f297a2d61..90f4c3b94e474 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -2094,7 +2094,9 @@ def next_retry_datetime(self): # If the min_backoff calculation is below 1, it will be converted to 0 via int. Thus, # we must round up prior to converting to an int, otherwise a divide by zero error # will occur in the modded_hash calculation. - min_backoff = math.ceil(delay.total_seconds() * (2 ** (self.try_number - 2))) + # this probably gives unexpected results if a task instance has previously been cleared, + # because try_number can increase without bound + min_backoff = math.ceil(delay.total_seconds() * (2 ** (self.try_number - 1))) # In the case when delay.total_seconds() is 0, min_backoff will not be rounded up to 1. # To address this, we impose a lower bound of 1 on min_backoff. This effectively makes From 1e1b94bec9fa5df1911edf697bc6efa25511a226 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 May 2024 15:06:03 -0700 Subject: [PATCH 18/53] comment --- airflow/models/taskinstancekey.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/models/taskinstancekey.py b/airflow/models/taskinstancekey.py index 50906e47b0a31..b705ecbe8785d 100644 --- a/airflow/models/taskinstancekey.py +++ b/airflow/models/taskinstancekey.py @@ -37,6 +37,7 @@ def primary(self) -> tuple[str, str, str, int]: @property def reduced(self) -> TaskInstanceKey: """Remake the key by subtracting 1 from try number to match in memory information.""" + # todo (dstandish): remove this property return TaskInstanceKey( self.dag_id, self.task_id, self.run_id, max(1, self.try_number - 1), self.map_index ) From 4938f409f3c441e7598a1a8626e736fd3c89af9c Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Thu, 2 May 2024 15:07:36 -0700 Subject: [PATCH 19/53] next batch of fixes --- tests/models/test_cleartasks.py | 59 ++++++++++++++++++------------- tests/models/test_dag.py | 36 +++++++++++++------ tests/models/test_taskinstance.py | 33 ++++++++++++----- 3 files changed, 85 insertions(+), 43 deletions(-) diff --git a/tests/models/test_cleartasks.py b/tests/models/test_cleartasks.py index bce9dc4668a1d..ec2059c84e87b 100644 --- a/tests/models/test_cleartasks.py +++ b/tests/models/test_cleartasks.py @@ -25,7 +25,7 @@ from airflow import settings from airflow.models.dag import DAG from airflow.models.serialized_dag import SerializedDagModel -from airflow.models.taskinstance import TaskInstance as TI, clear_task_instances +from airflow.models.taskinstance import TaskInstance, TaskInstance as TI, clear_task_instances from airflow.models.taskreschedule import TaskReschedule from airflow.operators.empty import EmptyOperator from airflow.sensors.python import PythonSensor @@ -500,37 +500,35 @@ def test_dag_clear(self, dag_maker): ti0, ti1 = sorted(dr.task_instances, key=lambda ti: ti.task_id) ti0.refresh_from_task(task0) ti1.refresh_from_task(task1) - + session.get(TaskInstance, ti0.key.primary).try_number += 1 + session.commit() # Next try to run will be try 1 assert ti0.try_number == 1 ti0.run() - assert ti0.try_number == 2 + assert ti0.try_number == 1 dag.clear() ti0.refresh_from_db() - assert ti0.try_number == 2 + assert ti0.try_number == 1 assert ti0.state == State.NONE assert ti0.max_tries == 1 assert ti1.max_tries == 2 - ti1.try_number = 1 - session.merge(ti1) + session.get(TaskInstance, ti1.key.primary).try_number += 1 session.commit() - - # Next try will be 2 ti1.run() - assert ti1.try_number == 3 + assert ti1.try_number == 1 assert ti1.max_tries == 2 dag.clear() ti0.refresh_from_db() ti1.refresh_from_db() - # after clear dag, ti2 should show attempt 3 of 5 - assert ti1.max_tries == 4 - assert ti1.try_number == 3 - # after clear dag, ti1 should show attempt 2 of 2 - assert ti0.try_number == 2 + # after clear dag, we have 2 remaining tries + assert ti1.max_tries == 3 + assert ti1.try_number == 1 + # after clear dag, ti0 has no remaining tries + assert ti0.try_number == 1 assert ti0.max_tries == 1 def test_dags_clear(self): @@ -559,9 +557,11 @@ def test_dags_clear(self): # test clear all dags for i in range(num_of_dags): + session.get(TaskInstance, tis[i].key.primary).try_number += 1 + session.commit() tis[i].run() assert tis[i].state == State.SUCCESS - assert tis[i].try_number == 2 + assert tis[i].try_number == 1 assert tis[i].max_tries == 0 DAG.clear_dags(dags) @@ -569,14 +569,16 @@ def test_dags_clear(self): for i in range(num_of_dags): tis[i].refresh_from_db() assert tis[i].state == State.NONE - assert tis[i].try_number == 2 + assert tis[i].try_number == 1 assert tis[i].max_tries == 1 # test dry_run for i in range(num_of_dags): + session.get(TaskInstance, tis[i].key.primary).try_number += 1 + session.commit() tis[i].run() assert tis[i].state == State.SUCCESS - assert tis[i].try_number == 3 + assert tis[i].try_number == 2 assert tis[i].max_tries == 1 DAG.clear_dags(dags, dry_run=True) @@ -584,7 +586,7 @@ def test_dags_clear(self): for i in range(num_of_dags): tis[i].refresh_from_db() assert tis[i].state == State.SUCCESS - assert tis[i].try_number == 3 + assert tis[i].try_number == 2 assert tis[i].max_tries == 1 # test only_failed @@ -599,14 +601,14 @@ def test_dags_clear(self): ti.refresh_from_db() if ti is failed_dag: assert ti.state == State.NONE - assert ti.try_number == 3 + assert ti.try_number == 2 assert ti.max_tries == 2 else: assert ti.state == State.SUCCESS - assert ti.try_number == 3 + assert ti.try_number == 2 assert ti.max_tries == 1 - def test_operator_clear(self, dag_maker): + def test_operator_clear(self, dag_maker, session): with dag_maker( "test_operator_clear", start_date=DEFAULT_DATE, @@ -625,18 +627,27 @@ def test_operator_clear(self, dag_maker): ti1.task = op1 ti2.task = op2 + session.get(TaskInstance, ti2.key.primary).try_number += 1 + session.commit() ti2.run() # Dependency not met assert ti2.try_number == 1 assert ti2.max_tries == 1 op2.clear(upstream=True) + # max tries will be set to retries + curr try number == 1 + 1 == 2 + assert session.get(TaskInstance, ti2.key.primary).max_tries == 2 + + session.get(TaskInstance, ti1.key.primary).try_number += 1 + session.commit() ti1.run() + assert ti1.try_number == 1 + + session.get(TaskInstance, ti2.key.primary).try_number += 1 + session.commit() ti2.run(ignore_ti_state=True) - assert ti1.try_number == 2 # max_tries is 0 because there is no task instance in db for ti1 # so clear won't change the max_tries. assert ti1.max_tries == 0 assert ti2.try_number == 2 - # try_number (0) + retries(1) - assert ti2.max_tries == 1 + assert ti2.max_tries == 2 # max tries has not changed since it was updated when op2.clear called diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index aa6be108a1019..5f290b570d4a7 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -3047,7 +3047,8 @@ def teardown_method(self): def test_fileloc(self): @dag_decorator(default_args=self.DEFAULT_ARGS) - def noop_pipeline(): ... + def noop_pipeline(): + ... dag = noop_pipeline() assert isinstance(dag, DAG) @@ -3058,7 +3059,8 @@ def test_set_dag_id(self): """Test that checks you can set dag_id from decorator.""" @dag_decorator("test", default_args=self.DEFAULT_ARGS) - def noop_pipeline(): ... + def noop_pipeline(): + ... dag = noop_pipeline() assert isinstance(dag, DAG) @@ -3068,7 +3070,8 @@ def test_default_dag_id(self): """Test that @dag uses function name as default dag id.""" @dag_decorator(default_args=self.DEFAULT_ARGS) - def noop_pipeline(): ... + def noop_pipeline(): + ... dag = noop_pipeline() assert isinstance(dag, DAG) @@ -3124,7 +3127,8 @@ def test_resolve_documentation_template_file_rendered(self, tmp_path): @dag_decorator( "test-dag", start_date=DEFAULT_DATE, template_searchpath=os.fspath(path.parent), doc_md=path.name ) - def markdown_docs(): ... + def markdown_docs(): + ... dag = markdown_docs() assert isinstance(dag, DAG) @@ -3220,7 +3224,8 @@ def return_num(num): def test_warning_location(self): # NOTE: This only works as long as there is some warning we can emit from `DAG()` @dag_decorator(schedule_interval=None) - def mydag(): ... + def mydag(): + ... with pytest.warns(RemovedInAirflow3Warning) as warnings: line = sys._getframe().f_lineno + 1 @@ -3257,7 +3262,13 @@ def test_dag_timetable_change_after_init(timetable): assert not dag._check_schedule_interval_matches_timetable() -@pytest.mark.parametrize("run_id, execution_date", [(None, datetime_tz(2020, 1, 1)), ("test-run-id", None)]) +@pytest.mark.parametrize( + "run_id, execution_date", + [ + (None, datetime_tz(2020, 1, 1)), + ("test-run-id", None), + ], +) def test_set_task_instance_state(run_id, execution_date, session, dag_maker): """Test that set_task_instance_state updates the TaskInstance state and clear downstream failed""" @@ -3321,7 +3332,9 @@ def get_ti_from_db(task): # dagrun should be set to QUEUED assert dagrun.get_state() == State.QUEUED - assert {t.key for t in altered} == {("test_set_task_instance_state", "task_1", dagrun.run_id, 1, -1)} + assert {tuple(t.key) for t in altered} == { + ("test_set_task_instance_state", "task_1", dagrun.run_id, 0, -1) + } def test_set_task_instance_state_mapped(dag_maker, session): @@ -4098,13 +4111,16 @@ def test_clearing_setup_clears_teardown_taskflow(self, upstream, downstream, exp with DAG(dag_id="test_dag", start_date=pendulum.now()) as dag: @setup - def my_setup(): ... + def my_setup(): + ... @task_decorator - def my_work(): ... + def my_work(): + ... @teardown - def my_teardown(): ... + def my_teardown(): + ... s1 = my_setup() w1 = my_work() diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index da42d74dd3874..688abfdf1064d 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -621,21 +621,31 @@ def run_with_error(ti): ti = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances[0] ti.task = task - assert ti.try_number == 1 + with create_session() as session: + session.get(TaskInstance, ti.key.primary).try_number += 1 + # first run -- up for retry run_with_error(ti) assert ti.state == State.UP_FOR_RETRY - assert ti.try_number == 2 + assert ti.try_number == 1 + + with create_session() as session: + session.get(TaskInstance, ti.key.primary).try_number += 1 # second run -- still up for retry because retry_delay hasn't expired time_machine.coordinates.shift(3) run_with_error(ti) assert ti.state == State.UP_FOR_RETRY + assert ti.try_number == 2 + + with create_session() as session: + session.get(TaskInstance, ti.key.primary).try_number += 1 # third run -- failed time_machine.coordinates.shift(datetime.datetime.resolution) run_with_error(ti) assert ti.state == State.FAILED + assert ti.try_number == 3 def test_retry_handling(self, dag_maker): """ @@ -890,7 +900,6 @@ def func(): ti.task = task assert ti.try_number == 0 - assert ti.try_number == 1 def run_ti_and_assert( run_date, @@ -911,7 +920,6 @@ def run_ti_and_assert( ti.refresh_from_db() assert ti.state == expected_state assert ti.try_number == expected_try_number - assert ti.try_number == expected_try_number + 1 assert ti.start_date == expected_start_date assert ti.end_date == expected_end_date assert ti.duration == expected_duration @@ -938,29 +946,36 @@ def run_ti_and_assert( run_ti_and_assert(date3, date1, date3, 120, State.UP_FOR_RESCHEDULE, 0, 3) done, fail = True, False - run_ti_and_assert(date4, date1, date4, 180, State.SUCCESS, 1, 0) + run_ti_and_assert(date4, date1, date4, 180, State.SUCCESS, 0, 3) # Clear the task instance. dag.clear() ti.refresh_from_db() assert ti.state == State.NONE - assert ti.try_number == 1 + assert ti.try_number == 0 # Run again after clearing with reschedules and a retry. - # The retry increments the try number, and for that try no reschedule is expected. + + # We increment the try number because that's what the scheduler would do + with create_session() as session: + session.get(TaskInstance, ti.key.primary).try_number += 1 + # After the retry the start date is reset, hence the duration is also reset. done, fail = False, False run_ti_and_assert(date1, date1, date1, 0, State.UP_FOR_RESCHEDULE, 1, 1) done, fail = False, True - run_ti_and_assert(date2, date1, date2, 60, State.UP_FOR_RETRY, 2, 0) + run_ti_and_assert(date2, date1, date2, 60, State.UP_FOR_RETRY, 1, 1) + + with create_session() as session: + session.get(TaskInstance, ti.key.primary).try_number += 1 done, fail = False, False run_ti_and_assert(date3, date3, date3, 0, State.UP_FOR_RESCHEDULE, 2, 1) done, fail = True, False - run_ti_and_assert(date4, date3, date4, 60, State.SUCCESS, 3, 0) + run_ti_and_assert(date4, date3, date4, 60, State.SUCCESS, 2, 1) @pytest.mark.usefixtures("test_pool") def test_mapped_task_reschedule_handling_clear_reschedules(self, dag_maker, task_reschedules_for_ti): From f740a148ef3290a504e1abadd2313f4f00417796 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 May 2024 09:05:24 -0700 Subject: [PATCH 20/53] fix sentry --- tests/core/test_sentry.py | 4 ++-- tests/models/test_dag.py | 24 ++++++++---------------- 2 files changed, 10 insertions(+), 18 deletions(-) diff --git a/tests/core/test_sentry.py b/tests/core/test_sentry.py index 10c88675ea265..de78912041d55 100644 --- a/tests/core/test_sentry.py +++ b/tests/core/test_sentry.py @@ -38,7 +38,7 @@ DAG_ID = "test_dag" TASK_ID = "test_task" OPERATOR = "PythonOperator" -TRY_NUMBER = 1 +TRY_NUMBER = 0 STATE = State.SUCCESS TEST_SCOPE = { "dag_id": DAG_ID, @@ -149,7 +149,7 @@ def test_add_tagging(self, sentry, task_instance): sentry.add_tagging(task_instance=task_instance) with configure_scope() as scope: for key, value in scope._tags.items(): - assert TEST_SCOPE[key] == value + assert value == TEST_SCOPE[key] @pytest.mark.db_test @time_machine.travel(CRUMB_DATE) diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 5f290b570d4a7..64acad83deab2 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -3047,8 +3047,7 @@ def teardown_method(self): def test_fileloc(self): @dag_decorator(default_args=self.DEFAULT_ARGS) - def noop_pipeline(): - ... + def noop_pipeline(): ... dag = noop_pipeline() assert isinstance(dag, DAG) @@ -3059,8 +3058,7 @@ def test_set_dag_id(self): """Test that checks you can set dag_id from decorator.""" @dag_decorator("test", default_args=self.DEFAULT_ARGS) - def noop_pipeline(): - ... + def noop_pipeline(): ... dag = noop_pipeline() assert isinstance(dag, DAG) @@ -3070,8 +3068,7 @@ def test_default_dag_id(self): """Test that @dag uses function name as default dag id.""" @dag_decorator(default_args=self.DEFAULT_ARGS) - def noop_pipeline(): - ... + def noop_pipeline(): ... dag = noop_pipeline() assert isinstance(dag, DAG) @@ -3127,8 +3124,7 @@ def test_resolve_documentation_template_file_rendered(self, tmp_path): @dag_decorator( "test-dag", start_date=DEFAULT_DATE, template_searchpath=os.fspath(path.parent), doc_md=path.name ) - def markdown_docs(): - ... + def markdown_docs(): ... dag = markdown_docs() assert isinstance(dag, DAG) @@ -3224,8 +3220,7 @@ def return_num(num): def test_warning_location(self): # NOTE: This only works as long as there is some warning we can emit from `DAG()` @dag_decorator(schedule_interval=None) - def mydag(): - ... + def mydag(): ... with pytest.warns(RemovedInAirflow3Warning) as warnings: line = sys._getframe().f_lineno + 1 @@ -4111,16 +4106,13 @@ def test_clearing_setup_clears_teardown_taskflow(self, upstream, downstream, exp with DAG(dag_id="test_dag", start_date=pendulum.now()) as dag: @setup - def my_setup(): - ... + def my_setup(): ... @task_decorator - def my_work(): - ... + def my_work(): ... @teardown - def my_teardown(): - ... + def my_teardown(): ... s1 = my_setup() w1 = my_work() From da4ba4d9c8b8d9b25994a561313c4a7e0f5799c0 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Fri, 3 May 2024 13:01:22 -0700 Subject: [PATCH 21/53] --wip-- [skip ci] --- airflow/jobs/backfill_job_runner.py | 34 ++++++++++++++++++++++------- airflow/jobs/job.py | 3 +++ tests/jobs/test_backfill_job.py | 34 ++++++++++++++++------------- 3 files changed, 48 insertions(+), 23 deletions(-) diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index ace6a001311a1..15292b16b7601 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -22,7 +22,7 @@ import attr import pendulum -from sqlalchemy import select, tuple_, update +from sqlalchemy import case, or_, select, tuple_, update from sqlalchemy.exc import OperationalError from sqlalchemy.orm.session import make_transient from tabulate import tabulate @@ -245,7 +245,16 @@ def _update_counters(self, ti_status: _DagRunTaskStatus, session: Session) -> No session.execute( update(TI) .where(filter_for_tis) - .values(state=TaskInstanceState.SCHEDULED) + .values( + state=TaskInstanceState.SCHEDULED, + try_number=case( + ( + or_(TI.state.is_(None), TI.state != TaskInstanceState.UP_FOR_RESCHEDULE), + TI.try_number + 1, + ), + else_=TI.try_number, + ), + ) .execution_options(synchronize_session=False) ) session.flush() @@ -407,6 +416,7 @@ def _task_instances_for_dag_run( :param dag_run: the dag run to get the tasks from :param session: the database session object """ + self.log.info("_task_instances_for_dag_run") tasks_to_run = {} if dag_run is None: @@ -422,9 +432,13 @@ def _task_instances_for_dag_run( dag_run.dag = dag info = dag_run.task_instance_scheduling_decisions(session=session) schedulable_tis = info.schedulable_tis + self.log.info("schedulable_tis: %s", schedulable_tis) try: for ti in dag_run.get_task_instances(session=session): + session.merge(ti) if ti in schedulable_tis: + if not ti.state == TaskInstanceState.UP_FOR_RESCHEDULE: + ti.try_number += 1 ti.set_state(TaskInstanceState.SCHEDULED) if ti.state != TaskInstanceState.REMOVED: tasks_to_run[ti.key] = ti @@ -489,20 +503,20 @@ def _per_task_process(key, ti: TaskInstance, session): task = self.dag.get_task(ti.task_id, include_subdags=True) ti.task = task - self.log.debug("Task instance to run %s state %s", ti, ti.state) + self.log.info("Task instance to run %s state %s", ti, ti.state) # The task was already marked successful or skipped by a # different Job. Don't rerun it. if ti.state == TaskInstanceState.SUCCESS: ti_status.succeeded.add(key) - self.log.debug("Task instance %s succeeded. Don't rerun.", ti) + self.log.info("Task instance %s succeeded. Don't rerun.", ti) ti_status.to_run.pop(key) if key in ti_status.running: ti_status.running.pop(key) return elif ti.state == TaskInstanceState.SKIPPED: ti_status.skipped.add(key) - self.log.debug("Task instance %s skipped. Don't rerun.", ti) + self.log.info("Task instance %s skipped. Don't rerun.", ti) ti_status.to_run.pop(key) if key in ti_status.running: ti_status.running.pop(key) @@ -511,7 +525,7 @@ def _per_task_process(key, ti: TaskInstance, session): if self.rerun_failed_tasks: # Rerun failed tasks or upstreamed failed tasks if ti.state in (TaskInstanceState.FAILED, TaskInstanceState.UPSTREAM_FAILED): - self.log.error("Task instance %s with state %s", ti, ti.state) + self.log.info("Task instance %s with state %s", ti, ti.state) if key in ti_status.running: ti_status.running.pop(key) # Reset the failed task in backfill to scheduled state @@ -695,7 +709,9 @@ def _per_task_process(key, ti: TaskInstance, session): self.log.debug(e) perform_heartbeat( - job=self.job, heartbeat_callback=self.heartbeat_callback, only_if_necessary=True + job=self.job, + heartbeat_callback=self.heartbeat_callback, + only_if_necessary=True, ) # execute the tasks in the queue executor.heartbeat() @@ -835,14 +851,16 @@ def _execute_dagruns( :param start_date: backfill start date :param session: the current session object """ + self.log.info("_execute_dagruns") for dagrun_info in dagrun_infos: for dag in self._get_dag_with_subdags(): dag_run = self._get_dag_run(dagrun_info, dag, session=session) if dag_run is not None: tis_map = self._task_instances_for_dag_run(dag, dag_run, session=session) + self.log.info("tis_map=%s", tis_map) ti_status.active_runs.add(dag_run) ti_status.to_run.update(tis_map or {}) - + self.log.info("") processed_dag_run_dates = self._process_backfill_task_instances( ti_status=ti_status, executor=executor, diff --git a/airflow/jobs/job.py b/airflow/jobs/job.py index 4273f1d3345b5..dff472d1fee72 100644 --- a/airflow/jobs/job.py +++ b/airflow/jobs/job.py @@ -202,7 +202,9 @@ def heartbeat( try: # This will cause it to load from the db + self.log.info("fetching from DB") self._merge_from(Job._fetch_from_db(self, session)) + self.log.info("finished fetching from DB") previous_heartbeat = self.latest_heartbeat if self.state == JobState.RESTARTING: @@ -215,6 +217,7 @@ def heartbeat( self.heartrate - (timezone.utcnow() - self.latest_heartbeat).total_seconds() ) sleep_for = max(0, seconds_remaining) + print(f"sleeping for {sleep_for}") sleep(sleep_for) job = Job._update_heartbeat(job=self, session=session) diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index 740ffa675cd68..56deacbf55ab1 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -185,7 +185,7 @@ def test_trigger_controller_dag(self, session): assert task_instances_list - @pytest.mark.backend("postgres", "mysql") + # @pytest.mark.backend("postgres", "mysql") def test_backfill_multi_dates(self): dag = self.dagbag.get_dag("miscellaneous_test_dag") @@ -217,11 +217,15 @@ def test_backfill_multi_dates(self): ("run_this_last", DEFAULT_DATE), ("run_this_last", end_date), ] - assert [ - ((dag.dag_id, task_id, f"backfill__{when.isoformat()}", 1, -1), (State.SUCCESS, None)) + actual = [(tuple(x), y) for x, y in executor.sorted_tasks] + expected = [ + ( + (dag.dag_id, task_id, f"backfill__{when.isoformat()}", 0, -1), + (State.SUCCESS, None), + ) for (task_id, when) in expected_execution_order - ] == executor.sorted_tasks - + ] + assert actual == expected session = settings.Session() drs = session.query(DagRun).filter(DagRun.dag_id == dag.dag_id).order_by(DagRun.execution_date).all() @@ -879,12 +883,12 @@ def test_backfill_retry_intermittent_failed_task(self, dag_maker): dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1)] = ( - State.UP_FOR_RETRY - ) - executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2)] = ( - State.UP_FOR_RETRY - ) + executor.mock_task_results[ + TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1) + ] = State.UP_FOR_RETRY + executor.mock_task_results[ + TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2) + ] = State.UP_FOR_RETRY job = Job(executor=executor) job_runner = BackfillJobRunner( job=job, @@ -907,10 +911,10 @@ def test_backfill_retry_always_failed_task(self, dag_maker): dr = dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=1)] = ( - State.UP_FOR_RETRY - ) - executor.mock_task_fail(dag.dag_id, task1.task_id, dr.run_id, try_number=2) + executor.mock_task_results[ + TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0) + ] = State.UP_FOR_RETRY + executor.mock_task_fail(dag.dag_id, task1.task_id, dr.run_id, try_number=1) job = Job(executor=executor) job_runner = BackfillJobRunner( job=job, From 4f6e2d0d809d6aba7a009844fe27fd59cc2324e7 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 08:30:39 -0700 Subject: [PATCH 22/53] handle try_number in backfill --- airflow/jobs/backfill_job_runner.py | 19 +++++++++++++++++-- tests/jobs/test_backfill_job.py | 18 +++++++++--------- 2 files changed, 26 insertions(+), 11 deletions(-) diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index 15292b16b7601..58fa38d436edf 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -442,6 +442,13 @@ def _task_instances_for_dag_run( ti.set_state(TaskInstanceState.SCHEDULED) if ti.state != TaskInstanceState.REMOVED: tasks_to_run[ti.key] = ti + if ti.state == TaskInstanceState.UP_FOR_RETRY and ti not in schedulable_tis: + self.log.warning( + "unexpected. task_id=%s has state=%s but is not in schedulable_tis=%s", + ti.task_id, + ti.state, + schedulable_tis, + ) session.commit() except Exception: session.rollback() @@ -529,6 +536,7 @@ def _per_task_process(key, ti: TaskInstance, session): if key in ti_status.running: ti_status.running.pop(key) # Reset the failed task in backfill to scheduled state + ti.try_number += 1 ti.set_state(TaskInstanceState.SCHEDULED, session=session) if ti.dag_run not in ti_status.active_runs: ti_status.active_runs.add(ti.dag_run) @@ -566,6 +574,14 @@ def _per_task_process(key, ti: TaskInstance, session): else: self.log.debug("Sending %s to executor", ti) # Skip scheduled state, we are executing immediately + if ti.state == TaskInstanceState.UP_FOR_RETRY: + # i am not sure why this is necessary. + # seemingly a quirk of backfill runner. + # it should be handled elsewhere i think. + # but i am not going to look too closely since we need + # to nuke the current backfill approach anyway. + ti.try_number += 1 + executor.heartbeat() ti.state = TaskInstanceState.QUEUED ti.queued_by_job_id = self.job.id ti.queued_dttm = timezone.utcnow() @@ -741,6 +757,7 @@ def to_keep(key: TaskInstanceKey) -> bool: ti_status.to_run.update({ti.key: ti for ti in new_mapped_tis}) for new_ti in new_mapped_tis: + new_ti.try_number += 1 new_ti.set_state(TaskInstanceState.SCHEDULED, session=session) # Set state to failed for running TIs that are set up for retry if disable-retry flag is set @@ -860,7 +877,6 @@ def _execute_dagruns( self.log.info("tis_map=%s", tis_map) ti_status.active_runs.add(dag_run) ti_status.to_run.update(tis_map or {}) - self.log.info("") processed_dag_run_dates = self._process_backfill_task_instances( ti_status=ti_status, executor=executor, @@ -948,7 +964,6 @@ def _execute(self, session: Session = NEW_SESSION) -> None: "combination. Please adjust backfill dates or wait for this DagRun to finish.", ) return - # picklin' pickle_id = None executor_class, _ = ExecutorLoader.import_default_executor_cls() diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index 56deacbf55ab1..8f66c417c4c9a 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -883,12 +883,12 @@ def test_backfill_retry_intermittent_failed_task(self, dag_maker): dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[ - TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1) - ] = State.UP_FOR_RETRY - executor.mock_task_results[ - TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2) - ] = State.UP_FOR_RETRY + executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1)] = ( + State.UP_FOR_RETRY + ) + executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2)] = ( + State.UP_FOR_RETRY + ) job = Job(executor=executor) job_runner = BackfillJobRunner( job=job, @@ -911,9 +911,9 @@ def test_backfill_retry_always_failed_task(self, dag_maker): dr = dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[ - TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0) - ] = State.UP_FOR_RETRY + executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0)] = ( + State.UP_FOR_RETRY + ) executor.mock_task_fail(dag.dag_id, task1.task_id, dr.run_id, try_number=1) job = Job(executor=executor) job_runner = BackfillJobRunner( From c7805f9f926d3664c30fb7e44a444ead3091b6ad Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 12:24:53 -0700 Subject: [PATCH 23/53] fix mock executor changing try_number --- tests/test_utils/mock_executor.py | 1 - 1 file changed, 1 deletion(-) diff --git a/tests/test_utils/mock_executor.py b/tests/test_utils/mock_executor.py index 17845b0861399..eaf5d32d8695a 100644 --- a/tests/test_utils/mock_executor.py +++ b/tests/test_utils/mock_executor.py @@ -71,7 +71,6 @@ def sort_by(item): sorted_queue = sorted(self.queued_tasks.items(), key=sort_by) for key, (_, _, _, ti) in sorted_queue[:open_slots]: self.queued_tasks.pop(key) - ti.try_number += 1 state = self.mock_task_results[key] ti.set_state(state, session=session) self.change_state(key, state) From e23e251363d3418b3e0ae8ddccda614b51eece83 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 12:25:40 -0700 Subject: [PATCH 24/53] misc test fixes --- tests/jobs/test_scheduler_job.py | 3 +- tests/jobs/test_triggerer_job.py | 2 +- tests/models/test_cleartasks.py | 72 +++++++++++++++++++++++++------- tests/models/test_dag.py | 4 +- 4 files changed, 63 insertions(+), 18 deletions(-) diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index f122f122658b5..e10abeb3eff16 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -3180,7 +3180,7 @@ def run_with_error(ti, ignore_ti_state=False): # executing task. run_with_error(ti, ignore_ti_state=True) assert ti.state == State.UP_FOR_RETRY - assert ti.try_number == 2 + assert ti.try_number == 1 with create_session() as session: ti.refresh_from_db(lock_for_update=True, session=session) @@ -3191,6 +3191,7 @@ def run_with_error(ti, ignore_ti_state=False): executor.do_update = True do_schedule() ti.refresh_from_db() + assert ti.try_number == 2 assert ti.state == State.SUCCESS def test_retry_handling_job(self): diff --git a/tests/jobs/test_triggerer_job.py b/tests/jobs/test_triggerer_job.py index 9279c97d533af..df9699806fb4c 100644 --- a/tests/jobs/test_triggerer_job.py +++ b/tests/jobs/test_triggerer_job.py @@ -152,7 +152,7 @@ def __init__(self, password, **kwargs): # give it more time for the trigger event to write the log. time.sleep(0.5) - assert "test_dag/test_run/sensitive_arg_task/-1/1 (ID 1) starting" in caplog.text + assert "test_dag/test_run/sensitive_arg_task/-1/0 (ID 1) starting" in caplog.text assert "some_password" not in caplog.text diff --git a/tests/models/test_cleartasks.py b/tests/models/test_cleartasks.py index ec2059c84e87b..26e3dcc9e4de3 100644 --- a/tests/models/test_cleartasks.py +++ b/tests/models/test_cleartasks.py @@ -68,6 +68,13 @@ def test_clear_task_instances(self, dag_maker): ti1.run() with create_session() as session: + # do the incrementing of try_number ordinarily handled by scheduler + ti0.try_number += 1 + ti1.try_number += 1 + session.merge(ti0) + session.merge(ti1) + session.commit() + # we use order_by(task_id) here because for the test DAG structure of ours # this is equivalent to topological sort. It would not work in general case # but it works for our case because we specifically constructed test DAGS @@ -79,10 +86,10 @@ def test_clear_task_instances(self, dag_maker): ti1.refresh_from_db() # Next try to run will be try 2 assert ti0.state is None - assert ti0.try_number == 2 + assert ti0.try_number == 1 assert ti0.max_tries == 1 assert ti1.state is None - assert ti1.try_number == 2 + assert ti1.try_number == 1 assert ti1.max_tries == 3 def test_clear_task_instances_external_executor_id(self, dag_maker): @@ -279,6 +286,14 @@ def test_clear_task_instances_without_task(self, dag_maker): ti0.refresh_from_task(task0) ti1.refresh_from_task(task1) + with create_session() as session: + # do the incrementing of try_number ordinarily handled by scheduler + ti0.try_number += 1 + ti1.try_number += 1 + session.merge(ti0) + session.merge(ti1) + session.commit() + ti0.run() ti1.run() @@ -298,10 +313,9 @@ def test_clear_task_instances_without_task(self, dag_maker): # When no task is found, max_tries will be maximum of original max_tries or try_number. ti0.refresh_from_db() ti1.refresh_from_db() - # Next try to run will be try 2 - assert ti0.try_number == 2 + assert ti0.try_number == 1 assert ti0.max_tries == 1 - assert ti1.try_number == 2 + assert ti1.try_number == 1 assert ti1.max_tries == 2 def test_clear_task_instances_without_dag(self, dag_maker): @@ -323,6 +337,14 @@ def test_clear_task_instances_without_dag(self, dag_maker): ti0.refresh_from_task(task0) ti1.refresh_from_task(task1) + with create_session() as session: + # do the incrementing of try_number ordinarily handled by scheduler + ti0.try_number += 1 + ti1.try_number += 1 + session.merge(ti0) + session.merge(ti1) + session.commit() + ti0.run() ti1.run() @@ -337,10 +359,9 @@ def test_clear_task_instances_without_dag(self, dag_maker): # When no DAG is found, max_tries will be maximum of original max_tries or try_number. ti0.refresh_from_db() ti1.refresh_from_db() - # Next try to run will be try 2 - assert ti0.try_number == 2 + assert ti0.try_number == 1 assert ti0.max_tries == 1 - assert ti1.try_number == 2 + assert ti1.try_number == 1 assert ti1.max_tries == 2 def test_clear_task_instances_without_dag_param(self, dag_maker, session): @@ -365,6 +386,14 @@ def test_clear_task_instances_without_dag_param(self, dag_maker, session): ti0.refresh_from_task(task0) ti1.refresh_from_task(task1) + with create_session() as session: + # do the incrementing of try_number ordinarily handled by scheduler + ti0.try_number += 1 + ti1.try_number += 1 + session.merge(ti0) + session.merge(ti1) + session.commit() + ti0.run(session=session) ti1.run(session=session) @@ -377,10 +406,9 @@ def test_clear_task_instances_without_dag_param(self, dag_maker, session): ti0.refresh_from_db(session=session) ti1.refresh_from_db(session=session) - # Next try to run will be try 2 - assert ti0.try_number == 2 + assert ti0.try_number == 1 assert ti0.max_tries == 1 - assert ti1.try_number == 2 + assert ti1.try_number == 1 assert ti1.max_tries == 3 def test_clear_task_instances_in_multiple_dags(self, dag_maker, session): @@ -418,6 +446,14 @@ def test_clear_task_instances_in_multiple_dags(self, dag_maker, session): ti0.refresh_from_task(task0) ti1.refresh_from_task(task1) + with create_session() as session: + # do the incrementing of try_number ordinarily handled by scheduler + ti0.try_number += 1 + ti1.try_number += 1 + session.merge(ti0) + session.merge(ti1) + session.commit() + ti0.run(session=session) ti1.run(session=session) @@ -426,10 +462,9 @@ def test_clear_task_instances_in_multiple_dags(self, dag_maker, session): ti0.refresh_from_db(session=session) ti1.refresh_from_db(session=session) - # Next try to run will be try 2 - assert ti0.try_number == 2 + assert ti0.try_number == 1 assert ti0.max_tries == 1 - assert ti1.try_number == 2 + assert ti1.try_number == 1 assert ti1.max_tries == 3 def test_clear_task_instances_with_task_reschedule(self, dag_maker): @@ -451,6 +486,15 @@ def test_clear_task_instances_with_task_reschedule(self, dag_maker): ti0, ti1 = sorted(dr.task_instances, key=lambda ti: ti.task_id) ti0.refresh_from_task(task0) ti1.refresh_from_task(task1) + + with create_session() as session: + # do the incrementing of try_number ordinarily handled by scheduler + ti0.try_number += 1 + ti1.try_number += 1 + session.merge(ti0) + session.merge(ti1) + session.commit() + ti0.run() ti1.run() diff --git a/tests/models/test_dag.py b/tests/models/test_dag.py index 64acad83deab2..bb134d8728077 100644 --- a/tests/models/test_dag.py +++ b/tests/models/test_dag.py @@ -3480,8 +3480,8 @@ def get_ti_from_db(task): assert dagrun.get_state() == State.QUEUED assert {t.key for t in altered} == { - ("test_set_task_group_state", "section_1.task_1", dagrun.run_id, 1, -1), - ("test_set_task_group_state", "section_1.task_3", dagrun.run_id, 1, -1), + ("test_set_task_group_state", "section_1.task_1", dagrun.run_id, 0, -1), + ("test_set_task_group_state", "section_1.task_3", dagrun.run_id, 0, -1), } From c59c4bb67a0ea6af4eb9290f973508a3607b5c9d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 12:26:03 -0700 Subject: [PATCH 25/53] fix try number incrementing in backfill for non-scheduled tasks --- airflow/jobs/backfill_job_runner.py | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index 58fa38d436edf..4456faecd09f8 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -574,10 +574,11 @@ def _per_task_process(key, ti: TaskInstance, session): else: self.log.debug("Sending %s to executor", ti) # Skip scheduled state, we are executing immediately - if ti.state == TaskInstanceState.UP_FOR_RETRY: + if ti.state in (TaskInstanceState.UP_FOR_RETRY, None): # i am not sure why this is necessary. # seemingly a quirk of backfill runner. # it should be handled elsewhere i think. + # seems the leaf tasks are set SCHEDULED but others not. # but i am not going to look too closely since we need # to nuke the current backfill approach anyway. ti.try_number += 1 From fc83374fce23ffd97f26064c1372728e22b2bd6e Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 12:26:25 -0700 Subject: [PATCH 26/53] openlineage take actual try_number --- airflow/providers/dbt/cloud/utils/openlineage.py | 2 +- airflow/providers/openlineage/plugins/listener.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/providers/dbt/cloud/utils/openlineage.py b/airflow/providers/dbt/cloud/utils/openlineage.py index 5e4550b677697..973ec459fe454 100644 --- a/airflow/providers/dbt/cloud/utils/openlineage.py +++ b/airflow/providers/dbt/cloud/utils/openlineage.py @@ -131,7 +131,7 @@ async def get_artifacts_for_steps(steps, artifacts): dag_id=task_instance.dag_id, task_id=operator.task_id, execution_date=task_instance.execution_date, - try_number=task_instance.try_number - 1, + try_number=task_instance.try_number, ) parent_job = ParentRunMetadata( diff --git a/airflow/providers/openlineage/plugins/listener.py b/airflow/providers/openlineage/plugins/listener.py index 25ded6d7f4361..27a221db92ad0 100644 --- a/airflow/providers/openlineage/plugins/listener.py +++ b/airflow/providers/openlineage/plugins/listener.py @@ -165,7 +165,7 @@ def on_success(): dag_id=dag.dag_id, task_id=task.task_id, execution_date=task_instance.execution_date, - try_number=task_instance.try_number - 1, + try_number=task_instance.try_number, ) event_type = RunState.COMPLETE.value.lower() operator_name = task.task_type.lower() From f1549419aba64f8e8f9ca5af4c0aa04ba7713edc Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 12:27:52 -0700 Subject: [PATCH 27/53] backfill test fixes --- tests/jobs/test_backfill_job.py | 39 ++++++++++++++------------------- 1 file changed, 17 insertions(+), 22 deletions(-) diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index 8f66c417c4c9a..76ccbab121e62 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -202,7 +202,6 @@ def test_backfill_multi_dates(self): ) run_job(job=job, execute_callable=job_runner._execute) - expected_execution_order = [ ("runme_0", DEFAULT_DATE), ("runme_1", DEFAULT_DATE), @@ -220,7 +219,7 @@ def test_backfill_multi_dates(self): actual = [(tuple(x), y) for x, y in executor.sorted_tasks] expected = [ ( - (dag.dag_id, task_id, f"backfill__{when.isoformat()}", 0, -1), + (dag.dag_id, task_id, f"backfill__{when.isoformat()}", 1, -1), (State.SUCCESS, None), ) for (task_id, when) in expected_execution_order @@ -883,12 +882,12 @@ def test_backfill_retry_intermittent_failed_task(self, dag_maker): dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1)] = ( - State.UP_FOR_RETRY - ) - executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2)] = ( - State.UP_FOR_RETRY - ) + executor.mock_task_results[ + TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1) + ] = State.UP_FOR_RETRY + executor.mock_task_results[ + TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2) + ] = State.UP_FOR_RETRY job = Job(executor=executor) job_runner = BackfillJobRunner( job=job, @@ -911,9 +910,9 @@ def test_backfill_retry_always_failed_task(self, dag_maker): dr = dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0)] = ( - State.UP_FOR_RETRY - ) + executor.mock_task_results[ + TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0) + ] = State.UP_FOR_RETRY executor.mock_task_fail(dag.dag_id, task1.task_id, dr.run_id, try_number=1) job = Job(executor=executor) job_runner = BackfillJobRunner( @@ -1611,10 +1610,6 @@ def test_update_counters(self, dag_maker, session): # test for none ti.set_state(State.NONE, session) - # Setting ti.try_number = 0 brings us to ti.try_number==1 - # so that the in-memory key access will work fine - ti.try_number = 0 - assert ti.try_number == 1 # see ti.try_number property in taskinstance module session.merge(ti) session.commit() ti_status.running[ti.key] = ti @@ -1956,20 +1951,20 @@ def on_change_state(key, state, info=None): ) assert ti_status.failed == set() assert ti_status.succeeded == { - TaskInstanceKey(dag_id=dr.dag_id, task_id="consumer", run_id="test", try_number=1, map_index=0), - TaskInstanceKey(dag_id=dr.dag_id, task_id="consumer", run_id="test", try_number=1, map_index=1), - TaskInstanceKey(dag_id=dr.dag_id, task_id="consumer", run_id="test", try_number=1, map_index=2), + TaskInstanceKey(dag_id=dr.dag_id, task_id="consumer", run_id="test", try_number=0, map_index=0), + TaskInstanceKey(dag_id=dr.dag_id, task_id="consumer", run_id="test", try_number=0, map_index=1), + TaskInstanceKey(dag_id=dr.dag_id, task_id="consumer", run_id="test", try_number=0, map_index=2), TaskInstanceKey( - dag_id=dr.dag_id, task_id="consumer_literal", run_id="test", try_number=1, map_index=0 + dag_id=dr.dag_id, task_id="consumer_literal", run_id="test", try_number=0, map_index=0 ), TaskInstanceKey( - dag_id=dr.dag_id, task_id="consumer_literal", run_id="test", try_number=1, map_index=1 + dag_id=dr.dag_id, task_id="consumer_literal", run_id="test", try_number=0, map_index=1 ), TaskInstanceKey( - dag_id=dr.dag_id, task_id="consumer_literal", run_id="test", try_number=1, map_index=2 + dag_id=dr.dag_id, task_id="consumer_literal", run_id="test", try_number=0, map_index=2 ), TaskInstanceKey( - dag_id=dr.dag_id, task_id="make_arg_lists", run_id="test", try_number=1, map_index=-1 + dag_id=dr.dag_id, task_id="make_arg_lists", run_id="test", try_number=0, map_index=-1 ), } From eb8f4d5748365ca3e67cc2f0ef9f74abedda567d Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 15:35:13 -0700 Subject: [PATCH 28/53] test fixes --- tests/jobs/test_backfill_job.py | 18 +++++++++--------- tests/jobs/test_scheduler_job.py | 4 +--- tests/models/test_taskinstance.py | 30 ++++++++++++++++++------------ tests/utils/test_log_handlers.py | 4 ++-- 4 files changed, 30 insertions(+), 26 deletions(-) diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index 76ccbab121e62..6b127a8d20a76 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -882,12 +882,12 @@ def test_backfill_retry_intermittent_failed_task(self, dag_maker): dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[ - TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1) - ] = State.UP_FOR_RETRY - executor.mock_task_results[ - TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2) - ] = State.UP_FOR_RETRY + executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1)] = ( + State.UP_FOR_RETRY + ) + executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2)] = ( + State.UP_FOR_RETRY + ) job = Job(executor=executor) job_runner = BackfillJobRunner( job=job, @@ -910,9 +910,9 @@ def test_backfill_retry_always_failed_task(self, dag_maker): dr = dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[ - TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0) - ] = State.UP_FOR_RETRY + executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0)] = ( + State.UP_FOR_RETRY + ) executor.mock_task_fail(dag.dag_id, task1.task_id, dr.run_id, try_number=1) job = Job(executor=executor) job_runner = BackfillJobRunner( diff --git a/tests/jobs/test_scheduler_job.py b/tests/jobs/test_scheduler_job.py index e10abeb3eff16..491e345649fe1 100644 --- a/tests/jobs/test_scheduler_job.py +++ b/tests/jobs/test_scheduler_job.py @@ -3191,7 +3191,7 @@ def run_with_error(ti, ignore_ti_state=False): executor.do_update = True do_schedule() ti.refresh_from_db() - assert ti.try_number == 2 + assert ti.try_number == 1 assert ti.state == State.SUCCESS def test_retry_handling_job(self): @@ -3215,8 +3215,6 @@ def test_retry_handling_job(self): .filter(TaskInstance.dag_id == dag.dag_id, TaskInstance.task_id == dag_task1.task_id) .first() ) - # make sure the counter has increased - assert ti.try_number == 2 assert ti.state == State.UP_FOR_RETRY def test_dag_get_active_runs(self, dag_maker): diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index 688abfdf1064d..b44e994170373 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -647,7 +647,7 @@ def run_with_error(ti): assert ti.state == State.FAILED assert ti.try_number == 3 - def test_retry_handling(self, dag_maker): + def test_retry_handling(self, dag_maker, session): """ Test that task retries are handled properly """ @@ -671,36 +671,44 @@ def run_with_error(ti): ti = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances[0] ti.task = task - assert ti.try_number == 1 + assert ti.try_number == 0 + + session.get(TaskInstance, ti.key.primary).try_number += 1 + session.commit() # first run -- up for retry run_with_error(ti) assert ti.state == State.UP_FOR_RETRY assert ti.try_number == 1 - assert ti.try_number == 2 + + session.get(TaskInstance, ti.key.primary).try_number += 1 + session.commit() # second run -- fail run_with_error(ti) assert ti.state == State.FAILED assert ti.try_number == 2 - assert ti.try_number == 3 # Clear the TI state since you can't run a task with a FAILED state without # clearing it first dag.clear() + session.get(TaskInstance, ti.key.primary).try_number += 1 + session.commit() + # third run -- up for retry run_with_error(ti) assert ti.state == State.UP_FOR_RETRY assert ti.try_number == 3 - assert ti.try_number == 4 + + session.get(TaskInstance, ti.key.primary).try_number += 1 + session.commit() # fourth run -- fail run_with_error(ti) ti.refresh_from_db() assert ti.state == State.FAILED assert ti.try_number == 4 - assert ti.try_number == 5 assert RenderedTaskInstanceFields.get_templated_fields(ti) == expected_rendered_ti_fields def test_next_retry_datetime(self, dag_maker): @@ -1003,7 +1011,6 @@ def func(): ti = dag_maker.create_dagrun(execution_date=timezone.utcnow()).task_instances[0] ti.task = task assert ti.try_number == 0 - assert ti.try_number == 1 def run_ti_and_assert( run_date, @@ -1024,7 +1031,6 @@ def run_ti_and_assert( ti.refresh_from_db() assert ti.state == expected_state assert ti.try_number == expected_try_number - assert ti.try_number == expected_try_number + 1 assert ti.start_date == expected_start_date assert ti.end_date == expected_end_date assert ti.duration == expected_duration @@ -3042,7 +3048,7 @@ def test_handle_failure(self, create_dummy_dag, session=None): assert "task_instance" in context_arg_3 mock_on_retry_3.assert_not_called() - def test_handle_failure_updates_queued_task_try_number(self, dag_maker): + def test_handle_failure_updates_queued_task_updates_state(self, dag_maker): session = settings.Session() with dag_maker(): task = EmptyOperator(task_id="mytask", retries=1) @@ -3052,11 +3058,8 @@ def test_handle_failure_updates_queued_task_try_number(self, dag_maker): session.merge(ti) session.flush() assert ti.state == State.QUEUED - assert ti.try_number == 1 ti.handle_failure("test queued ti", test_mode=True) assert ti.state == State.UP_FOR_RETRY - # try_number remains at 1 - assert ti.try_number == 1 @patch.object(Stats, "incr") def test_handle_failure_no_task(self, Stats_incr, dag_maker): @@ -3069,6 +3072,7 @@ def test_handle_failure_no_task(self, Stats_incr, dag_maker): task = EmptyOperator(task_id="mytask", retries=1) dr = dag_maker.create_dagrun() ti = TI(task=task, run_id=dr.run_id) + ti.try_number += 1 ti = session.merge(ti) ti.task = None ti.state = State.QUEUED @@ -4707,6 +4711,8 @@ def test__refresh_from_db_should_not_increment_try_number(dag_maker, session): BashOperator(task_id="hello", bash_command="hi") dag_maker.create_dagrun(state="success") ti = session.scalar(select(TaskInstance)) + session.get(TaskInstance, ti.key.primary).try_number += 1 + session.commit() assert ti.task_id == "hello" # just to confirm... assert ti.try_number == 1 # starts out as 1 ti.refresh_from_db() diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index 077270480042d..a7da313145abb 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -122,7 +122,7 @@ def task_callable(ti): # We expect set_context generates a file locally. log_filename = file_handler.handler.baseFilename assert os.path.isfile(log_filename) - assert log_filename.endswith("1.log"), log_filename + assert log_filename.endswith("0.log"), log_filename ti.run(ignore_ti_state=True) @@ -161,7 +161,7 @@ def task_callable(ti): python_callable=task_callable, ) ti = TaskInstance(task=task, run_id=dagrun.run_id) - + ti.try_number += 1 logger = ti.log ti.log.disabled = False From d4d69d2661291e80edc726536039a779d038c1f8 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 18:36:55 -0700 Subject: [PATCH 29/53] fix tests --- tests/jobs/test_backfill_job.py | 18 +++++++++--------- tests/utils/test_log_handlers.py | 2 +- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index 6b127a8d20a76..76ccbab121e62 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -882,12 +882,12 @@ def test_backfill_retry_intermittent_failed_task(self, dag_maker): dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1)] = ( - State.UP_FOR_RETRY - ) - executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2)] = ( - State.UP_FOR_RETRY - ) + executor.mock_task_results[ + TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1) + ] = State.UP_FOR_RETRY + executor.mock_task_results[ + TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2) + ] = State.UP_FOR_RETRY job = Job(executor=executor) job_runner = BackfillJobRunner( job=job, @@ -910,9 +910,9 @@ def test_backfill_retry_always_failed_task(self, dag_maker): dr = dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0)] = ( - State.UP_FOR_RETRY - ) + executor.mock_task_results[ + TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0) + ] = State.UP_FOR_RETRY executor.mock_task_fail(dag.dag_id, task1.task_id, dr.run_id, try_number=1) job = Job(executor=executor) job_runner = BackfillJobRunner( diff --git a/tests/utils/test_log_handlers.py b/tests/utils/test_log_handlers.py index a7da313145abb..4f7eb9471c57f 100644 --- a/tests/utils/test_log_handlers.py +++ b/tests/utils/test_log_handlers.py @@ -498,7 +498,7 @@ def test_set_context_trigger(self, create_dummy_dag, dag_maker, is_a_trigger, se t.task_instance = ti h = FileTaskHandler(base_log_folder=os.fspath(tmp_path)) h.set_context(ti) - expected = "dag_id=test_fth/run_id=test/task_id=dummy/attempt=1.log" + expected = "dag_id=test_fth/run_id=test/task_id=dummy/attempt=0.log" if is_a_trigger: expected += f".trigger.{job.id}.log" actual = h.handler.baseFilename From 3efc668ae9ae5902ad09397725d3b46e2aa08e6a Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 18:38:18 -0700 Subject: [PATCH 30/53] revert some backfill changes --- airflow/jobs/backfill_job_runner.py | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index 4456faecd09f8..33bdf87e3aaab 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -416,7 +416,6 @@ def _task_instances_for_dag_run( :param dag_run: the dag run to get the tasks from :param session: the database session object """ - self.log.info("_task_instances_for_dag_run") tasks_to_run = {} if dag_run is None: @@ -432,23 +431,14 @@ def _task_instances_for_dag_run( dag_run.dag = dag info = dag_run.task_instance_scheduling_decisions(session=session) schedulable_tis = info.schedulable_tis - self.log.info("schedulable_tis: %s", schedulable_tis) try: for ti in dag_run.get_task_instances(session=session): - session.merge(ti) if ti in schedulable_tis: if not ti.state == TaskInstanceState.UP_FOR_RESCHEDULE: ti.try_number += 1 ti.set_state(TaskInstanceState.SCHEDULED) if ti.state != TaskInstanceState.REMOVED: tasks_to_run[ti.key] = ti - if ti.state == TaskInstanceState.UP_FOR_RETRY and ti not in schedulable_tis: - self.log.warning( - "unexpected. task_id=%s has state=%s but is not in schedulable_tis=%s", - ti.task_id, - ti.state, - schedulable_tis, - ) session.commit() except Exception: session.rollback() From 8f8ceb091a785ad0107c89a6f7f9316200e2f43b Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 18:40:06 -0700 Subject: [PATCH 31/53] revert logging changes --- airflow/jobs/backfill_job_runner.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index 33bdf87e3aaab..ab721b9d0c7c9 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -500,20 +500,20 @@ def _per_task_process(key, ti: TaskInstance, session): task = self.dag.get_task(ti.task_id, include_subdags=True) ti.task = task - self.log.info("Task instance to run %s state %s", ti, ti.state) + self.log.debug("Task instance to run %s state %s", ti, ti.state) # The task was already marked successful or skipped by a # different Job. Don't rerun it. if ti.state == TaskInstanceState.SUCCESS: ti_status.succeeded.add(key) - self.log.info("Task instance %s succeeded. Don't rerun.", ti) + self.log.debug("Task instance %s succeeded. Don't rerun.", ti) ti_status.to_run.pop(key) if key in ti_status.running: ti_status.running.pop(key) return elif ti.state == TaskInstanceState.SKIPPED: ti_status.skipped.add(key) - self.log.info("Task instance %s skipped. Don't rerun.", ti) + self.log.debug("Task instance %s skipped. Don't rerun.", ti) ti_status.to_run.pop(key) if key in ti_status.running: ti_status.running.pop(key) @@ -522,7 +522,7 @@ def _per_task_process(key, ti: TaskInstance, session): if self.rerun_failed_tasks: # Rerun failed tasks or upstreamed failed tasks if ti.state in (TaskInstanceState.FAILED, TaskInstanceState.UPSTREAM_FAILED): - self.log.info("Task instance %s with state %s", ti, ti.state) + self.log.error("Task instance %s with state %s", ti, ti.state) if key in ti_status.running: ti_status.running.pop(key) # Reset the failed task in backfill to scheduled state From c98f0c8ff85f3642d82be03c2ede07398258b1ba Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 19:43:24 -0700 Subject: [PATCH 32/53] fix tests --- tests/jobs/test_backfill_job.py | 30 +++++++++++++++++++++++------- 1 file changed, 23 insertions(+), 7 deletions(-) diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index 76ccbab121e62..0c40f225fc6cd 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -955,10 +955,14 @@ def test_backfill_ordered_concurrent_execute(self, dag_maker): runid1 = f"backfill__{(DEFAULT_DATE + datetime.timedelta(days=1)).isoformat()}" runid2 = f"backfill__{(DEFAULT_DATE + datetime.timedelta(days=2)).isoformat()}" - # test executor history keeps a list - history = executor.history - - assert [sorted(item[-1].key[1:3] for item in batch) for batch in history] == [ + actual = [] + for batch in executor.history: + this_batch = [] + for cmd, idx, queue, ti in batch: + key = ti.key + this_batch.append((key.task_id, key.run_id)) + actual.append(sorted(this_batch)) + assert actual == [ [ ("leave1", runid0), ("leave1", runid1), @@ -967,9 +971,21 @@ def test_backfill_ordered_concurrent_execute(self, dag_maker): ("leave2", runid1), ("leave2", runid2), ], - [("upstream_level_1", runid0), ("upstream_level_1", runid1), ("upstream_level_1", runid2)], - [("upstream_level_2", runid0), ("upstream_level_2", runid1), ("upstream_level_2", runid2)], - [("upstream_level_3", runid0), ("upstream_level_3", runid1), ("upstream_level_3", runid2)], + [ + ("upstream_level_1", runid0), + ("upstream_level_1", runid1), + ("upstream_level_1", runid2), + ], + [ + ("upstream_level_2", runid0), + ("upstream_level_2", runid1), + ("upstream_level_2", runid2), + ], + [ + ("upstream_level_3", runid0), + ("upstream_level_3", runid1), + ("upstream_level_3", runid2), + ], ] def test_backfill_pooled_tasks(self): From fdcb6e5b5a6c720b831c6443cacfdde21ce01a06 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 19:49:51 -0700 Subject: [PATCH 33/53] revert backfill heartbeat change --- airflow/jobs/backfill_job_runner.py | 1 - 1 file changed, 1 deletion(-) diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index ab721b9d0c7c9..40ae0576fa5cc 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -572,7 +572,6 @@ def _per_task_process(key, ti: TaskInstance, session): # but i am not going to look too closely since we need # to nuke the current backfill approach anyway. ti.try_number += 1 - executor.heartbeat() ti.state = TaskInstanceState.QUEUED ti.queued_by_job_id = self.job.id ti.queued_dttm = timezone.utcnow() From d440d412465d11714b118f49f06e62c3a121a626 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 19:49:59 -0700 Subject: [PATCH 34/53] formatting --- tests/jobs/test_backfill_job.py | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index 0c40f225fc6cd..d54dccd0a2799 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -882,12 +882,12 @@ def test_backfill_retry_intermittent_failed_task(self, dag_maker): dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[ - TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1) - ] = State.UP_FOR_RETRY - executor.mock_task_results[ - TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2) - ] = State.UP_FOR_RETRY + executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=1)] = ( + State.UP_FOR_RETRY + ) + executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, DEFAULT_DATE, try_number=2)] = ( + State.UP_FOR_RETRY + ) job = Job(executor=executor) job_runner = BackfillJobRunner( job=job, @@ -910,9 +910,9 @@ def test_backfill_retry_always_failed_task(self, dag_maker): dr = dag_maker.create_dagrun(state=None) executor = MockExecutor(parallelism=16) - executor.mock_task_results[ - TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0) - ] = State.UP_FOR_RETRY + executor.mock_task_results[TaskInstanceKey(dag.dag_id, task1.task_id, dr.run_id, try_number=0)] = ( + State.UP_FOR_RETRY + ) executor.mock_task_fail(dag.dag_id, task1.task_id, dr.run_id, try_number=1) job = Job(executor=executor) job_runner = BackfillJobRunner( From c359514de0c2665b500355fb3814b88d7c793356 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 19:50:09 -0700 Subject: [PATCH 35/53] fix test --- tests/models/test_taskinstance.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tests/models/test_taskinstance.py b/tests/models/test_taskinstance.py index b44e994170373..d5e72f96d8d5b 100644 --- a/tests/models/test_taskinstance.py +++ b/tests/models/test_taskinstance.py @@ -3437,7 +3437,7 @@ def test_refresh_from_db(self, create_task_instance): "end_date": run_date + datetime.timedelta(days=1, seconds=1, milliseconds=234), "duration": 1.234, "state": State.SUCCESS, - "_try_number": 1, + "try_number": 1, "max_tries": 1, "hostname": "some_unique_hostname", "unixname": "some_unique_unixname", @@ -3463,7 +3463,7 @@ def test_refresh_from_db(self, create_task_instance): "task_display_name": "Test Refresh from DB Task", } # Make sure we aren't missing any new value in our expected_values list. - expected_keys = {f"task_instance.{key.lstrip('_')}" for key in expected_values} + expected_keys = {f"task_instance.{key}" for key in expected_values} assert {str(c) for c in TI.__table__.columns} == expected_keys, ( "Please add all non-foreign values of TaskInstance to this list. " "This prevents refresh_from_db() from missing a field." From ca169c63028804bca1cfeb1f3fdb35e622d8392c Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 20:12:42 -0700 Subject: [PATCH 36/53] static check fix --- tests/jobs/test_backfill_job.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index d54dccd0a2799..5e102600ee57e 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -958,7 +958,7 @@ def test_backfill_ordered_concurrent_execute(self, dag_maker): actual = [] for batch in executor.history: this_batch = [] - for cmd, idx, queue, ti in batch: + for cmd, idx, queue, ti in batch: # noqa: B007 key = ti.key this_batch.append((key.task_id, key.run_id)) actual.append(sorted(this_batch)) From 42f08e818cc2f34ddc642e0b3cd50f5c8b095daa Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 20:49:15 -0700 Subject: [PATCH 37/53] fix tests --- .../cncf/kubernetes/operators/test_pod.py | 14 +++--- .../kubernetes/test_template_rendering.py | 4 +- .../openlineage/plugins/test_listener.py | 45 ++++++++++--------- .../providers/smtp/notifications/test_smtp.py | 2 +- 4 files changed, 34 insertions(+), 31 deletions(-) diff --git a/tests/providers/cncf/kubernetes/operators/test_pod.py b/tests/providers/cncf/kubernetes/operators/test_pod.py index fe7c94146011d..224824edbc5e3 100644 --- a/tests/providers/cncf/kubernetes/operators/test_pod.py +++ b/tests/providers/cncf/kubernetes/operators/test_pod.py @@ -358,7 +358,7 @@ def test_labels(self, hook_mock, in_cluster): "dag_id": "dag", "kubernetes_pod_operator": "True", "task_id": "task", - "try_number": "1", + "try_number": "0", "airflow_version": mock.ANY, "run_id": "test", "airflow_kpo_in_cluster": str(in_cluster), @@ -374,7 +374,7 @@ def test_labels_mapped(self): "dag_id": "dag", "kubernetes_pod_operator": "True", "task_id": "task", - "try_number": "1", + "try_number": "0", "airflow_version": mock.ANY, "run_id": "test", "map_index": "10", @@ -884,7 +884,7 @@ def test_full_pod_spec(self, randomize_name, pod_spec): "dag_id": "dag", "kubernetes_pod_operator": "True", "task_id": "task", - "try_number": "1", + "try_number": "0", "airflow_version": mock.ANY, "airflow_kpo_in_cluster": str(k.hook.is_in_cluster), "run_id": "test", @@ -920,7 +920,7 @@ def test_full_pod_spec_kwargs(self, randomize_name, pod_spec): "dag_id": "dag", "kubernetes_pod_operator": "True", "task_id": "task", - "try_number": "1", + "try_number": "0", "airflow_version": mock.ANY, "airflow_kpo_in_cluster": str(k.hook.is_in_cluster), "run_id": "test", @@ -991,7 +991,7 @@ def test_pod_template_file(self, randomize_name, pod_template_file): "dag_id": "dag", "kubernetes_pod_operator": "True", "task_id": "task", - "try_number": "1", + "try_number": "0", "airflow_version": mock.ANY, "airflow_kpo_in_cluster": str(k.hook.is_in_cluster), "run_id": "test", @@ -1061,7 +1061,7 @@ def test_pod_template_file_kwargs_override(self, randomize_name, pod_template_fi "dag_id": "dag", "kubernetes_pod_operator": "True", "task_id": "task", - "try_number": "1", + "try_number": "0", "airflow_version": mock.ANY, "airflow_kpo_in_cluster": str(k.hook.is_in_cluster), "run_id": "test", @@ -1112,7 +1112,7 @@ def test_pod_template_dict(self, randomize_name): "dag_id": "dag", "kubernetes_pod_operator": "True", "task_id": "task", - "try_number": "1", + "try_number": "0", "airflow_version": mock.ANY, "airflow_kpo_in_cluster": str(k.hook.is_in_cluster), "run_id": "test", diff --git a/tests/providers/cncf/kubernetes/test_template_rendering.py b/tests/providers/cncf/kubernetes/test_template_rendering.py index 0627eb8d586f2..f3e61101eab8f 100644 --- a/tests/providers/cncf/kubernetes/test_template_rendering.py +++ b/tests/providers/cncf/kubernetes/test_template_rendering.py @@ -48,7 +48,7 @@ def test_render_k8s_pod_yaml(pod_mutation_hook, create_task_instance): "dag_id": "test_render_k8s_pod_yaml", "run_id": "test_run_id", "task_id": "op1", - "try_number": "1", + "try_number": "0", }, "labels": { "airflow-worker": "0", @@ -57,7 +57,7 @@ def test_render_k8s_pod_yaml(pod_mutation_hook, create_task_instance): "run_id": "test_run_id", "kubernetes_executor": "True", "task_id": "op1", - "try_number": "1", + "try_number": "0", }, "name": mock.ANY, "namespace": "default", diff --git a/tests/providers/openlineage/plugins/test_listener.py b/tests/providers/openlineage/plugins/test_listener.py index c37892c1f3dbe..fa651de1b22d9 100644 --- a/tests/providers/openlineage/plugins/test_listener.py +++ b/tests/providers/openlineage/plugins/test_listener.py @@ -297,12 +297,14 @@ def mock_task_id(dag_id, task_id, execution_date, try_number): listener.on_task_instance_success(None, task_instance, None) # This run_id will be different as we did NOT simulate increase of the try_number attribute, # which happens in Airflow. - listener.adapter.complete_task.assert_called_once_with( + calls = listener.adapter.complete_task.call_args_list + assert len(calls) == 1 + assert calls[0][1] == dict( end_time="2023-01-03T13:01:01", job_name="job_name", parent_job_name="dag_id", parent_run_id="dag_id.dag_run_run_id", - run_id="dag_id.task_id.execution_date.0", + run_id="dag_id.task_id.execution_date.1", task=listener.extractor_manager.extract_metadata(), ) @@ -310,12 +312,14 @@ def mock_task_id(dag_id, task_id, execution_date, try_number): listener.adapter.complete_task.reset_mock() task_instance.try_number += 1 listener.on_task_instance_success(None, task_instance, None) - listener.adapter.complete_task.assert_called_once_with( + calls = listener.adapter.complete_task.call_args_list + assert len(calls) == 1 + assert calls[0][1] == dict( end_time="2023-01-03T13:01:01", job_name="job_name", parent_job_name="dag_id", parent_run_id="dag_id.dag_run_run_id", - run_id="dag_id.task_id.execution_date.1", + run_id="dag_id.task_id.execution_date.2", task=listener.extractor_manager.extract_metadata(), ) @@ -334,24 +338,23 @@ def mock_task_id(dag_id, task_id, execution_date, try_number): listener, task_instance = _create_listener_and_task_instance() mocked_adapter.build_task_instance_run_id.side_effect = mock_task_id - + expected_run_id_1 = "dag_id.task_id.execution_date.1" + expected_run_id_2 = "dag_id.task_id.execution_date.2" listener.on_task_instance_running(None, task_instance, None) - expected_run_id = listener.adapter.start_task.call_args.kwargs["run_id"] - assert expected_run_id == "dag_id.task_id.execution_date.1" + assert listener.adapter.start_task.call_args.kwargs["run_id"] == expected_run_id_1 listener.on_task_instance_failed(None, task_instance, None) - assert listener.adapter.fail_task.call_args.kwargs["run_id"] == expected_run_id + assert listener.adapter.fail_task.call_args.kwargs["run_id"] == expected_run_id_1 - # This run_id will be different as we did NOT simulate increase of the try_number attribute, - # which happens in Airflow. + # This run_id will not be different as we did NOT simulate increase of the try_number attribute, listener.on_task_instance_success(None, task_instance, None) - assert listener.adapter.complete_task.call_args.kwargs["run_id"] == "dag_id.task_id.execution_date.0" + assert listener.adapter.complete_task.call_args.kwargs["run_id"] == expected_run_id_1 # Now we simulate the increase of try_number, and the run_id should reflect that change. # This is how airflow works, and that's why we expect the run_id to remain constant across all methods. task_instance.try_number += 1 listener.on_task_instance_success(None, task_instance, None) - assert listener.adapter.complete_task.call_args.kwargs["run_id"] == expected_run_id + assert listener.adapter.complete_task.call_args.kwargs["run_id"] == expected_run_id_2 def test_running_task_correctly_calls_openlineage_adapter_run_id_method(): @@ -403,7 +406,7 @@ def test_successful_task_correctly_calls_openlineage_adapter_run_id_method(mock_ dag_id="dag_id", task_id="task_id", execution_date="execution_date", - try_number=0, + try_number=1, ) @@ -428,16 +431,16 @@ def fail_callable(**kwargs): _, task_instance = _create_test_dag_and_task(fail_callable, "failure") # try_number before execution - assert task_instance.try_number == 1 + assert task_instance.try_number == 0 with suppress(CustomError): task_instance.run() # try_number at the moment of function being called - assert captured_try_numbers["running"] == 1 - assert captured_try_numbers["failed"] == 1 + assert captured_try_numbers["running"] == 0 + assert captured_try_numbers["failed"] == 0 # try_number after task has been executed - assert task_instance.try_number == 2 + assert task_instance.try_number == 0 @mock.patch("airflow.models.taskinstance.get_listener_manager") @@ -457,15 +460,15 @@ def success_callable(**kwargs): _, task_instance = _create_test_dag_and_task(success_callable, "success") # try_number before execution - assert task_instance.try_number == 1 + assert task_instance.try_number == 0 task_instance.run() # try_number at the moment of function being called - assert captured_try_numbers["running"] == 1 - assert captured_try_numbers["success"] == 2 + assert captured_try_numbers["running"] == 0 + assert captured_try_numbers["success"] == 0 # try_number after task has been executed - assert task_instance.try_number == 2 + assert task_instance.try_number == 0 @mock.patch("airflow.providers.openlineage.plugins.listener.is_operator_disabled") diff --git a/tests/providers/smtp/notifications/test_smtp.py b/tests/providers/smtp/notifications/test_smtp.py index f1a71e2b53c1b..b19cc4baa873b 100644 --- a/tests/providers/smtp/notifications/test_smtp.py +++ b/tests/providers/smtp/notifications/test_smtp.py @@ -129,7 +129,7 @@ def test_notifier_with_defaults(self, mock_smtphook_hook, create_task_instance): from_email=conf.get("smtp", "smtp_mail_from"), to="test_reciver@test.com", subject="DAG dag - Task op - Run ID test in State None", - html_content="""\n\n \n \n \n \n\n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n
Run ID:test
Try:1 of 1
Task State:None
Host:
Log Link:http://localhost:8080/dags/dag/grid?dag_run_id=test&task_id=op&map_index=-1&tab=logs
Mark Success Link:http://localhost:8080/confirm?task_id=op&dag_id=dag&dag_run_id=test&upstream=false&downstream=false&state=success
\n\n""", + html_content="""\n\n \n \n \n \n\n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n \n
Run ID:test
Try:0 of 1
Task State:None
Host:
Log Link:http://localhost:8080/dags/dag/grid?dag_run_id=test&task_id=op&map_index=-1&tab=logs
Mark Success Link:http://localhost:8080/confirm?task_id=op&dag_id=dag&dag_run_id=test&upstream=false&downstream=false&state=success
\n\n""", smtp_conn_id="smtp_default", files=None, cc=None, From 76c56ad44a3149535e83919f773eaf6e2b5e980f Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 22:32:58 -0700 Subject: [PATCH 38/53] fix tests --- tests/providers/celery/executors/test_celery_executor.py | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/tests/providers/celery/executors/test_celery_executor.py b/tests/providers/celery/executors/test_celery_executor.py index 4c62a24f816b3..e0913054d1dae 100644 --- a/tests/providers/celery/executors/test_celery_executor.py +++ b/tests/providers/celery/executors/test_celery_executor.py @@ -200,8 +200,6 @@ def test_try_adopt_task_instances_none(self): def test_try_adopt_task_instances(self): start_date = timezone.utcnow() - timedelta(days=2) - try_number = 1 - with DAG("test_try_adopt_task_instances_none") as dag: task_1 = BaseOperator(task_id="task_1", start_date=start_date) task_2 = BaseOperator(task_id="task_2", start_date=start_date) @@ -221,8 +219,8 @@ def test_try_adopt_task_instances(self): not_adopted_tis = executor.try_adopt_task_instances(tis) - key_1 = TaskInstanceKey(dag.dag_id, task_1.task_id, None, try_number) - key_2 = TaskInstanceKey(dag.dag_id, task_2.task_id, None, try_number) + key_1 = TaskInstanceKey(dag.dag_id, task_1.task_id, None, 0) + key_2 = TaskInstanceKey(dag.dag_id, task_2.task_id, None, 0) assert executor.running == {key_1, key_2} assert executor.tasks == {key_1: AsyncResult("231"), key_2: AsyncResult("232")} From 7559b88eeb4ba553aaab9f392d84636b9a90e84f Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Mon, 6 May 2024 22:38:33 -0700 Subject: [PATCH 39/53] fix tests --- kubernetes_tests/test_kubernetes_pod_operator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kubernetes_tests/test_kubernetes_pod_operator.py b/kubernetes_tests/test_kubernetes_pod_operator.py index 775217e872b82..d41e24ba91d0f 100644 --- a/kubernetes_tests/test_kubernetes_pod_operator.py +++ b/kubernetes_tests/test_kubernetes_pod_operator.py @@ -112,7 +112,7 @@ def setup_tests(self, test_label): "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "dag_id": "dag", "task_id": ANY, - "try_number": "1", + "try_number": "0", }, }, "spec": { From 017a2c43d58afbabd63f0db180339acba035c5b1 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 7 May 2024 09:40:29 -0700 Subject: [PATCH 40/53] fix test --- kubernetes_tests/test_kubernetes_pod_operator.py | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/kubernetes_tests/test_kubernetes_pod_operator.py b/kubernetes_tests/test_kubernetes_pod_operator.py index d41e24ba91d0f..fd1f1799eda38 100644 --- a/kubernetes_tests/test_kubernetes_pod_operator.py +++ b/kubernetes_tests/test_kubernetes_pod_operator.py @@ -768,7 +768,7 @@ def test_pod_template_file_with_overrides_system( "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "kubernetes_pod_operator": "True", "task_id": mock.ANY, - "try_number": "1", + "try_number": "0", } assert k.pod.spec.containers[0].env == [k8s.V1EnvVar(name="env_name", value="value")] assert result == {"hello": "world"} @@ -808,7 +808,7 @@ def test_pod_template_file_with_full_pod_spec(self, test_label, mock_get_connect "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "kubernetes_pod_operator": "True", "task_id": mock.ANY, - "try_number": "1", + "try_number": "0", } assert k.pod.spec.containers[0].env == [k8s.V1EnvVar(name="env_name", value="value")] assert result == {"hello": "world"} @@ -853,7 +853,7 @@ def test_full_pod_spec(self, test_label, mock_get_connection): "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "kubernetes_pod_operator": "True", "task_id": mock.ANY, - "try_number": "1", + "try_number": "0", } assert k.pod.spec.containers[0].env == [k8s.V1EnvVar(name="env_name", value="value")] assert result == {"hello": "world"} @@ -974,7 +974,7 @@ def test_pod_template_file( "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "kubernetes_pod_operator": "True", "task_id": mock.ANY, - "try_number": "1", + "try_number": "0", }, "name": "memory-demo", "namespace": "mem-example", From f9205af3d581f431d47fd977dacbefb494ff6536 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 7 May 2024 15:05:35 -0700 Subject: [PATCH 41/53] deal with some backcompat stuff --- airflow/models/taskinstance.py | 30 ++++++++++++++++--- airflow/providers/dbt/cloud/CHANGELOG.rst | 5 ++++ .../providers/dbt/cloud/utils/openlineage.py | 16 +++++++++- airflow/providers/openlineage/CHANGELOG.rst | 5 ++++ .../providers/openlineage/plugins/listener.py | 17 ++++++++++- airflow/www/views.py | 2 +- .../aws/executors/ecs/test_ecs_executor.py | 2 +- 7 files changed, 69 insertions(+), 8 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 90f4c3b94e474..fdcc8bb13687f 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -38,6 +38,7 @@ import jinja2 import lazy_object_proxy import pendulum +from deprecated import deprecated from jinja2 import TemplateAssertionError, UndefinedError from sqlalchemy import ( Column, @@ -1457,6 +1458,26 @@ def __init__( def __hash__(self): return hash((self.task_id, self.dag_id, self.run_id, self.map_index)) + @property + @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning) + def _try_number(self): + """ + Do not use. For semblance of backcompat. + + :meta private: + """ + return self.try_number + + @_try_number.setter + @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning) + def _try_number(self, val): + """ + Do not use. For semblance of backcompat. + + :meta private: + """ + self.try_number = val + @property def stats_tags(self) -> dict[str, str]: """Returns task instance tags.""" @@ -1499,13 +1520,14 @@ def init_on_load(self) -> None: self.test_mode = False # can be changed when calling 'run' @property + @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning) def prev_attempted_tries(self) -> int: """ - Calculate the number of previously attempted tries, defaulting to 0. + Calculate the total number of attempted tries, defaulting to 0. + + This used to be necessary because try_number did not always tell the truth. - Expose this for the Task Tries and Gantt graph views. - Using `try_number` throws off the counts for non-running tasks. - Also useful in error logging contexts to get the try number for the last try that was attempted. + :meta private: """ return self.try_number diff --git a/airflow/providers/dbt/cloud/CHANGELOG.rst b/airflow/providers/dbt/cloud/CHANGELOG.rst index cae2a00c3c056..bca94cf3ea249 100644 --- a/airflow/providers/dbt/cloud/CHANGELOG.rst +++ b/airflow/providers/dbt/cloud/CHANGELOG.rst @@ -28,6 +28,11 @@ Changelog --------- +main +..... + +In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state. Importantly, after the task is done, it no longer shows curr_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior. + 3.8.0 ..... diff --git a/airflow/providers/dbt/cloud/utils/openlineage.py b/airflow/providers/dbt/cloud/utils/openlineage.py index 973ec459fe454..fc75ce6004593 100644 --- a/airflow/providers/dbt/cloud/utils/openlineage.py +++ b/airflow/providers/dbt/cloud/utils/openlineage.py @@ -21,6 +21,11 @@ from contextlib import suppress from typing import TYPE_CHECKING +try: + from airflow import __version__ as airflow_version +except ImportError: + from airflow.version import version as airflow_version + if TYPE_CHECKING: from airflow.models.taskinstance import TaskInstance from airflow.providers.dbt.cloud.operators.dbt import DbtCloudRunJobOperator @@ -28,6 +33,15 @@ from airflow.providers.openlineage.extractors.base import OperatorLineage +def _get_try_number(val): + from packaging.version import parse + + if parse(parse(airflow_version).base_version) < parse("2.10.0"): + return val.try_number - 1 + else: + return val.try_number + + def generate_openlineage_events_from_dbt_cloud_run( operator: DbtCloudRunJobOperator | DbtCloudJobRunSensor, task_instance: TaskInstance ) -> OperatorLineage: @@ -131,7 +145,7 @@ async def get_artifacts_for_steps(steps, artifacts): dag_id=task_instance.dag_id, task_id=operator.task_id, execution_date=task_instance.execution_date, - try_number=task_instance.try_number, + try_number=_get_try_number(task_instance.try_number), ) parent_job = ParentRunMetadata( diff --git a/airflow/providers/openlineage/CHANGELOG.rst b/airflow/providers/openlineage/CHANGELOG.rst index 4e1f6ff80dd8f..ededd779d655b 100644 --- a/airflow/providers/openlineage/CHANGELOG.rst +++ b/airflow/providers/openlineage/CHANGELOG.rst @@ -26,6 +26,11 @@ Changelog --------- +main +..... + +In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state. Importantly, after the task is done, it no longer shows curr_try + 1. Thus in 1.7.2 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior. + 1.7.1 ..... diff --git a/airflow/providers/openlineage/plugins/listener.py b/airflow/providers/openlineage/plugins/listener.py index 27a221db92ad0..c18394637f576 100644 --- a/airflow/providers/openlineage/plugins/listener.py +++ b/airflow/providers/openlineage/plugins/listener.py @@ -37,6 +37,12 @@ from airflow.stats import Stats from airflow.utils.timeout import timeout +try: + from airflow import __version__ as airflow_version +except ImportError: + from airflow.version import version as airflow_version + + if TYPE_CHECKING: from sqlalchemy.orm import Session @@ -45,6 +51,15 @@ _openlineage_listener: OpenLineageListener | None = None +def _get_try_number_success(val): + from packaging.version import parse + + if parse(parse(airflow_version).base_version) < parse("2.10.0"): + return val.try_number - 1 + else: + return val.try_number + + class OpenLineageListener: """OpenLineage listener sends events on task instance and dag run starts, completes and failures.""" @@ -165,7 +180,7 @@ def on_success(): dag_id=dag.dag_id, task_id=task.task_id, execution_date=task_instance.execution_date, - try_number=task_instance.try_number, + try_number=_get_try_number_success(task_instance.try_number), ) event_type = RunState.COMPLETE.value.lower() operator_name = task.task_type.lower() diff --git a/airflow/www/views.py b/airflow/www/views.py index 625995854df06..131b25f20be9f 100644 --- a/airflow/www/views.py +++ b/airflow/www/views.py @@ -5196,7 +5196,7 @@ class TaskInstanceModelView(AirflowModelView): "pool", "queued_by_job_id", ] - + # todo: don't use prev_attempted_tries; just use try_number label_columns = {"dag_run.execution_date": "Logical Date", "prev_attempted_tries": "Try Number"} search_columns = [ diff --git a/tests/providers/amazon/aws/executors/ecs/test_ecs_executor.py b/tests/providers/amazon/aws/executors/ecs/test_ecs_executor.py index 524360dbac2ad..6c6bef5f7c7b1 100644 --- a/tests/providers/amazon/aws/executors/ecs/test_ecs_executor.py +++ b/tests/providers/amazon/aws/executors/ecs/test_ecs_executor.py @@ -1179,7 +1179,7 @@ def test_try_adopt_task_instances(self, mock_executor): orphaned_tasks[1].external_executor_id = "002" # Matches a running task_arn orphaned_tasks[2].external_executor_id = None # One orphaned task has no external_executor_id for task in orphaned_tasks: - task.prev_attempted_tries = 1 + task.try_number = 1 not_adopted_tasks = mock_executor.try_adopt_task_instances(orphaned_tasks) From f570d9fa6f743d7ea299a27a2e5442f55464ea56 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 7 May 2024 15:15:09 -0700 Subject: [PATCH 42/53] remove log adsd --- airflow/jobs/backfill_job_runner.py | 3 +-- airflow/jobs/job.py | 3 --- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index 40ae0576fa5cc..a5aaf463669b2 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -858,15 +858,14 @@ def _execute_dagruns( :param start_date: backfill start date :param session: the current session object """ - self.log.info("_execute_dagruns") for dagrun_info in dagrun_infos: for dag in self._get_dag_with_subdags(): dag_run = self._get_dag_run(dagrun_info, dag, session=session) if dag_run is not None: tis_map = self._task_instances_for_dag_run(dag, dag_run, session=session) - self.log.info("tis_map=%s", tis_map) ti_status.active_runs.add(dag_run) ti_status.to_run.update(tis_map or {}) + processed_dag_run_dates = self._process_backfill_task_instances( ti_status=ti_status, executor=executor, diff --git a/airflow/jobs/job.py b/airflow/jobs/job.py index dff472d1fee72..4273f1d3345b5 100644 --- a/airflow/jobs/job.py +++ b/airflow/jobs/job.py @@ -202,9 +202,7 @@ def heartbeat( try: # This will cause it to load from the db - self.log.info("fetching from DB") self._merge_from(Job._fetch_from_db(self, session)) - self.log.info("finished fetching from DB") previous_heartbeat = self.latest_heartbeat if self.state == JobState.RESTARTING: @@ -217,7 +215,6 @@ def heartbeat( self.heartrate - (timezone.utcnow() - self.latest_heartbeat).total_seconds() ) sleep_for = max(0, seconds_remaining) - print(f"sleeping for {sleep_for}") sleep(sleep_for) job = Job._update_heartbeat(job=self, session=session) From d04677497613e48dbc3ac0a98ab2e41ad5a98743 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 7 May 2024 15:39:44 -0700 Subject: [PATCH 43/53] fix spelling --- airflow/providers/dbt/cloud/CHANGELOG.rst | 2 +- airflow/providers/openlineage/CHANGELOG.rst | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/providers/dbt/cloud/CHANGELOG.rst b/airflow/providers/dbt/cloud/CHANGELOG.rst index bca94cf3ea249..bd5e64da00d1a 100644 --- a/airflow/providers/dbt/cloud/CHANGELOG.rst +++ b/airflow/providers/dbt/cloud/CHANGELOG.rst @@ -31,7 +31,7 @@ Changelog main ..... -In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state. Importantly, after the task is done, it no longer shows curr_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior. +In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state. Importantly, after the task is done, it no longer shows current_try + 1. Thus in 3.8.1 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior. 3.8.0 ..... diff --git a/airflow/providers/openlineage/CHANGELOG.rst b/airflow/providers/openlineage/CHANGELOG.rst index ededd779d655b..882c12139d1d3 100644 --- a/airflow/providers/openlineage/CHANGELOG.rst +++ b/airflow/providers/openlineage/CHANGELOG.rst @@ -29,7 +29,7 @@ Changelog main ..... -In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state. Importantly, after the task is done, it no longer shows curr_try + 1. Thus in 1.7.2 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior. +In Airflow 2.10.0, we fix the way try_number works, so that it no longer returns different values depending on task instance state. Importantly, after the task is done, it no longer shows current_try + 1. Thus in 1.7.2 we patch this provider to fix try_number references so they no longer adjust for the old, bad behavior. 1.7.1 ..... From 839d15a9ede4ea6032abdcb161715b1919c2aa18 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Tue, 7 May 2024 18:49:58 -0700 Subject: [PATCH 44/53] fix tests --- airflow/providers/dbt/cloud/utils/openlineage.py | 2 +- airflow/providers/openlineage/plugins/listener.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/airflow/providers/dbt/cloud/utils/openlineage.py b/airflow/providers/dbt/cloud/utils/openlineage.py index fc75ce6004593..4495201d950ce 100644 --- a/airflow/providers/dbt/cloud/utils/openlineage.py +++ b/airflow/providers/dbt/cloud/utils/openlineage.py @@ -145,7 +145,7 @@ async def get_artifacts_for_steps(steps, artifacts): dag_id=task_instance.dag_id, task_id=operator.task_id, execution_date=task_instance.execution_date, - try_number=_get_try_number(task_instance.try_number), + try_number=_get_try_number(task_instance), ) parent_job = ParentRunMetadata( diff --git a/airflow/providers/openlineage/plugins/listener.py b/airflow/providers/openlineage/plugins/listener.py index c18394637f576..880801b80519c 100644 --- a/airflow/providers/openlineage/plugins/listener.py +++ b/airflow/providers/openlineage/plugins/listener.py @@ -180,7 +180,7 @@ def on_success(): dag_id=dag.dag_id, task_id=task.task_id, execution_date=task_instance.execution_date, - try_number=_get_try_number_success(task_instance.try_number), + try_number=_get_try_number_success(task_instance), ) event_type = RunState.COMPLETE.value.lower() operator_name = task.task_type.lower() From 635e6b424705f02ee44719b2d1ff05391ddc5573 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 8 May 2024 09:13:07 -0700 Subject: [PATCH 45/53] add todo --- airflow/models/taskinstance.py | 1 + 1 file changed, 1 insertion(+) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index fdcc8bb13687f..dd0f1c7e31f81 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -1533,6 +1533,7 @@ def prev_attempted_tries(self) -> int: @property def next_try_number(self) -> int: + # todo (dstandish): deprecate this property; we don't need a property that is just + 1 return self.try_number + 1 @property From 2e2069de22a496b38856a7d27441409896e4b512 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 8 May 2024 10:11:47 -0700 Subject: [PATCH 46/53] news fragment --- newsfragments/39336.significant.rst | 5 +++++ 1 file changed, 5 insertions(+) create mode 100644 newsfragments/39336.significant.rst diff --git a/newsfragments/39336.significant.rst b/newsfragments/39336.significant.rst new file mode 100644 index 0000000000000..e29ea356086f8 --- /dev/null +++ b/newsfragments/39336.significant.rst @@ -0,0 +1,5 @@ +``try_number`` is no longer incremented during task execution + +Previously, the try_number was incremented at the beginning of task execution on the worker. This was problematic for many reasons. For one it meant that the try_number was incremented when it was not supposed to, namely when resuming from reschedule or deferral. And it also resulted in the try_number being "wrong" when the task had not yet started. The workarounds for these two issues caused a lot of confusion. What we do instead now, is we only increment the try_number when it is scheduled by the scheduler. So the try number for a task run is determined in advanced, and does not change in flight, and it is never decremented. So after the task runs, the observed try_number be remain the same as it was when the task was running; only when there is a "new try" will the try_number be incremented again. + +One consequence of this change is, if users were "manually" running tasks (e.g. by calling ``ti.run()`` directly, or command line ``airflow tasks run``), try_number will no longer be incremented. Airflow assumes that tasks are always run after being scheduled by the scheduler, so we do not regard this as a breaking change. From 41107f6bc416befe9ad54afafa39c4a96d019977 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 8 May 2024 10:13:43 -0700 Subject: [PATCH 47/53] todo --- airflow/providers/dbt/cloud/utils/openlineage.py | 1 + airflow/providers/openlineage/plugins/listener.py | 1 + 2 files changed, 2 insertions(+) diff --git a/airflow/providers/dbt/cloud/utils/openlineage.py b/airflow/providers/dbt/cloud/utils/openlineage.py index 4495201d950ce..a62df64c26246 100644 --- a/airflow/providers/dbt/cloud/utils/openlineage.py +++ b/airflow/providers/dbt/cloud/utils/openlineage.py @@ -34,6 +34,7 @@ def _get_try_number(val): + # todo: remove when min airflow version >= 2.10.0 from packaging.version import parse if parse(parse(airflow_version).base_version) < parse("2.10.0"): diff --git a/airflow/providers/openlineage/plugins/listener.py b/airflow/providers/openlineage/plugins/listener.py index 880801b80519c..a43c65830abd8 100644 --- a/airflow/providers/openlineage/plugins/listener.py +++ b/airflow/providers/openlineage/plugins/listener.py @@ -52,6 +52,7 @@ def _get_try_number_success(val): + # todo: remove when min airflow version >= 2.10.0 from packaging.version import parse if parse(parse(airflow_version).base_version) < parse("2.10.0"): From 54e5daf8d8294d190bd6cacec32b4b0db4deb545 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 8 May 2024 12:11:33 -0700 Subject: [PATCH 48/53] Apply suggestions from code review Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- airflow/models/taskinstance.py | 2 +- airflow/providers/amazon/aws/executors/ecs/ecs_executor.py | 2 +- newsfragments/39336.significant.rst | 6 ++++-- tests/jobs/test_backfill_job.py | 4 ++-- 4 files changed, 8 insertions(+), 6 deletions(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index dd0f1c7e31f81..16a172f76e859 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -1472,7 +1472,7 @@ def _try_number(self): @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning) def _try_number(self, val): """ - Do not use. For semblance of backcompat. + Do not use. For semblance of backcompat. :meta private: """ diff --git a/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py b/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py index a62058a4235a2..ca7047bb1ca22 100644 --- a/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py +++ b/airflow/providers/amazon/aws/executors/ecs/ecs_executor.py @@ -527,7 +527,7 @@ def try_adopt_task_instances(self, tis: Sequence[TaskInstance]) -> Sequence[Task ti.queue, ti.command_as_list(), ti.executor_config, - ti.try_number, # prev attempted tries + ti.try_number, ) adopted_tis.append(ti) diff --git a/newsfragments/39336.significant.rst b/newsfragments/39336.significant.rst index e29ea356086f8..750a1807881e4 100644 --- a/newsfragments/39336.significant.rst +++ b/newsfragments/39336.significant.rst @@ -1,5 +1,7 @@ ``try_number`` is no longer incremented during task execution -Previously, the try_number was incremented at the beginning of task execution on the worker. This was problematic for many reasons. For one it meant that the try_number was incremented when it was not supposed to, namely when resuming from reschedule or deferral. And it also resulted in the try_number being "wrong" when the task had not yet started. The workarounds for these two issues caused a lot of confusion. What we do instead now, is we only increment the try_number when it is scheduled by the scheduler. So the try number for a task run is determined in advanced, and does not change in flight, and it is never decremented. So after the task runs, the observed try_number be remain the same as it was when the task was running; only when there is a "new try" will the try_number be incremented again. +Previously, the try number (``try_number``) was incremented at the beginning of task execution on the worker. This was problematic for many reasons. For one it meant that the try number was incremented when it was not supposed to, namely when resuming from reschedule or deferral. And it also resulted in the try number being "wrong" when the task had not yet started. The workarounds for these two issues caused a lot of confusion. -One consequence of this change is, if users were "manually" running tasks (e.g. by calling ``ti.run()`` directly, or command line ``airflow tasks run``), try_number will no longer be incremented. Airflow assumes that tasks are always run after being scheduled by the scheduler, so we do not regard this as a breaking change. +Now, instead, the try number for a task run is determined at the time the task is scheduled, and does not change in flight, and it is never decremented. So after the task runs, the observed try number remains the same as it was when the task was running; only when there is a "new try" will the try number be incremented again. + +One consequence of this change is, if users were "manually" running tasks (e.g. by calling ``ti.run()`` directly, or command line ``airflow tasks run``), try number will no longer be incremented. Airflow assumes that tasks are always run after being scheduled by the scheduler, so we do not regard this as a breaking change. diff --git a/tests/jobs/test_backfill_job.py b/tests/jobs/test_backfill_job.py index 5e102600ee57e..5b7bc36b4de56 100644 --- a/tests/jobs/test_backfill_job.py +++ b/tests/jobs/test_backfill_job.py @@ -185,7 +185,7 @@ def test_trigger_controller_dag(self, session): assert task_instances_list - # @pytest.mark.backend("postgres", "mysql") + @pytest.mark.backend("postgres", "mysql") def test_backfill_multi_dates(self): dag = self.dagbag.get_dag("miscellaneous_test_dag") @@ -1544,7 +1544,7 @@ def test_update_counters(self, dag_maker, session): # match what's in the in-memory ti_status.running map. This is the same # for skipped, failed and retry states. ti_status.running[ti.key] = ti # Task is queued and marked as running - ti.try_number += 1 # Try number is increased during ti.run() + ti.try_number += 1 ti.set_state(State.SUCCESS, session) # Task finishes with success state job_runner._update_counters(ti_status=ti_status, session=session) # Update counters assert len(ti_status.running) == 0 From 62529002e4a9b0977e325dcc1be088d6ba6496ed Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 8 May 2024 12:12:52 -0700 Subject: [PATCH 49/53] Apply suggestions from code review Co-authored-by: Jed Cunningham <66968678+jedcunningham@users.noreply.github.com> --- airflow/models/taskinstance.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airflow/models/taskinstance.py b/airflow/models/taskinstance.py index 16a172f76e859..3bc96ec01ca92 100644 --- a/airflow/models/taskinstance.py +++ b/airflow/models/taskinstance.py @@ -1462,7 +1462,7 @@ def __hash__(self): @deprecated(reason="Use try_number instead.", version="2.10.0", category=RemovedInAirflow3Warning) def _try_number(self): """ - Do not use. For semblance of backcompat. + Do not use. For semblance of backcompat. :meta private: """ From cdf549fe3223be0cea5245392a6d3857dedd59fd Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 8 May 2024 12:37:48 -0700 Subject: [PATCH 50/53] imports --- airflow/providers/dbt/cloud/utils/openlineage.py | 5 +---- airflow/providers/openlineage/plugins/listener.py | 7 +------ 2 files changed, 2 insertions(+), 10 deletions(-) diff --git a/airflow/providers/dbt/cloud/utils/openlineage.py b/airflow/providers/dbt/cloud/utils/openlineage.py index a62df64c26246..ad50552b5146c 100644 --- a/airflow/providers/dbt/cloud/utils/openlineage.py +++ b/airflow/providers/dbt/cloud/utils/openlineage.py @@ -21,10 +21,7 @@ from contextlib import suppress from typing import TYPE_CHECKING -try: - from airflow import __version__ as airflow_version -except ImportError: - from airflow.version import version as airflow_version +from airflow import __version__ as airflow_version if TYPE_CHECKING: from airflow.models.taskinstance import TaskInstance diff --git a/airflow/providers/openlineage/plugins/listener.py b/airflow/providers/openlineage/plugins/listener.py index a43c65830abd8..03c60059d6686 100644 --- a/airflow/providers/openlineage/plugins/listener.py +++ b/airflow/providers/openlineage/plugins/listener.py @@ -23,6 +23,7 @@ from openlineage.client.serde import Serde +from airflow import __version__ as airflow_version from airflow.listeners import hookimpl from airflow.providers.openlineage.extractors import ExtractorManager from airflow.providers.openlineage.plugins.adapter import OpenLineageAdapter, RunState @@ -37,12 +38,6 @@ from airflow.stats import Stats from airflow.utils.timeout import timeout -try: - from airflow import __version__ as airflow_version -except ImportError: - from airflow.version import version as airflow_version - - if TYPE_CHECKING: from sqlalchemy.orm import Session From 3428b8919d0e8b02f92c4d9e60020ae7610933eb Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 8 May 2024 14:34:18 -0700 Subject: [PATCH 51/53] use try_number=1 for k8s synth ti --- kubernetes_tests/test_kubernetes_pod_operator.py | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/kubernetes_tests/test_kubernetes_pod_operator.py b/kubernetes_tests/test_kubernetes_pod_operator.py index fd1f1799eda38..a20fad9286358 100644 --- a/kubernetes_tests/test_kubernetes_pod_operator.py +++ b/kubernetes_tests/test_kubernetes_pod_operator.py @@ -62,6 +62,7 @@ def create_context(task) -> Context: task_instance = TaskInstance(task=task) task_instance.dag_run = dag_run task_instance.dag_id = dag.dag_id + task_instance.try_number = 1 task_instance.xcom_push = mock.Mock() # type: ignore return Context( dag=dag, @@ -112,7 +113,7 @@ def setup_tests(self, test_label): "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "dag_id": "dag", "task_id": ANY, - "try_number": "0", + "try_number": "1", }, }, "spec": { @@ -768,7 +769,7 @@ def test_pod_template_file_with_overrides_system( "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "kubernetes_pod_operator": "True", "task_id": mock.ANY, - "try_number": "0", + "try_number": "1", } assert k.pod.spec.containers[0].env == [k8s.V1EnvVar(name="env_name", value="value")] assert result == {"hello": "world"} @@ -808,7 +809,7 @@ def test_pod_template_file_with_full_pod_spec(self, test_label, mock_get_connect "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "kubernetes_pod_operator": "True", "task_id": mock.ANY, - "try_number": "0", + "try_number": "1", } assert k.pod.spec.containers[0].env == [k8s.V1EnvVar(name="env_name", value="value")] assert result == {"hello": "world"} @@ -853,7 +854,7 @@ def test_full_pod_spec(self, test_label, mock_get_connection): "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "kubernetes_pod_operator": "True", "task_id": mock.ANY, - "try_number": "0", + "try_number": "1", } assert k.pod.spec.containers[0].env == [k8s.V1EnvVar(name="env_name", value="value")] assert result == {"hello": "world"} From 2277e22732f25f73c9980a5f28823b65cc29321a Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 8 May 2024 15:11:18 -0700 Subject: [PATCH 52/53] fix test --- kubernetes_tests/test_kubernetes_pod_operator.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kubernetes_tests/test_kubernetes_pod_operator.py b/kubernetes_tests/test_kubernetes_pod_operator.py index a20fad9286358..2d10cdac6fef4 100644 --- a/kubernetes_tests/test_kubernetes_pod_operator.py +++ b/kubernetes_tests/test_kubernetes_pod_operator.py @@ -975,7 +975,7 @@ def test_pod_template_file( "run_id": "manual__2016-01-01T0100000100-da4d1ce7b", "kubernetes_pod_operator": "True", "task_id": mock.ANY, - "try_number": "0", + "try_number": "1", }, "name": "memory-demo", "namespace": "mem-example", From da82a1ce7177873637d7182cf6ad7072086ebc77 Mon Sep 17 00:00:00 2001 From: Daniel Standish <15932138+dstandish@users.noreply.github.com> Date: Wed, 8 May 2024 15:11:34 -0700 Subject: [PATCH 53/53] tweaks --- airflow/jobs/backfill_job_runner.py | 2 +- airflow/models/dag.py | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/airflow/jobs/backfill_job_runner.py b/airflow/jobs/backfill_job_runner.py index a5aaf463669b2..4fd6b24bbfc2a 100644 --- a/airflow/jobs/backfill_job_runner.py +++ b/airflow/jobs/backfill_job_runner.py @@ -434,7 +434,7 @@ def _task_instances_for_dag_run( try: for ti in dag_run.get_task_instances(session=session): if ti in schedulable_tis: - if not ti.state == TaskInstanceState.UP_FOR_RESCHEDULE: + if ti.state != TaskInstanceState.UP_FOR_RESCHEDULE: ti.try_number += 1 ti.set_state(TaskInstanceState.SCHEDULED) if ti.state != TaskInstanceState.REMOVED: diff --git a/airflow/models/dag.py b/airflow/models/dag.py index a6e5930a3e48c..c9a8424a831f0 100644 --- a/airflow/models/dag.py +++ b/airflow/models/dag.py @@ -2948,8 +2948,9 @@ def add_logger_if_needed(ti: TaskInstance): session.expire_all() schedulable_tis, _ = dr.update_state(session=session) for s in schedulable_tis: + if s.state != TaskInstanceState.UP_FOR_RESCHEDULE: + s.try_number += 1 s.state = TaskInstanceState.SCHEDULED - s.try_number += 1 session.commit() # triggerer may mark tasks scheduled so we read from DB all_tis = set(dr.get_task_instances(session=session))