Skip to content

Commit

Permalink
Displaying "actual" try number in TaskInstance view (#34635)
Browse files Browse the repository at this point in the history
(cherry picked from commit ce6ac41)
  • Loading branch information
SamWheating authored and jedcunningham committed Feb 9, 2024
1 parent a2c9a0a commit 0fc68e0
Show file tree
Hide file tree
Showing 2 changed files with 14 additions and 4 deletions.
12 changes: 12 additions & 0 deletions airflow/models/taskinstance.py
Original file line number Diff line number Diff line change
Expand Up @@ -1421,6 +1421,18 @@ def try_number(self):
"""
return _get_try_number(task_instance=self)

@try_number.expression
def try_number(cls):
"""
This is what will 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:
"""
Expand Down
6 changes: 2 additions & 4 deletions airflow/www/views.py
Original file line number Diff line number Diff line change
Expand Up @@ -5483,7 +5483,7 @@ class TaskInstanceModelView(AirflowModelView):
"priority_weight",
"queue",
"queued_dttm",
"try_number",
"prev_attempted_tries",
"pool",
"queued_by_job_id",
"external_executor_id",
Expand Down Expand Up @@ -5512,9 +5512,7 @@ class TaskInstanceModelView(AirflowModelView):
"queued_by_job_id",
]

label_columns = {
"dag_run.execution_date": "Logical Date",
}
label_columns = {"dag_run.execution_date": "Logical Date", "prev_attempted_tries": "Try Number"}

search_columns = [
"state",
Expand Down

0 comments on commit 0fc68e0

Please sign in to comment.