[feature] Add Kafka microbatch ingestion plugin#17688
[feature] Add Kafka microbatch ingestion plugin#17688udaysagar2177 wants to merge 1 commit intoapache:masterfrom
Conversation
|
Should we consider enabling a consumer to read directly from the upstream object store? This could eliminate the requirement here for users to provision Kafka. |
|
@noob-se7en Thanks for the feedback. The current Kafka-based approach remains useful for users who prefer to avoid the operational complexity of streaming commits into Apache Iceberg. Regarding reading directly from the upstream object store: clarification on the intended mechanism would help. How would Apache Pinot discover new files and track which ones have already been processed? One possible direction is using Iceberg tables, which provide snapshots, manifests with file metadata, and incremental scans for new files. The existing microbatch framework would remain largely unchanged - file discovery from Iceberg snapshots instead of Kafka, while MicroBatchQueueManager continues handling download and processing. I posted a short PEP describing this Iceberg-based approach here: #17694. Could you clarify whether your suggestion refers to a different model? |
Codecov Report✅ All modified and coverable lines are covered by tests.
Additional details and impacted files@@ Coverage Diff @@
## master #17688 +/- ##
============================================
- Coverage 63.25% 55.64% -7.62%
+ Complexity 1499 721 -778
============================================
Files 3174 2479 -695
Lines 190373 140388 -49985
Branches 29089 22366 -6723
============================================
- Hits 120419 78116 -42303
+ Misses 60610 55685 -4925
+ Partials 9344 6587 -2757
Flags with carried forward coverage won't be shown. Click here to find out more. ☔ View full report in Codecov by Sentry. 🚀 New features to boost your workflow:
|
This PR introduces a new streaming ingestion pattern for Apache Pinot - MicroBatch ingestion from Kafka. Instead of processing individual records from Kafka messages, this consumer reads Kafka messages containing JSON protocol references to batch files stored in PinotFS (S3, HDFS, GCS, Azure, etc.) or inline base64-encoded data. See #17331 for motivation.
Design Overview
Wire Protocol (Version 1):
JSON Payload Examples:
Key Components:
KafkaMicroBatchConsumerFactory- Factory that creates MicroBatch consumersMicroBatchProtocol/MicroBatchPayloadV1- Protocol parsing and validationMicroBatchQueueManager- Orchestrates parallel file downloads and batch conversionMicroBatchStreamPartitionMsgOffset- Composite offset (Kafka offset + record offset within batch) enabling mid-batch resume after segment commitsMessageBatchReader- Converts downloaded files toMessageBatch<GenericRow>using Pinot's RecordReaderSupported Formats: AVRO, Parquet, JSON
Configuration
{ "streamConfigs": { "streamType": "kafka", "stream.kafka.topic.name": "microbatch-topic", "stream.kafka.broker.list": "localhost:9092", "stream.kafka.consumer.factory.class.name": "org.apache.pinot.plugin.stream.microbatch.kafka30.KafkaMicroBatchConsumerFactory", "stream.microbatch.kafka.file.fetch.threads": "2" } }See
CONFIGURATION.mdfor complete documentation including PinotFS setup for S3/HDFS/GCS/Azure.Testing
MicroBatchProtocolTest(30+ test cases),MicroBatchStreamPartitionMsgOffsetFactoryTest,MicroBatchStreamMetadataProviderTest,MicroBatchQueueManagerTestKafkaPartitionLevelMicroBatchConsumerTest- Tests with real embedded Kafka, including mid-batch resume scenariosMicroBatchRealtimeClusterIntegrationTest- End-to-end test with full Pinot clusterCompatibility Notes
{"kmo":X,"mbro":Y}) is specific to MicroBatch consumers and not compatible with standard Kafka consumer offsetssupportsOffsetLag()returns false since we cannot accurately estimate lag without knowing record counts in unconsumed batches. We can have it return batch counts if needed.Open Questions for Reviewers
I’ve added TODOs in the code to highlight open questions for reviewers, please take a look.
Checklist
CONFIGURATION.md)