NIFI-15614 - ConsumeKafka - Duplicate messages during consumer group rebalance#10908
NIFI-15614 - ConsumeKafka - Duplicate messages during consumer group rebalance#10908pvillard31 wants to merge 3 commits intoapache:mainfrom
Conversation
…rebalance Signed-off-by: Pierre Villard <pierre.villard.fr@gmail.com>
51c58f9 to
085137c
Compare
| System.out.println("Consumer 1 polled: " + consumer1Count.get() + " records"); | ||
| System.out.println("Consumer 2 polled: " + consumer2Count.get() + " records"); | ||
| System.out.println("Total unique messages: " + allConsumedMessages.size()); | ||
| System.out.println("Duplicate count: " + duplicateCount.get()); | ||
| System.out.println("Rebalance count: " + rebalanceCount.get()); |
There was a problem hiding this comment.
We should be using loggers here, not System.out.println
| revokedPartitions.size()); | ||
|
|
||
| try { | ||
| session.commit(); |
There was a problem hiding this comment.
We should avoid session.commit and instead ensure that we're using session.commitAsync with callbacks
There was a problem hiding this comment.
I'm not sure I agree with that comment. The synchronous commit is intentional and necessary. The whole point of this fix is that:
- The Kafka consumer is only in a valid state to commit offsets during
onPartitionsRevoked() - Once this callback returns, the consumer's group membership is revoked
- Any commit attempt after that fails with
RebalanceInProgressException
Using commitAsync() would defeat the purpose because the async callback would execute after onPartitionsRevoked() returns, no? At which point the Kafka consumer is no longer valid and we'd get the same duplicate message issue. IMO, the sequence must be:
onPartitionsRevoked()is called by Kafka- NiFi session commit (must complete before step 3)
- Kafka offset commit (must happen while still in valid callback)
onPartitionsRevoked()returns
Or did I miss something with your comment?
| private final ThreadLocal<ProcessSession> currentSession = new ThreadLocal<>(); | ||
| private final ThreadLocal<OffsetTracker> currentOffsetTracker = new ThreadLocal<>(); |
There was a problem hiding this comment.
We generally want to avoid ThreadLocal in processors because of the shared thread pool.
| } | ||
|
|
||
| private RebalanceCallback createRebalanceCallback() { | ||
| return revokedPartitions -> { |
There was a problem hiding this comment.
We should avoid any lambdas over 3-5 lines long in favor of anonymous inner classes.
| final ProcessSession session = currentSession.get(); | ||
| final OffsetTracker offsetTracker = currentOffsetTracker.get(); |
There was a problem hiding this comment.
Not sure that I understand the intention behind the ThreadLocal here. This should just be passed into the createRebalanceCallback method (piped through the getConsumerService method)
There was a problem hiding this comment.
I don't think we can just pass the session directly because the consumer service is created once and pooled, and the session is different for each onTrigger() call. I'll change for a holder approach that goes with your previous comment for ThreadLocal.
Summary
NIFI-15614 -
ConsumeKafka- Duplicate messages during consumer group rebalanceWhen using
ConsumeKafkawithKafka3ConnectionService, duplicate messages may be processed when a consumer group rebalance occurs.NIFI-15464 addressed a related issue by deferring offset commits during rebalance. The fix stored revoked partitions in
onPartitionsRevoked()and had the processor callcommitOffsetsForRevokedPartitions()after its session commit.The deferred commit approach is not enough because by the time
poll()returns and the processor attempts to commit, the consumer is no longer part of an active group. Kafka rejects the commit withRebalanceInProgressException, offsets are rolled back, and messages are re-consumed as duplicates.The Kafka consumer is only in a valid state to commit offsets during the
onPartitionsRevoked()callback. Once this callback returns, the consumer's group membership is revoked and any commit attempt will fail.We need to implement synchronous offset commit inside
onPartitionsRevoked()callback, similar to how NiFi 1.x handled rebalances inConsumerLease. This requires introducing a callback mechanism to ensure the NiFi session is committed before Kafka offsets are committed, preventing both data loss and duplicates.Tracking
Please complete the following tracking steps prior to pull request creation.
Issue Tracking
Pull Request Tracking
NIFI-00000NIFI-00000VerifiedstatusPull Request Formatting
mainbranchVerification
Please indicate the verification steps performed prior to pull request creation.
Build
./mvnw clean install -P contrib-checkLicensing
LICENSEandNOTICEfilesDocumentation