Thanks to visit codestin.com
Credit goes to github.com

Skip to content

Conversation

Shekharrajak
Copy link

@Shekharrajak Shekharrajak commented Sep 1, 2025

Ref https://issues.apache.org/jira/browse/FLINK-38287

This implementation adds Kafka 4.x share group semantics to Flink's Kafka connector while maintaining full backward compatibility with existing code. The code changes are following KIP-932 and FLIP-27 main source architecture and implicit mode acknowledgement.

This directly addresses use cases where:

  1. Multiple consumers need to process items efficiently in parallel from a single/multiple topic(s).
  2. Messages need explicit acknowledgment/release (to avoid reprocessing or allow retries).
    Use cases where scaling Flink ML/LLM workload is critical - Shifting Kafka coordination and assignment logic to the broker side would simplify today’s complex Flink source management, making consumption more efficient, scalable, and far less error-prone.
    Operational Benefits
  • Higher Throughput: ShareGroupHeartbeat helps in Queue-like workloads, maximum throughput scenarios. Share groups distribute messages at the record level, not partition level, so multiple readers can consume from the same topic with Kafka coordinating message distribution.
  • Better Availability and Flexible Scaling: consumers assignment logic is simpler in server side and rebalancing frequency is minimised.

Let's have discussion over the design and how the checkpointing will work when we use KafkaShareConsumer API from Kafka 4.1

Screenshot 2025-09-02 at 10 59 55 AM Screenshot 2025-09-02 at 10 59 05 AM

Add comprehensive support for Kafka Share Groups with queue-like message
distribution semantics. This implementation provides an alternative to
traditional partition-based consumption by distributing messages at the
record level across multiple consumers.

Key features:
- KafkaShareGroupSource using KafkaShareConsumer API (Kafka 4.1.0+)
- Automatic message distribution without partition assignment
- Share group configuration validation and error handling
- Flink SQL table integration with 'kafka-sharegroup' connector
- Comprehensive metrics and monitoring support
- Proper split management adapted for share group semantics

Components added:
- KafkaShareGroupSource: Main source implementation
- KafkaShareGroupSourceReader: Share group-aware source reader
- KafkaShareConsumerSplitReader: Split reader using KafkaShareConsumer
- KafkaShareGroupFetcherManager: Custom fetcher for share groups
- KafkaShareGroupDynamicTableFactory: SQL integration
- KafkaShareGroupSourceMetrics: Metrics collection
- Comprehensive test suite

Configuration improvements:
- Remove incompatible properties (enable.auto.commit, auto.offset.reset)
- Add share group specific validation
- Enhanced version detection for Kafka 4.1.0+ features
- Proper split registration synchronization
This commit adds full support for Kafka Share Groups (KIP-932) in Flink,
enabling message-level consumption with parallelism beyond partition limits.

Key features:
- Topic-based splits instead of partition-based for share group semantics
- Multiple readers per topic with automatic message distribution
- Proper Flink FLIP-27 connector architecture integration
- Support for subtasks > partitions use cases
- Built-in metrics and state management

Core components added:
- KafkaShareGroupSplit: Topic-based split implementation
- KafkaShareGroupEnumerator: Assigns topics to multiple readers
- KafkaShareGroupSplitReader: Uses KafkaShareConsumer.subscribe()
- KafkaShareGroupRecordEmitter: Handles deserialization
- Updated fetcher manager and source reader for proper integration

Removed obsolete KafkaShareConsumerSplitReader in favor of new architecture.
- Update dependency scopes from 'provided' to 'compile' for standalone execution
- Add slf4j-simple logging dependency for debugging
- Upgrade Java target from 10 to 11 for better compatibility
- Update E2E test dependencies for share group testing
- Clean up .gitignore to properly exclude .idea directory
- Remove obsolete META-INF factory service file

These changes enable proper standalone testing and development of the
Kafka Share Group connector with source parallelism beyond partition limits.
- Enable explicit acknowledgment mode for precise record control
- Add checkpoint-based acknowledgment mechanism to prevent data loss
- Implement record release on checkpoint failures for redelivery
- Add proper error handling for InvalidRecordStateException
- Include memory leak prevention with pending record cleanup
- Add comprehensive logging for debugging acknowledgment flow
- Add checkpoint start notification to associate records with checkpoint ID
- Enhance checkpoint completion to trigger record acknowledgment
- Add checkpoint abortion handling to release records for redelivery
- Coordinate acknowledgment timing with Flink checkpoint lifecycle
- Bridge communication between source reader and split readers
…anager

- Add checkpoint start, complete, and abort notification methods
- Include comprehensive INFO-level logging for checkpoint tracking
- Establish coordination interface between source reader and split readers
- Provide foundation for future fetcher-level checkpoint coordination
- Add proper logger for fetcher manager operations
- Add recordSuccessfulCommit() method for tracking successful acknowledgments
- Add recordFailedCommit() method for tracking failed acknowledgments
- Include debug logging for commit operation visibility
- Complete metrics interface required by enhanced split reader
- Enable monitoring of acknowledgment success and failure rates
- Replace full record storage with memory-bounded caching approach
- Add configurable cache memory limit (flink.share.group.cache.max.memory.bytes)
- Implement record release under memory pressure to maintain at-least-once semantics
- Follow Pulsar connector pattern for lightweight acknowledgment metadata
- Add automatic cache cleanup and memory pressure handling
- Reduce memory usage from ~1KB per record to ~50 bytes per record
- Implement acknowledgment metadata storage following Pulsar connector pattern
- Add lightweight AcknowledgmentMetadata class for efficient checkpointing
- Update snapshotState to store metadata instead of full records
- Enhance notifyCheckpointComplete with metadata-based acknowledgment
- Add proper cleanup of acknowledgment metadata after checkpoint completion
- Maintain split state tracking for finished splits acknowledgment
- Add acknowledgment metadata tracking for Pulsar-style checkpointing
- Implement addPendingAcknowledgment method for offset tracking
- Add getLatestAcknowledgmentMetadata for checkpoint integration
- Include clearPendingAcknowledgments for cleanup after commit
- Add pending record count tracking for monitoring
- Maintain minimal state while supporting metadata-only approach
- Implement acknowledgeMessages method for metadata-only acknowledgment
- Add import for AcknowledgmentMetadata class
- Maintain compatibility with SourceReader acknowledgment pattern
- Support checkpoint notification infrastructure
- Log acknowledgment operations for debugging and monitoring
- Add ShareGroupBatchManager to control polling and store complete record batches in checkpoint state for crash recovery

Enhanced batch management, checkpoint snapshots, and recovery operations with structured logging

Enhance KafkaShareGroupSourceReader with structured checkpoint lifecycle logging
Enhanced fetch failures, acknowledgment errors, and checkpoint abort scenarios with detailed context
Copy link

boring-cyborg bot commented Sep 1, 2025

Thanks for opening this pull request! Please check out our contributing guidelines. (https://flink.apache.org/contributing/how-to-contribute.html)

<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<scope>provided</scope>
<scope>compile</scope>
Copy link
Author

Choose a reason for hiding this comment

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

These changes can be reverted.

* <p>This test validates builder functionality, error handling, and property management
* for Kafka share group source construction.
*/
@DisplayName("KafkaShareGroupSourceBuilder Tests")
Copy link
Author

Choose a reason for hiding this comment

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

Testcases improvements is required, I will check and update them accordingly.

<confluent.version>7.9.2</confluent.version>
<flink.version>2.0.0</flink.version>
<kafka.version>3.9.1</kafka.version>
<kafka.version>4.1.0</kafka.version>
Copy link
Author

Choose a reason for hiding this comment

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

This is not yet released and expected to be available to download. Meantime the testing is done by adding this into class path.

* Manages batches of records from Kafka share consumer for checkpoint persistence.
* Controls when new batches can be fetched to work within share consumer's auto-commit constraints.
*/
public class ShareGroupBatchManager<K, V>
Copy link
Author

Choose a reason for hiding this comment

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

ListCheckpointed will help to store the records which is polled but not yet processed in Flink persistent checkpoint state - this will make sure in case of any failure / crash we process the records that we read & ack.

@Shekharrajak Shekharrajak changed the title [WIP] Kafka 4.x Queue Semantics support in Flink Connector Kafka [WIP] Kafka 4.x Queue Semantics support Sep 1, 2025
@Shekharrajak
Copy link
Author

Using Flink SQL, we can have some validation :

CREATE TABLE kafka_share_source (
      message STRING
  ) WITH (
      'connector' = 'kafka-sharegroup',
      'bootstrap.servers' = 'localhost:9092',
      'share-group-id' = 'flink-sql-test-group',
      'topic' = 'test-topic',
      'format' = 'raw',
      'source.parallelism' = '4'  -- 4 subtasks regardless of partition count
  );
 
select * from kafka_share_source;

dafd0c9c-ec40-4af2-a929-ca27007076d8

<flink.version>2.1.0</flink.version>
<kafka.version>4.0.0</kafka.version>
<flink.version>2.0.0</flink.version>
<kafka.version>4.1.0</kafka.version>

Choose a reason for hiding this comment

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

PR 190 is also upping the Kafka client level to 4.1. If we do it here then we should amend the NOTOCE as per pr 190. fyi @tomncooper

@jnh5y
Copy link

jnh5y commented Sep 16, 2025

As a high-level note, since share groups do not use transactions, there will be some possibility for reprocessing messages. Is that ok for your use cases?

Generally, do you have any performance numbers to show that this consumer is faster? (Of course, since transactions are not available, I could imagine it being a little bit faster anyhow...)

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