kaxil opened a new pull request, #67652:
URL: https://github.com/apache/airflow/pull/67652

   When a container running under `KubernetesPodOperator` writes an empty line, 
kubelet streams it back (with `timestamps=True`) as `"<rfc3339-ts> \n"` -- a 
timestamp followed by a separator space and an empty message. `parse_log_line` 
in `pod_manager.py` called `line.strip().partition(" ")`, which removed the 
trailing separator space before partitioning, so the function returned 
`timestamp=None` and the caller treated the line as a continuation of the 
previous buffered log record. The bare RFC3339 string was then appended onto 
the previous message and emitted as a multi-line log where only the first line 
carried the Airflow `[ts] {pod_manager.py:N} INFO -` prefix, leaving unprefixed 
timestamp rows interleaved in task logs:
   
   ```
   [2026-05-28T13:07:50.160+0000] {pod_manager.py:520} INFO - [base] 
private_alphas_ttan==0.0.2.dev409
   2026-05-28T13:07:57.030578889Z 
   2026-05-28T13:07:57.030581518Z
   2026-05-28T13:07:57.030642740Z 
   [2026-05-28T13:07:57.034+0000] {pod_manager.py:520} INFO - [base] Running 
calculate for entity-v1
   ```
   
   Downstream that breaks 
[`airflow.utils.log.file_task_handler._parse_timestamp`](https://github.com/apache/airflow/blob/main/airflow-core/src/airflow/utils/log/file_task_handler.py#L201-L203),
 which feeds the line to `pendulum.parse` after stripping `[]`: malformed 
fragments from these orphan rows can raise `ValueError: month must be in 1..12` 
and fail the task entirely.
   
   Closes #36571.
   
   ## Root cause and history
   
   Regressed in [#33675](https://github.com/apache/airflow/pull/33675) (merged 
2023-08-24, shipped in cncf-kubernetes **7.5.0**) which replaced the original 
`line.find(\" \")` split with a `line.strip().partition(\" \")` pattern under 
the banner of a refactor:
   
   ```diff
   - split_at = line.find(\" \")
   - if split_at == -1:
   -     ...
   - timestamp = line[:split_at]
   - message = line[split_at + 1 :].rstrip()
   + timestamp, sep, message = line.strip().partition(\" \")
   + if not sep:
   +     ...
   ```
   
   The pre-refactor implementation correctly handled `<ts> \n` because `find(\" 
\")` matched the separator space directly and the message-side `.rstrip()` 
produced an empty string. The new code strips the separator off before 
partitioning, so the function loses its only signal that the line is 
well-formed.
   
   This matches the regression window the original reporter described in 
[#36571](https://github.com/apache/airflow/issues/36571): the bug appeared 
after upgrading cncf-kubernetes from 7.4.2 (pre-refactor) to 7.12.0+ 
(post-refactor) and is still reproducible on current `main` (10.17.x).
   
   ## Fix
   
   * `parse_log_line` no longer pre-strips the line; it `rstrip(\"\\n\")` only 
and partitions on the original separator, so empty container writes are 
recognised as `(timestamp, \"\")` rather than as continuations. If the 
partition yields no separator the whole line is tried as a bare timestamp (some 
kubelet versions emit `<ts>\\n` with no trailing space), and parse failures 
fall through to the original return-the-raw-line path. It also catches 
`ValueError`, not just `ParserError`, so a malformed timestamp can never escape 
into Airflow's downstream parsers.
   * The sync (`PodManager.fetch_container_logs.consume_logs`) and async 
(`AsyncPodManager.fetch_container_logs_before_current_sec`) log consumer loops 
skip emit for empty messages -- the resume marker still advances in the sync 
path so reconnect-since-time stays correct, but no noisy `[base] ` row is 
written.
   
   ## Tests
   
   * Parametrized `test_parse_log_line_handles_empty_container_writes` covers 
`<ts> \\n`, `<ts>\\n`, and `<ts> ` (no newline). Verified RED on `main`, GREEN 
with the fix.
   * End-to-end `test_empty_container_lines_do_not_pollute_previous_message` 
drives `fetch_container_logs` with the exact log sequence from the issue and 
asserts no orphan timestamps land in `caplog`. Also RED on `main`, GREEN with 
the fix.
   
   ## Gotchas
   
   * Truly empty container output (just `\\n`) is no longer surfaced as a 
`[base]` row. That output carries no information for the task log reader and 
was previously the trigger for downstream pendulum failures, so dropping it is 
a net improvement; if a future use case needs to count blank container lines, 
that's separable work.


-- 
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]

Reply via email to