Skip to content

KAFKA-20413: Add missing headers-aware ListValueStore for streams-streams join#22156

Open
aliehsaeedii wants to merge 5 commits intoapache:trunkfrom
aliehsaeedii:headers-aware-list-value-store
Open

KAFKA-20413: Add missing headers-aware ListValueStore for streams-streams join#22156
aliehsaeedii wants to merge 5 commits intoapache:trunkfrom
aliehsaeedii:headers-aware-list-value-store

Conversation

@aliehsaeedii
Copy link
Copy Markdown
Contributor

@aliehsaeedii aliehsaeedii commented Apr 27, 2026

Stream-stream outer joins lost per-record headers because
ListValueStore had no headers-aware variant (second part of
KAFKA-20413). This PR adds optional per-record headers preservation to
the stream-stream outer-join store, gated by dsl.store.format:

  • PLAIN (default): no on-disk change, no changelog change, no behavior
    change for existing apps.

  • HEADERS: each list element is serialized as a
    ValueTimestampHeaders<LeftOrRightValue<…>> blob via
    ValueTimestampHeadersSerde, so per-entry headers survive round-trip
    through the store, the changelog, and back into the join's null-side
    emit.

    Testing

    • New ListValueStoreWithHeadersTest (InMemory + RocksDB) -
      Parametrized the existing ListValueStoreTest
  • Updated KStreamKStreamOuterJoinTest.testShouldForwardCurrentHeaders

@github-actions github-actions Bot added triage PRs from the community streams labels Apr 27, 2026
// read it from the key (TimestampedKeyAndJoinSide#timestamp). Callers that need the
// headers gate on isHeadersStore() — the lifted headers slot here is unused on the plain
// path.
return new LiftingIterator<>(plainStore.all());
Copy link
Copy Markdown
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

adds up cost to default store users (but only in memory)!

@github-actions github-actions Bot removed the triage PRs from the community label Apr 28, 2026
protected abstract class KStreamKStreamJoinProcessor extends ContextualProcessor<K, VThis, K, VOut> {
private TimestampedWindowStoreWithHeaders<K, VOther> otherWindowStore;
private Sensor droppedRecordsSensor;
private Optional<KeyValueStore<TimestampedKeyAndJoinSide<K>, LeftOrRightValue<VLeft, VRight>>> outerJoinStore = Optional.empty();
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

IMO: Optional makes sense here.

Optional<OuterJoinStoreWrapper<K, VLeft, VRight>> WDYT?

Comment on lines 199 to 205
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

ditto
With previous version we was forced to provide store, so it was hard to end up with NPE
Now we implies that developer needs to check that store is non-null before calling this method

// values in the list.
// we do not use delete() calls since it would incur extra get()
store.put(prevKey, null);
outerJoinStoreWrapper.put(prevKey, null, null, 0L);
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldn't we propagate headers here? I case of plain store they will be ignored

ditto in other places

*/
@ParameterizedTest
@ValueSource(booleans = {false, true})
public void testShouldForwardPerRecordHeadersForMultipleBufferedOuterEmits(final boolean withHeaders) {
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 for test. Like it a lot

final InternalMockProcessorContext<Integer, ValueTimestampHeaders<String>> context = new InternalMockProcessorContext<>(
baseDir,
Serdes.Integer(),
null,
Copy link
Copy Markdown
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why null, not Integer?

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants