Uploaded image for project: 'Kafka'
  1. Kafka
  2. KAFKA-13636

Committed offsets could be deleted during a rebalance if a group did not commit for a while

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.4.0, 2.5.1, 2.6.2, 2.7.2, 2.8.1, 3.0.0
    • 3.0.1, 2.8.2, 3.2.0, 3.1.1
    • core, offset manager
    • None

    Description

      The group coordinator might delete invalid offsets during a group rebalance. During a rebalance, the coordinator is relying on the last commit timestamp (offsetAndMetadata.commitTimestamp) instead of the last state modification timestamp (currentStateTimestamp) to detect expired offsets.

       

      This is relatively easy to reproduce by playing with group.initial.rebalance.delay.ms, offset.retention.minutes and offset.check.retention.interval, I uploaded an example on: https://github.com/Dabz/kafka-example/tree/master/docker/offsets-retention .

      This script does:

      • Start a broker with: offset.retention.minute=2, offset.check.retention.interval.ms=1000,  group.initial.rebalance.delay=20000
      • Produced 10 messages
      • Create a consumer group to consume 10 messages, and disable auto.commit to only commit a few times
      • Wait 3 minutes, then the Consumer get a kill -9
      • Restart the consumer after a few seconds
      • The consumer restart from auto.offset.reset , the offset got removed

       

      The cause is due to the GroupMetadata.scala:

       

      This seem to be a regression generated by KIP-496 https://cwiki.apache.org/confluence/display/KAFKA/KIP-496%3A+Administrative+API+to+delete+consumer+offsets#KIP496:AdministrativeAPItodeleteconsumeroffsets-ProposedChanges (KAFKA-8338, KAFKA-8370)

      Attachments

        Issue Links

          Activity

            People

              princem Prince Mahajan
              Dabz Damien Gasparina
              David Jacot David Jacot
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: