Skip to content

KAFKA-18479: RocksDBTimeOrderedKeyValueBuffer not initialized correctly#18490

Merged
mjsax merged 2 commits intoapache:trunkfrom
mjsax:kafka-18479-stream-table-join-serde
Jan 15, 2025
Merged

KAFKA-18479: RocksDBTimeOrderedKeyValueBuffer not initialized correctly#18490
mjsax merged 2 commits intoapache:trunkfrom
mjsax:kafka-18479-stream-table-join-serde

Conversation

@mjsax
Copy link
Copy Markdown
Member

@mjsax mjsax commented Jan 11, 2025

RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides via Joined, but always uses serdes from StreamsConfig.

RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides
via Joined, but always uses serdes from StreamsConfig.
@mjsax mjsax added the streams label Jan 11, 2025
@github-actions github-actions Bot added the small Small PRs label Jan 11, 2025

final KeyValueIterator<Long, ValueAndTimestamp<String>> all = store.all();
final KeyValue<Long, ValueAndTimestamp<String>> onlyEntry = all.next();
try (final KeyValueIterator<Long, ValueAndTimestamp<String>> all = store.all()) {
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Side cleanup

final KTable<Long, String> rightTable = builder.table(INPUT_TOPIC_RIGHT, Consumed.with(Serdes.Long(), Serdes.String()), Materialized.as(
Stores.persistentVersionedKeyValueStore(STORE_NAME, Duration.ofMinutes(5))));
final Properties streamsConfig = setupConfigsAndUtils(cacheEnabled);
final Properties streamsConfig = setupConfigsAndUtils(cacheEnabled, false);
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

To test Joined we use false here

final KTable<Long, String> rightTable = builder.table(INPUT_TOPIC_RIGHT, Materialized.as(
Stores.persistentVersionedKeyValueStore(STORE_NAME, Duration.ofMinutes(5))));
final Properties streamsConfig = setupConfigsAndUtils(cacheEnabled);
final Properties streamsConfig = setupConfigsAndUtils(cacheEnabled, true);
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

For leftJoin we stick with using StreamsConfig serde to test both setups.

}

private void createBuffer(final Duration grace) {
private void createBuffer(final Duration grace, final Serde<String> serde) {
Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

"randomizing" this test, to cover both code path.

Copy link
Copy Markdown
Member

@bbejeck bbejeck left a comment

Choose a reason for hiding this comment

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

LGTM

@mjsax mjsax merged commit 30f94b5 into apache:trunk Jan 15, 2025
@mjsax mjsax deleted the kafka-18479-stream-table-join-serde branch January 15, 2025 18:04
mjsax added a commit that referenced this pull request Jan 15, 2025
…ly (#18490)

RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides
via Joined, but always uses serdes from StreamsConfig.

Reviewers: Bill Bejeck <bill@confluent.io>
mjsax added a commit that referenced this pull request Jan 15, 2025
…ly (#18490)

RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides
via Joined, but always uses serdes from StreamsConfig.

Reviewers: Bill Bejeck <bill@confluent.io>
mjsax added a commit to mjsax/kafka that referenced this pull request Jan 15, 2025
…ly (apache#18490)

RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides
via Joined, but always uses serdes from StreamsConfig.

Reviewers: Bill Bejeck <bill@confluent.io>
@mjsax
Copy link
Copy Markdown
Member Author

mjsax commented Jan 15, 2025

Merged to trunk and cherry-picked to 4.0 and 3.9 branches. 3.9 was already tricky (some conflicts) but 3.8 was even more tricky to cherry-pick, so I did not push directly but opened a PR: #18564

mjsax added a commit that referenced this pull request Jan 15, 2025
…ly (#18490) (#18564)

RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides
via Joined, but always uses serdes from StreamsConfig.

Reviewers: Bill Bejeck <bill@confluent.io>
@@ -62,18 +62,16 @@ public class RocksDBTimeOrderedKeyValueBufferTest {
@SuppressWarnings({"rawtypes", "unchecked"})
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

Sorry for the delayed feedback. To address the following warnings, could you please move these suppression annotations to the class-level?

> Task :streams:compileTestJava
Note: Some input files use or override a deprecated API.
Note: Recompile with -Xlint:deprecation for details.
Note: /home/chia7712/project/kafka/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBTimeOrderedKeyValueBufferTest.java uses unchecked or unsafe operations.
Note: Recompile with -Xlint:unchecked for details.

Copy link
Copy Markdown
Member Author

Choose a reason for hiding this comment

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

Nice catch! -- I think we should never use class level suppression though. Opened a PR to suppress for the individual methods: #18580

pranavt84 pushed a commit to pranavt84/kafka that referenced this pull request Jan 27, 2025
…ly (apache#18490)

RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides
via Joined, but always uses serdes from StreamsConfig.

Reviewers: Bill Bejeck <bill@confluent.io>
airlock-confluentinc Bot pushed a commit to confluentinc/kafka that referenced this pull request Jan 27, 2025
…ly (apache#18490)

RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides
via Joined, but always uses serdes from StreamsConfig.

Reviewers: Bill Bejeck <bill@confluent.io>
manoj-mathivanan pushed a commit to manoj-mathivanan/kafka that referenced this pull request Feb 19, 2025
…ly (apache#18490)

RocksDBTimeOrderedKeyValueBuffer is not initialize with serdes provides
via Joined, but always uses serdes from StreamsConfig.

Reviewers: Bill Bejeck <bill@confluent.io>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants