dajac opened a new pull request, #22444: URL: https://github.com/apache/kafka/pull/22444
When a `__consumer_offsets` partition transitions to follower, its local log is truncated and re-replicated from the new leader. The group coordinator hosting the partition remains active until it is unloaded asynchronously. During that window, the partition's high watermark advances again over records that this coordinator did not write, while the coordinator still holds in-memory state (and pending deferred operations) for its own records that were truncated and never durably committed. Applying such a high watermark has two consequences. It can violate the invariants of the snapshot registry and fail the `HighWatermarkUpdate` event, logging a spurious error such as "Execution of HighWatermarkUpdate failed due to New committed offset X of __consumer_offsets-N must be less than or equal to Y". More importantly, when it does not fail, it advances the committed offset over the coordinator's uncommitted state and completes the corresponding deferred writes with a success response, even though those records were lost. A client can therefore receive a successful offset-commit acknowledgment for a commit that is silently dropped once the new coordinator takes over. This patch gates high watermark propagation in `CoordinatorPartitionWriter.ListenerAdapter` on the partition's leadership. The adapter stops forwarding high watermark updates once the partition transitions to follower, is deleted, or fails. The partition signals these transitions (via `PartitionListener`) before its fetcher is restarted (see `ReplicaManager#applyDelta`), i.e. before any such high watermark can be produced, so the coordinator never observes a high watermark that it should not apply. The pending deferred operations then remain in place and are failed with `NOT_COORDINATOR` when the coordinator is unloaded, so clients correctly retry against the new coordinator. Gating on leadership rather than inspecting the offset is deliberate: after truncation an offset can still have a snapshot in the registry while holding the new leader's data, so no offset-based check can tell whether a high watermark is safe to apply. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
