Skip to content

Conversation

@jhampson-dbre
Copy link

Problem

When committing consumer group offsets using protocol version 6+ (Kafka 2.1.0+), Sarama can commit a stale committed_leader_epoch that doesn't match the current partition leader epoch. This causes Kafka brokers to log "Truncation detected" errors and can lead to consumers receiving truncation errors when they resume consumption. The most likely condition that could cause this is using Sarama client to reset consumer group offsets for a consumer group that as be disconnected for an extended period and the current leader epoch of the partition has changed since the consumer group's last committed offset.

Root Cause

  1. During consumer group session initialization (sarama/consumer_group.go:823-872):
  • For each claimed partition, offsets.ManagePartition() is called
  • This creates a partitionOffsetManager via newPartitionOffsetManager().
  1. The partitionOffsetManager stores the leaderEpoch it receives during initialization from the OffsetFetchResponse.
  • Calls fetchInitialOffset() which performs an OffsetFetchRequest
  • Returns the previous committed offset's leader epoch 44 (from weeks ago)
  • This field is never updated after initialization

If a consumer group is disconnected for an extended period (e.g., weeks), and the partition leader changes multiple times during that period, the stored leaderEpoch becomes stale. When constructRequest() builds an OffsetCommitRequest, it uses the stale leaderEpoch from the partitionOffsetManager without checking if the partition leader has changed. For protocol version 6+, this stale epoch is included in the commit request, causing the broker to detect a mismatch when the consumer group connects and attempts to start consuming from the committed offsets.

Example Scenario

  1. Consumer group commits offset with leaderEpoch = 44
  2. Consumer group disconnects for several weeks
  3. Partition leader changes multiple times, current leader epoch is now 50
  4. Uses kafkactl (or similar tool) which uses sarama markOffset/resetOffset to reset consumer group offsets to the latest
  5. OffsetCommitRequest is sent with stale committed_leader_epoch = 44
  6. Consumer group connects and attempts to resume from the committed offsets
  7. Broker detects mismatch: offset was committed with epoch 44, but current leader is epoch 50
  8. Broker logs "Truncation detected" error indicating the commit is invalid.
  9. Consumer's auto.offset.reset behavior is triggered, causing consumer to re-read the entire partition if set to earliest.

Solution

This PR modifies constructRequest() in offset_manager.go to fetch the current partition leader epoch from metadata using client.LeaderAndEpoch() before constructing the OffsetCommitRequest for protocol version 6+ (Kafka 2.1.0+).

Changes

  1. offset_manager.go: In constructRequest(), when building the commit request for protocol version 6+, we now:

    • Call om.client.LeaderAndEpoch(pom.topic, pom.partition) to get the current leader epoch from metadata
    • Use the current epoch instead of the stale one stored in pom.leaderEpoch
    • Update pom.leaderEpoch to the current value for consistency
  2. offset_manager_test.go: Added TestConstructRequestUsesCurrentLeaderEpochUnit test case that:

    • Creates a partitionOffsetManager with a stale leader epoch (44)
    • Mocks the client to return a current leader epoch (50)
    • Verifies that constructRequest() uses the current epoch (50) in the commit request
    • Verifies that the internal state is updated to the current epoch

Implementation Details

The fix only applies to protocol version 6+ (Kafka 2.1.0+) because earlier versions don't include committed_leader_epoch in the OffsetCommitRequest. If LeaderAndEpoch() fails or returns an invalid epoch, we fall back to using the stored epoch, ensuring backward compatibility.

Testing

  • New unit test TestConstructRequestUsesCurrentLeaderEpochUnit passes
  • Test validates that stale leader epoch (44) is replaced with current epoch (50)
  • Test validates that internal state is updated correctly

Impact

  • Breaking Changes: None
  • Performance: one additional metadata lookup per partition per commit (only for Kafka 2.1.0+)
  • Backward Compatibility: Fully maintained - falls back to stored epoch if metadata lookup fails

@jhampson-dbre jhampson-dbre force-pushed the fix/stale-leader-epoch-offset-commit branch from 1bf38c0 to 6e79aaf Compare December 29, 2025 23:54
Copy link
Contributor

@puellanivis puellanivis left a comment

Choose a reason for hiding this comment

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

While I’ve proposed some improvement changes, I’m not sure this is at all an advisable course of action.

If a consumer group has been disconnected this long enough of a period, then it’s highly likely that it could have missed numerous messages, and we don’t at all want to accidentally “fast-forward” our leader epoch to whatever the partition leader says is the latest epoch?

I mean, we would want to restart consumption from the committed offsets and catch up to whatever the current leader epoch is by processing messages?

_, currentLeaderEpoch, err := om.client.LeaderAndEpoch(pom.topic, pom.partition)
if err == nil && currentLeaderEpoch >= 0 {
leaderEpoch = currentLeaderEpoch
pom.leaderEpoch = currentLeaderEpoch
Copy link
Contributor

Choose a reason for hiding this comment

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

I’m not sure why we need to keep a local-variable copy of leaderEpoch when we’re just going to assign the value into the struct anyways. With this change, we don’t even use the leaderEpoch until we pass it into AddBlockWithLeaderEpoch.

Removing lines 359, and 366, and reverting line 370 to what was the previous line 359 accomplishes the exact same behavior as the complete set of changes currently proposed.

type mockClientForLeaderEpoch struct {
Client
conf *Config
leaderAndEpochFn func(topic string, partitionID int32) (*Broker, int32, error)
Copy link
Contributor

Choose a reason for hiding this comment

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

I’m not clear about why we would need to provide a full arbitrary function here. For the purposes used, we could just have fields for epoch and err and then LeaderAndEpoch can just be return nil, m.epoch, m.err.

partition: 0,
offset: 100,
leaderEpoch: 44, // stale
metadata: "meta",
Copy link
Contributor

Choose a reason for hiding this comment

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

I’m unclear why all these fields are being set, are they actually necessary, or are they being filled in with unused dummy values? Nothing is actually testing this code, nor should anything fail to work properly if we didn’t set these values, so we should avoid setting them; as this indicates that the values are actually necessary for operation.

@jhampson-dbre
Copy link
Author

jhampson-dbre commented Dec 30, 2025

While I’ve proposed some improvement changes, I’m not sure this is at all an advisable course of action.

If a consumer group has been disconnected this long enough of a period, then it’s highly likely that it could have missed numerous messages, and we don’t at all want to accidentally “fast-forward” our leader epoch to whatever the partition leader says is the latest epoch?

I mean, we would want to restart consumption from the committed offsets and catch up to whatever the current leader epoch is by processing messages?

Thats a good point. I was misunderstanding the intent of the pom.dirty flag thinking that it was indicating an earlier or later offset was given (rather than the "next" offset). But it seems to be more of a "pending change" tracker to indicate an offset that has been "consumed" but not "committed" to the backend. So, the approach in the PR would not work since MarkMessage/MarkOffset is going to be called for sequential message processing and set the dirty flag under normal circumstances.

The problem arises when MarkOffset or ResetOffset is called for an offset that is likely relatively far in the future or past compared to the currently committed offset (i.e. a "fast-forward" or "rewind" scenario). In this case, partition offset manager makes no attempt to validate or otherwise check what the leader epoch of the offset was and continues to use pom.leaderEpoch for the currently committed offset and the committed_leader_epoch sent in the OffsetCommit Request can be invalid. When the consumer group tries to resume and fetch the offset, it will get LogTruncationException/OFFSET_OUT_OF_RANGE.

If you have any advice on a workable approach to solve this problem, I try to implement this another way. Or I can convert this into an issue if it would be helpful to discuss further?

@puellanivis
Copy link
Contributor

An issue would probably be a good idea. Myself, I’m not sure I fully understand what problem you’re facing, but won’t really be able to think on the topic until probably tomorrow. (Happy New Year.)

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

2 participants