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

NotEnoughReplicasException for __consumer_offsets topic due to out of order offset

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 2.6.2
    • None
    • core, log
    • None

    Description

      Hello,

      We faced a strange issue with Kafka during testing failover scenarios: this assumes forces shutdown nodes where Kafka pods are placed (Kafka is deployed to Kubernetes), and then return these nodes. 

      After this Kafka pods are started normally but some consumers could not connect to it with errors:

       

      [2022-01-27T14:35:09.051][level=DEBUG][class=kafka_client:utils.go:120]: Failed to sync group mae_processor: [15] Group Coordinator Not Available: the broker returns this error code for group coordinator requests, offset commits, and most group management requests if the offsets topic has not yet been created, or if the group coordinator is not active

       

       

      It looked like there were issues with __consumer_offsets topic. In logs of brokers we found this error:

      [2022-01-27T14:56:00,233][INFO][category=kafka.coordinator.group.GroupCoordinator] [GroupCoordinator 1]: Group mae_processor with generation 329 is now empty (__consumer_offsets-36)
      [2022-01-27T14:56:00,233][ERROR][category=kafka.server.ReplicaManager] [ReplicaManager broker=1] Error processing append operation on partition __consumer_offsets-36
      org.apache.kafka.common.errors.NotEnoughReplicasException: The size of the current ISR Set(1) is insufficient to satisfy the min.isr requirement of 2 for partition __consumer_offsets-36
      [2022-01-27T14:56:00,233][WARN][category=kafka.coordinator.group.GroupCoordinator] [GroupCoordinator 1]: Failed to write empty metadata for group mae_processor: The coordinator is not available.
      
      

      If we check partitions of __consumer_offsets it really has one partition with insufficient ISR:

      topic "__consumer_offsets" with 50 partitions:
          partition 0, leader 1, replicas: 1,3,2, isrs: 1,2,3
      ...
          partition 35, leader 3, replicas: 3,1,2, isrs: 1,2,3
          partition 36, leader 1, replicas: 1,3,2, isrs: 1
          partition 37, leader 2, replicas: 2,1,3, isrs: 1,2,3
      ....
          partition 49, leader 2, replicas: 2,1,3, isrs: 1,2,3

      We wait some time but the issue didn't go, we still had one partition with insufficient ISR.

      First of all we thought this is issue with Kafka-ZooKeeper coordinations, so we restarted ZooKeeper cluster and brokers 2 and 3, which didn't have ISR. But it didn't help.

      We also tried to manually ellect leader for this partition with kafka-leader-election.sh (in hope it will help). But it didn't help too.

      In logs we also found an issue:

      [2022-01-27T16:17:29,531][ERROR][category=kafka.server.ReplicaFetcherThread] [ReplicaFetcher replicaId=2, leaderId=1, fetcherId=0] Unexpected error occurred while processing data for partition __consumer_offsets-36 at offset 19536
      kafka.common.OffsetsOutOfOrderException: Out of order offsets found in append to __consumer_offsets-36: List(19536, 19536, 19537, 19538, 19539, 19540, 19541, 19542, 19543, 19544, 19545, 19546, 19547, 19548, 19549, 19550, 19551, 19552, 19553, 19554, 19555, 19556, 19557, 19558, 19559, 19560, 19561)
          at kafka.log.Log.$anonfun$append$2(Log.scala:1126)
          at kafka.log.Log.append(Log.scala:2349)
          at kafka.log.Log.appendAsFollower(Log.scala:1036)
          at 
      [2022-01-27T16:17:29,531][WARN][category=kafka.server.ReplicaFetcherThread] [ReplicaFetcher replicaId=2, leaderId=1, fetcherId=0] Partition __consumer_offsets-36 marked as failed
      

      This looks like root cause, right? Can force shutdown Kafka process lead to this issue?

      Looks like a bug, moreover, shall Kafka handle case of corrupting data (if it's the root cause of issue above)?

      Attachments

        Activity

          People

            Unassigned Unassigned
            mrMigles Sergey Ivanov
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated: