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

Loosing messages on OFFSET_OUT_OF_RANGE error in consumer

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 0.11.0.0
    • None
    • clients
    • None

    Description

      Steps to reproduce:

      • Setup test:
        • producer sends messages constantly. If cluster not available, then it will retry
        • consumer polling
        • topic has 3 partitions and replication factor 3.
        • min.insync.replicas=2
        • producer has "acks=all"
        • consumer has default "auto.offset.reset=latest"
        • consumer manually commitSync offsets after handling messages.
        • unclean leader election = false
        • kafka cluster has 3 brokers
      • Kill broker 0
      • In consumer's logs:
        2017-11-08 11:36:33,967 INFO  org.apache.kafka.clients.consumer.internals.Fetcher           - Fetch offset 10706682 is out of range for partition mytopic-2, resetting offset [kafka-consumer]
        2017-11-08 11:36:33,968 INFO  org.apache.kafka.clients.consumer.internals.Fetcher           - Fetch offset 8024431 is out of range for partition mytopic-1, resetting offset [kafka-consumer]
        2017-11-08 11:36:34,045 INFO  org.apache.kafka.clients.consumer.internals.Fetcher           - Fetch offset 8029505 is out of range for partition mytopic-0, resetting offset [kafka-consumer]
        

      After that, consumer lost several messages on each partition.

      Expected:

      • return upper bound of range
      • consumer should resume from that offset instead of "auto.offset.reset".

      Workaround:

      • put "auto.offset.reset=earliest"
      • get a lot of duplicate messages, instead of lost

      Looks like this is what happening during the recovery from broker failure (see attachment)

      Attachments

        Issue Links

          Activity

            People

              Unassigned Unassigned
              dernasherbrezon Andrey
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

              Dates

                Created:
                Updated: