feat: [Phase 1 - Draft] Queue Semantics support in Kafka Ingestion#19311
feat: [Phase 1 - Draft] Queue Semantics support in Kafka Ingestion#19311Shekharrajak wants to merge 49 commits into
Conversation
| cd apache-druid-31.0.0 | ||
|
|
||
| # Replace the kafka extension with our build | ||
| rm extensions/druid-kafka-indexing-service/*.jar |
There was a problem hiding this comment.
We can build from the source and use the tar
| agg -> agg.hasSumAtLeast(numRecords) | ||
| ); | ||
|
|
||
| Assertions.assertEquals( |
There was a problem hiding this comment.
Making sure we read all the records.
d31199b to
4a30bee
Compare
|
Spending time on Integeration testing embedded-tests/src/test/java/org/apache/druid/testing/embedded/indexing/EmbeddedShareGroupIngestionTest.java |
|
For smaller blast radius and clean separation - parking the suggestion #18439 (comment) , Also KafkaShareGroupSupervisor (planned) will be structurally different from KafkaSupervisor — no partition-to-task assignment, no offset bookkeeping, seek/rest . |
| } | ||
|
|
||
| @Test | ||
| public void test_shareGroupIngestion_basicEndToEnd() throws InterruptedException |
There was a problem hiding this comment.
Subscribe to topic with KafkaShareConsumer (groupId), poll → parse (CSV) → persist → publish → acknowledge(ACCEPT) → commitSync → broker offset persistence. SQL COUNT(*) == 10.
| final int numRecords = 5; | ||
| final int rowsPerRecord = 4; | ||
| kafkaServer.produceRecordsToTopic( | ||
| generateMultiObjectJsonRecords(topic, numRecords, rowsPerRecord, DateTimes.of("2025-07-01")) |
There was a problem hiding this comment.
StreamChunkReader multi-row path One Kafka record carrying 4 whitespace-separated JSON objects produces 4 rows. SQL count = numRecords × rowsPerRecord (5 × 4 = 20). Verifies no row is dropped before ACK.
|
To make sure no data loss we are ack only after the publish completed https://github.com/apache/druid/pull/19311/changes#diff-487176908fb14d87486db6f683c9ddb34465f3a1572f5c01b11d7b9c6a482289R381 Although this will be slow and no concurrency but I think for the intial phase this is fine and we can improve the performance step by step. WDYT @gianm @a2l007 @clintropolis @FrankChen021 ? |
|
CI checks are failing due to one flaky test unrelated to this PR : #19435 |
| recordMap.put(new TopicPartition("test-topic", 0), recordsP0); | ||
| recordMap.put(new TopicPartition("test-topic", 1), recordsP1); | ||
| Mockito.when(mockConsumer.poll(Mockito.any(Duration.class))) | ||
| .thenReturn(new ConsumerRecords<>(recordMap)); |
| final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> recordMap = new HashMap<>(); | ||
| recordMap.put(new TopicPartition(record.topic(), record.partition()), List.of(record)); | ||
| Mockito.when(mockConsumer.poll(Mockito.any(Duration.class))) | ||
| .thenReturn(new ConsumerRecords<>(recordMap)); |
| */ | ||
| Map<PartitionIdType, Optional<Exception>> commitSync(); | ||
|
|
||
| Set<PartitionIdType> getPartitionIds(String stream); |
234a8ec to
e5894ef
Compare
FrankChen021
left a comment
There was a problem hiding this comment.
| Severity | Findings |
|---|---|
| P0 | 0 |
| P1 | 0 |
| P2 | 2 |
| P3 | 0 |
| Total | 2 |
Reviewed 22 of 22 changed files. Findings focus on task scheduling semantics and Kafka compatibility test coverage.
This is an automated review by Codex GPT-5
| <artifactId>maven-surefire-plugin</artifactId> | ||
| <configuration> | ||
| <excludes> | ||
| <exclude>**/KafkaIndexTaskTest.java</exclude> |
There was a problem hiding this comment.
[P2] Kafka compatibility tests are skipped globally
This Surefire exclusion removes the main KafkaIndexTask, KafkaRecordSupplier, KafkaSamplerSpec, and KafkaSupervisor test classes from every normal test run. Since this PR also upgrades kafka-clients to 4.2.0, those are the regression tests most likely to catch compatibility breaks in existing index_kafka ingestion; the new share-group mock and embedded tests do not cover that existing API surface. Please migrate or narrowly isolate the TestBroker-dependent cases instead of excluding the whole classes.
There was a problem hiding this comment.
Yes, planning to fix (upgrade to 4.2 version in all places) as part of this #19322 - I feel that we need bunch of changes, API compatibility issues in running tests , kafka docker image for IT tests and in source code - so want to keep it in separate PR.
This PR mainly focus on Share group implementation.
|
Struggling with CI check related to license : |
e3a9ad7 to
7865104
Compare
FrankChen021
left a comment
There was a problem hiding this comment.
Reviewed 3 of 27 changed files. Follow-up review handled: the Kafka extension POM no longer has the global Surefire exclusions, and ShareGroupIndexTask now uses DEFAULT_REALTIME_TASK_PRIORITY with tests covering the default and context override cases.
This is an automated review by Codex GPT-5.5
…r queue-semantics ingestion
…re group ingestion
…plicit acknowledgement
ede5072 to
e159405
Compare
e159405 to
1a8bed2
Compare
FrankChen021
left a comment
There was a problem hiding this comment.
| Severity | Findings |
|---|---|
| P0 | 0 |
| P1 | 0 |
| P2 | 2 |
| P3 | 0 |
| Total | 2 |
Reviewed 21 of 21 changed files.
This is an automated review by Codex GPT-5.5
… share-group test
| final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> recordMap = new HashMap<>(); | ||
| recordMap.put(new TopicPartition("test-topic", 0), List.of(record1)); | ||
| recordMap.put(new TopicPartition("test-topic", 1), List.of(record2)); | ||
| final ConsumerRecords<byte[], byte[]> consumerRecords = new ConsumerRecords<>(recordMap); |
| ); | ||
| final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> recordMap = new HashMap<>(); | ||
| recordMap.put(new TopicPartition("test-topic", 0), List.of(record)); | ||
| when(mockConsumer.poll(any(Duration.class))).thenReturn(new ConsumerRecords<>(recordMap)); |
…OrderedPartitionableRecord data
FrankChen021
left a comment
There was a problem hiding this comment.
Follow-up handled. Reviewed 21 of 21 changed files. The demo now configures share.auto.offset.reset=earliest before producing sample records, and share-group consumer properties are sanitized again after dynamic config expansion. I do not have further inline replies on the two follow-up threads.
This is an automated review by Codex GPT-5.5
|
CI Check error - unrelated timeouts : Raised ticket #19463 |
|
Another flaky test: #19464 |
Phase 1 #18439.
Description
Fixed the bug ...
Renamed the class ...
Added a forbidden-apis entry ...
Release note
Key changed/added classes in this PR
MyFooOurBarTheirBazThis PR has:
Demo
https://youtu.be/K_O1MH-AaE8