Skip to content

KAFKA-20462: Preserve original source-record headers in ErrorHandlerContext#22155

Open
1fanwang wants to merge 1 commit intoapache:trunkfrom
1fanwang:KAFKA-20462
Open

KAFKA-20462: Preserve original source-record headers in ErrorHandlerContext#22155
1fanwang wants to merge 1 commit intoapache:trunkfrom
1fanwang:KAFKA-20462

Conversation

@1fanwang
Copy link
Copy Markdown

@1fanwang 1fanwang commented Apr 27, 2026

Problem

ErrorHandlerContext#headers() is documented to expose "the headers of the current source record". Both error paths in Streams violate that contract today: a user-supplied Deserializer is allowed to mutate the live Headers reference (e.g. a LargeMessageDeserializer that removes its serde-internal marker header so it doesn't leak downstream), and the same mutable reference flows into the DefaultErrorHandlerContext that the DeserializationExceptionHandler and ProcessingExceptionHandler see. ExceptionHandlerUtils#buildDeadLetterQueueRecord then copies the post-mutation headers into the DLQ record, so a DLQ record produced from a mutating-Deserializer pipeline can no longer be deserialized by the same Deserializer on a replay (its marker header is gone).

This complements #21370 (which made buildDeadLetterQueueRecord copy context.headers() into the DLQ record): the DLQ now copies something, but per KAFKA-20462 that something must be the original source-record headers, which is what this PR provides.

Evidence

  • Reproducer (deserialization-error path): a Deserializer that calls headers.remove("source-only") and then throws causes ErrorHandlerContext#headers() in the DeserializationExceptionHandler to return headers without "source-only". New test RecordDeserializerTest#shouldExposeOriginalSourceRecordHeadersToErrorHandlerWhenDeserializerMutatesHeaders is verified to fail before this fix and pass after.
  • Reproducer (processing-error path): a ProcessorRecordContext where the live headers were mutated to empty but sourceRawHeaders carries the original "source-only" header. With the live headers fed through to the error context, ErrorHandlerContext#headers() returns the empty (mutated) view; with sourceRawHeaders fed through, it returns the original. New test ProcessorNodeTest#shouldExposeSourceRawHeadersToProcessingExceptionHandlerWhenDeserializerMutatedHeaders covers this.

Fix

Single snapshot per record, taken in RecordQueue#updateHead (and the analogous spots in GlobalStateUpdateTask / GlobalStateManagerImpl) before the user-supplied Deserializer runs. The snapshot is propagated along two paths:

  1. As an explicit parameter to RecordDeserializer#deserialize so that, when deserialization throws, RecordDeserializer#handleDeserializationFailure builds DefaultErrorHandlerContext with the snapshot rather than the (possibly mutated) live reference. (This is the path my earlier commit on this branch addressed.)
  2. On a new transient sourceRawHeaders field on StampedRecord and ProcessorRecordContext, mirroring the existing sourceRawKey / sourceRawValue lifecycle: not serialized, cleared by freeRawRecord(). ProcessorNode reads the snapshot from the ProcessorRecordContext (cast through a small helper, so the public RecordContext interface is unchanged — no KIP) when constructing the DefaultErrorHandlerContext for the ProcessingExceptionHandler.

The downstream Record passed to processors still carries the live (possibly mutated) Headers reference, so the LargeMessageDeserializer pattern — Deserializers that legitimately mutate headers so processors don't see serde-internal data — continues to work unchanged. Only ErrorHandlerContext#headers() is changed to honor its documented contract.

Cost: one RecordHeaders copy (small ArrayList<Header> clone, typically 0–3 entries) per record on the deserialization path. Negligible relative to the existing per-record allocations in RecordQueue / RecordDeserializer.

Compatibility

  • Public API: unchanged. No additions to RecordContext or ErrorHandlerContext. sourceRawHeaders lives only on the internal ProcessorRecordContext and is read via a type-narrowing cast inside ProcessorNode. No KIP needed.
  • Wire format: unchanged. sourceRawHeaders is transient (like sourceRawKey / sourceRawValue), not serialized into ProcessorRecordContext#serialize() and not surfaced in changelog topics or state-store buffers.
  • User-visible behavior: ErrorHandlerContext#headers() now returns the original source-record headers (as documented) rather than the post-mutation reference. Users whose handlers depended on the buggy post-mutation behavior will see the unmutated headers instead.

Tests

  • New RecordDeserializerTest#shouldExposeOriginalSourceRecordHeadersToErrorHandlerWhenDeserializerMutatesHeaders — DeserializationExceptionHandler path.
  • New ProcessorNodeTest#shouldExposeSourceRawHeadersToProcessingExceptionHandlerWhenDeserializerMutatedHeaders — ProcessingExceptionHandler path.
  • Both verified to fail on trunk and pass after this PR.
  • Existing RecordDeserializerTest, ProcessorNodeTest, StreamTaskTest, RecordQueueTest, GlobalStateTaskTest, GlobalStateManagerImplTest, ProcessingExceptionHandlerIntegrationTest, ExceptionHandlerUtilsTest all pass.
  • :streams:checkstyleMain :streams:checkstyleTest :streams:spotbugsMain all green.

JIRA: https://issues.apache.org/jira/browse/KAFKA-20462

@github-actions github-actions Bot added triage PRs from the community streams labels Apr 27, 2026
…ontext

A user-supplied Deserializer is allowed to mutate the live Headers
reference (e.g. a LargeMessageDeserializer that strips serde-internal
headers so they don't leak downstream). The same Headers reference
flows into ErrorHandlerContext used by both the
DeserializationExceptionHandler and the ProcessingExceptionHandler,
so when these handlers read context.headers() -- and when
ExceptionHandlerUtils#buildDeadLetterQueueRecord copies those headers
into a DLQ record -- they observe the post-mutation state, even
though the javadoc on ErrorHandlerContext#headers() promises "the
headers of the current source record".

Snapshot rawRecord.headers() in RecordQueue#updateHead before
invoking RecordDeserializer#deserialize, and propagate the snapshot
through two paths:

  - As an explicit parameter to RecordDeserializer#deserialize so the
    DeserializationExceptionHandler error path uses the original
    source headers when the Deserializer threw.
  - On a new transient sourceRawHeaders field on StampedRecord and
    ProcessorRecordContext (mirroring the existing
    sourceRawKey/sourceRawValue lifecycle: not serialized, cleared
    by freeRawRecord), so that ProcessorNode -- when invoking the
    ProcessingExceptionHandler after a downstream processor failed
    on a successfully-deserialized record -- builds the
    DefaultErrorHandlerContext from the snapshot rather than the
    live (possibly mutated) Headers reference.

GlobalStateUpdateTask and GlobalStateManagerImpl get the same
treatment for the global-state restore path (deserialization-failure
and processing-failure respectively).

The downstream Record passed to processors still carries the live
(possibly mutated) Headers reference, so processors that legitimately
rely on a Deserializer mutating headers (the LargeMessageDeserializer
pattern) continue to see the mutated view -- only the error-handler
context is changed to honor its documented contract.

Two regression tests:

  - RecordDeserializerTest covers the
    DeserializationExceptionHandler path: a Deserializer that mutates
    headers then throws.
  - ProcessorNodeTest covers the ProcessingExceptionHandler path: a
    ProcessorRecordContext where the live headers have already been
    mutated but sourceRawHeaders preserves the originals, and a
    processor throws.
@1fanwang 1fanwang changed the title KAFKA-20462: Preserve original source-record headers in DeserializationExceptionHandler KAFKA-20462: Preserve original source-record headers in ErrorHandlerContext Apr 27, 2026
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

streams triage PRs from the community

Projects

None yet

Development

Successfully merging this pull request may close these issues.

1 participant