Skip to content

Kafka Connect: Tolerate CommitFailedException and InvalidProducerEpochException during rebalance#16366

Open
kumarpritam863 wants to merge 28 commits into
apache:mainfrom
kumarpritam863:feature/tolerate_commit_failed_and_producer_epoch_exp_during_rebalance
Open

Kafka Connect: Tolerate CommitFailedException and InvalidProducerEpochException during rebalance#16366
kumarpritam863 wants to merge 28 commits into
apache:mainfrom
kumarpritam863:feature/tolerate_commit_failed_and_producer_epoch_exp_during_rebalance

Conversation

@kumarpritam863
Copy link
Copy Markdown
Contributor

@kumarpritam863 kumarpritam863 commented May 16, 2026

Summary

When a Kafka consumer group re-balance happens between the time the iceberg-kafka-connect
sink task prepares a transactional offset commit and the time the broker processes it, the
connector currently dies with an unrecoverable ConnectException:

org.apache.kafka.connect.errors.ConnectException: Exiting WorkerSinkTask due to unrecoverable exception.
...
Caused by: org.apache.kafka.clients.consumer.CommitFailedException: Transaction offset Commit
failed due to consumer group metadata mismatch: Specified group generation id is not valid.
at org.apache.kafka.clients.producer.internals.TransactionManager$TxnOffsetCommitHandler.handleResponse(...)

This is a transient, expected failure mode under re-balance — the consumer-group
generation id captured in producer.sendOffsetsToTransaction(...) becomes stale by the time
the broker validates it. The same applies to InvalidProducerEpochException when the
producer epoch is bumped mid-flight. Both should be recoverable, not fatal.

Root cause

In Channel.send() the flow is:

producer.beginTransaction();
recordList.forEach(producer::send);
producer.sendOffsetsToTransaction(offsetsToCommit, KafkaUtils.consumerGroupMetadata(context));
producer.commitTransaction();   // <- raises CommitFailedException on stale generation id

The previous catch block aborted the transaction and rethrew the exception unchanged, so it
propagated as a non-retriable failure into WorkerSinkTask.deliverMessages and killed the
task.

Fix

Channel.send() now distinguishes recoverable re-balance failures from fatal ones:

  • Recoverable (CommitFailedException, InvalidProducerEpochException, including when
    wrapped in another KafkaException): abort the transaction and translate to
    org.apache.kafka.connect.errors.RetriableException. Connect's framework then pauses the
    consumer, retains the message batch, and re-delivers it after the re-balance settles. The
    aborted transaction never advanced source offsets, so once the partitions are reassigned
    the new owner resumes from the last broker-committed offset — no data loss. Any data files
    flushed before the abort become orphans, recoverable by Iceberg's orphan-file expiration.

CommitterImpl.processControlEvents adds a RetriableException log line at info level so
the re-balance recovery is visible in task logs but doesn't surface as an error.

KafkaUtils.seekToLastCommittedOffsets(SinkTaskContext) is added and called from
IcebergSinkTask.close(partitions) in addition to open(partitions). This is required
under incremental cooperative re-balance, where Connect can invoke close() on a
revoked partition without a paired open() — meaning the framework's own rewind in
onPartitionsAssigned never runs for it. Seeking the main consumer here guarantees that
records read past the broker-committed offset (and never committed transactionally) are
re-fetched on the next poll.

Guarantees preserved

  • Source offsets are still committed atomically with the control-topic DataWritten events
    via the producer transaction — unchanged.
  • No data loss: aborted transactions never advance source offsets, so re-delivered or
    re-fetched records flow through the next successful commit.
  • No double-commit to Iceberg: the Coordinator's per-snapshot offset properties
    (kafka.connect.offsets..) still gate replays on Coordinator restart.

Test plan

New ChannelTest covers:

  • First-attempt success — no retry, no abort
  • Empty source offsets path — sendOffsetsToTransaction is skipped
  • CommitFailedException → translated to RetriableException with cause preserved
  • InvalidProducerEpochException → translated to RetriableException
  • sendOffsetsToTransaction failure (not commitTransaction) is also translated
  • CommitFailedException wrapped in another KafkaException is detected via cause chain
  • Non-re-balance KafkaException is rethrown as-is
  • beginTransaction failure is rethrown without spurious abortTransaction
  • abortTransaction failure is swallowed and does not mask the RetriableException

Existing WorkerTest, CoordinatorTest, CommitterImplTest, IcebergSinkTaskTest and the
rest of the kafka-connect suite pass unchanged.

@kumarpritam863 kumarpritam863 changed the title Feature/tolerate commit failed and producer epoch exp during rebalance Kafka Connect: Tolerate CommitFailedException and InvalidProducerEpochException during rebalance May 16, 2026
@kumarpritam863
Copy link
Copy Markdown
Contributor Author

@danielcweeks can you please review this.

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.

1 participant