kaxil commented on code in PR #54547:
URL: https://github.com/apache/airflow/pull/54547#discussion_r2879817054
##########
airflow-core/src/airflow/utils/log/file_task_handler.py:
##########
@@ -644,7 +644,9 @@ def _read(
if ti.state in (TaskInstanceState.RUNNING, TaskInstanceState.DEFERRED)
and not has_k8s_exec_pod:
sources, served_logs = self._read_from_logs_server(ti,
worker_log_rel_path)
source_list.extend(sources)
- elif ti.state not in State.unfinished and not (local_logs or
remote_logs):
+ elif (ti.state not in State.unfinished or ti.state ==
TaskInstanceState.UP_FOR_RETRY) and not (
+ local_logs or remote_logs
Review Comment:
nit: This condition pattern (`not in set or == specific_member`) is a bit
fragile — if `State.unfinished` is modified in the future, this carve-out is
easy to miss. Consider defining a small frozenset at module level:
```python
_STATES_WITH_COMPLETED_ATTEMPT = frozenset({
TaskInstanceState.UP_FOR_RETRY,
TaskInstanceState.UP_FOR_RESCHEDULE,
})
```
Then:
```python
elif (ti.state not in State.unfinished or ti.state in
_STATES_WITH_COMPLETED_ATTEMPT) and not (
local_logs or remote_logs
):
```
This makes the intent self-documenting and easier to maintain.
##########
airflow-core/src/airflow/utils/log/file_task_handler.py:
##########
@@ -644,7 +644,9 @@ def _read(
if ti.state in (TaskInstanceState.RUNNING, TaskInstanceState.DEFERRED)
and not has_k8s_exec_pod:
sources, served_logs = self._read_from_logs_server(ti,
worker_log_rel_path)
source_list.extend(sources)
Review Comment:
`UP_FOR_RESCHEDULE` has the same bug. It is also in `State.unfinished` (see
`state.py` line 201), and a sensor task in `UP_FOR_RESCHEDULE` has completed
its current poke attempt — its logs should be viewable via the served-logs
fallback just like `UP_FOR_RETRY`.
```suggestion
elif (ti.state not in State.unfinished or ti.state in
(TaskInstanceState.UP_FOR_RETRY, TaskInstanceState.UP_FOR_RESCHEDULE)) and not (
local_logs or remote_logs
):
```
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]