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

AdminClient fails to describe consumer group

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Open
    • Critical
    • Resolution: Unresolved
    • 2.8.2, 3.6.1
    • None
    • admin, clients, consumer, log
    • None

    Description

      Dear Team,

      We have been investigating some quirky behavior around admin client.  Here is our conclusion:

      • Due to some bug (or a feature not known by us) AdminClient (both 2.8 and 3.6) fails to describe one of the consumer groups (with no known problems about it)
      • Pure GoLang admin client does not have the problem (github.com/twmb/franz-go) and able to describe the consumer group.

      We tried to understand what may cause the issue, first of all, the Java client 2.8 reported,

      kafka-consumer-groups --bootstrap-server broker:9092 --describe --group 'problematic-consumer'
      Error: Executing consumer group command failed due to org.apache.kafka.common.errors.LeaderNotAvailableException: There is no leader for this topic-partition as we are in the middle of a leadership election.
      java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.LeaderNotAvailableException: There is no leader for this topic-partition as we are in the middle of a leadership election.

      we waited if this is a transient error but it turned out it is not, there was no election for the given topic

      But it was not clear which topic admin client was talking about so TRACE log revealed some more information:

      [2023-12-18 10:36:38,434] DEBUG [AdminClient clientId=adminclient-1] Sending LIST_OFFSETS request with header RequestHeader(apiKey=LIST_OFFSETS, apiVersion=6, clientId=adminclient-1, correlationId=30) and timeout 4997 to node 40: ListOffsetsRequestData(replicaId=-1, isolationLevel=0, topics=[ListOffsetsTopic(name='problematic-topic', partitions=[ListOffsetsPartition(partitionIndex=4, currentLeaderEpoch=-1, timestamp=-1, maxNumOffsets=1), ListOffsetsPartition(partitionIndex=5, currentLeaderEpoch=-1, timestamp=-1, maxNumOffsets=1)])]) (org.apache.kafka.clients.NetworkClient)
      [2023-12-18 10:36:38,434] TRACE [AdminClient clientId=adminclient-1] Entering KafkaClient#poll(timeout=4997) (org.apache.kafka.clients.admin.KafkaAdminClient)
      [2023-12-18 10:36:38,435] TRACE [AdminClient clientId=adminclient-1] KafkaClient#poll retrieved 0 response(s) (org.apache.kafka.clients.admin.KafkaAdminClient)
      [2023-12-18 10:36:38,435] TRACE [AdminClient clientId=adminclient-1] Trying to choose nodes for [] at 1702884998435 (org.apache.kafka.clients.admin.KafkaAdminClient)
      [2023-12-18 10:36:38,435] TRACE [AdminClient clientId=adminclient-1] Entering KafkaClient#poll(timeout=4995) (org.apache.kafka.clients.admin.KafkaAdminClient)

      Error: Executing consumer group command failed due to org.apache.kafka.common.errors.LeaderNotAvailableException: There is no leader for this topic-partition as we are in the middle of a leadership election.
      [2023-12-18 10:36:38,436] DEBUG [AdminClient clientId=adminclient-1] Received LIST_OFFSETS response from node 40 for request with header RequestHeader(apiKey=LIST_OFFSETS, apiVersion=6, clientId=adminclient-1, correlationId=30): ListOffsetsResponseData(throttleTimeMs=0, topics=[ListOffsetsTopicResponse(name='problematic-topic', partitions=[ListOffsetsPartitionResponse(partitionIndex=5, errorCode=0, oldStyleOffsets=[], timestamp=-1, offset=822516, leaderEpoch=113, followerRestorePointObjectId=AAAAAAAAAAAAAAAAAAAAAA, followerRestorePointEpoch=0), ListOffsetsPartitionResponse(partitionIndex=4, errorCode=0, oldStyleOffsets=[], timestamp=-1, offset=827297, leaderEpoch=93, followerRestorePointObjectId=AAAAAAAAAAAAAAAAAAAAAA, followerRestorePointEpoch=0)])]) (org.apache.kafka.clients.NetworkClient)
      [2023-12-18 10:36:38,436] TRACE [AdminClient clientId=adminclient-1] KafkaClient#poll retrieved 1 response(s) (org.apache.kafka.clients.admin.KafkaAdminClient)
      [2023-12-18 10:36:38,437] TRACE [AdminClient clientId=adminclient-1] Call(callName=listOffsets on broker 40, deadlineMs=1702885003430, tries=0, nextAllowedTryMs=0) got response ListOffsetsResponseData(throttleTimeMs=0, topics=[ListOffsetsTopicResponse(name='problematic-topic', partitions=[ListOffsetsPartitionResponse(partitionIndex=5, errorCode=0, oldStyleOffsets=[], timestamp=-1, offset=822516, leaderEpoch=113, followerRestorePointObjectId=AAAAAAAAAAAAAAAAAAAAAA, followerRestorePointEpoch=0), ListOffsetsPartitionResponse(partitionIndex=4, errorCode=0, oldStyleOffsets=[], timestamp=-1, offset=827297, leaderEpoch=93, followerRestorePointObjectId=AAAAAAAAAAAAAAAAAAAAAA, followerRestorePointEpoch=0)])]) (org.apache.kafka.clients.admin.KafkaAdminClient)
      [2023-12-18 10:36:38,437] TRACE [AdminClient clientId=adminclient-1] Trying to choose nodes for [] at 1702884998436 (org.apache.kafka.clients.admin.KafkaAdminClient)
      [2023-12-18 10:36:38,437] TRACE [AdminClient clientId=adminclient-1] Entering KafkaClient#poll(timeout=299161) (org.apache.kafka.clients.admin.KafkaAdminClient)
      java.util.concurrent.ExecutionException: org.apache.kafka.common.errors.LeaderNotAvailableException: There is no leader for this topic-partition as we are in the middle of a leadership election.

      adminclient version 3.6 is not returning this error, but it fails with a timeout after retrying is exhausted.
      We have also took a look into "problematic-topic", reassigned replicas to other brokers, ran kafka-leader-election over all partitions, did not help

      Attachments

        Activity

          People

            Unassigned Unassigned
            omersiar Ömer Şiar Baysal
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated: