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

Retrying leaderEpoch request for partition xxx as the leader reported an error: UNKNOWN_SERVER_ERROR

    XMLWordPrintableJSON

Details

    • New Feature
    • Status: Open
    • Major
    • Resolution: Unresolved
    • 1.0.0
    • None
    • core
    • None
    • Ubuntu 16.04 LTS 64bit-server

    Description

      we have 3 brokers in a kafka cluster(brokerid:401,402,403). The broker-403 fails to fetch data from leader:

      [2018-02-02 08:58:26,861] INFO [ReplicaFetcher replicaId=403, leaderId=401, fetcherId=0] Retrying leaderEpoch request for partition sub_payone1hour-0 as the leader reported an error: UNKNOWN_SERVER_ERROR (kafka.server.ReplicaFetcherThread)
      [2018-02-02 08:58:26,865] WARN [ReplicaFetcher replicaId=403, leaderId=401, fetcherId=3] Error when sending leader epoch request for Map(sub_myshardSinfo-3 -> -1, sub_myshardUinfo-1 -> -1, sub_videoOnlineResourceType8Test-0 -> -1, pub_videoReportEevent-1 -> 9, sub_StreamNofity-3 -> -1, pub_RsVideoInfo-1 -> -1, pub_lidaTopic3-15 -> -1, pub_lidaTopic3-3 -> -1, sub_zwbtest-1 -> -1, sub_svAdminTagging-5 -> -1, pub_channelinfoupdate-1 -> -1, pub_RsPlayInfo-4 -> -1, sub_tinyVideoWatch-4 -> 14, __consumer_offsets-36 -> -1, pub_ybusAuditorChannel3-2 -> -1, pub_vipPush-4 -> -1, sub_LivingNotifyOnline-3 -> -1, sub_baseonline-4 -> -1, __consumer_offsets-24 -> -1, sub_lidaTopic-3 -> -1, sub_mobileGuessGameReward-0 -> -1, pub_lidaTopic-6 -> -1, sub_NewUserAlgo-0 -> -1, __consumer_offsets-48 -> -1, pub_RsUserBehavior-3 -> -1, sub_channelinfoupdate-0 -> -1, pub_tinyVideoComment-1 -> -1, pub_bulletin-2 -> -1, pub_RecordCompleteNotifition-6 -> -1, sub_lidaTopic2-3 -> -1, smsgateway-10 -> -1, __consumer_offsets-0 -> -1, pub_baseonlinetest-1 -> -1, __consumer_offsets-12 -> -1, pub_myshardUinfo-0 -> -1, pub_baseonline-3 -> -1, smsGatewayMarketDbInfo-6 -> -1, sub_tinyVideoComment-0 -> 14) (kafka.server.ReplicaFetcherThread)
      java.io.IOException: Connection to 401 was disconnected before the response was read
       at org.apache.kafka.clients.NetworkClientUtils.sendAndReceive(NetworkClientUtils.java:95)
       at kafka.server.ReplicaFetcherBlockingSend.sendRequest(ReplicaFetcherBlockingSend.scala:96)
       at kafka.server.ReplicaFetcherThread.fetchEpochsFromLeader(ReplicaFetcherThread.scala:312)
       at kafka.server.AbstractFetcherThread.maybeTruncate(AbstractFetcherThread.scala:130)
       at kafka.server.AbstractFetcherThread.doWork(AbstractFetcherThread.scala:102)
       at kafka.utils.ShutdownableThread.run(ShutdownableThread.scala:64)

       

      on the leader(broker-401) side, the log shows:

      [2018-02-02 08:58:26,859] ERROR Closing socket for 192.168.100.101:9099-192.168.100.103:30476 because of error (kafka.network.Processor)
      org.apache.kafka.common.errors.InvalidRequestException: Error getting request for apiKey: 23 and apiVersion: 0
      Caused by: java.lang.IllegalArgumentException: Unexpected ApiKeys id `23`, it should be between `0` and `20` (inclusive)
       at org.apache.kafka.common.protocol.ApiKeys.forId(ApiKeys.java:73)
       at org.apache.kafka.common.requests.AbstractRequest.getRequest(AbstractRequest.java:39)
       at kafka.network.RequestChannel$Request.liftedTree2$1(RequestChannel.scala:96)
       at kafka.network.RequestChannel$Request.<init>(RequestChannel.scala:91)
       at kafka.network.Processor$$anonfun$processCompletedReceives$1.apply(SocketServer.scala:492)
       at kafka.network.Processor$$anonfun$processCompletedReceives$1.apply(SocketServer.scala:487)
       at scala.collection.Iterator$class.foreach(Iterator.scala:893)
       at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
       at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
       at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
       at kafka.network.Processor.processCompletedReceives(SocketServer.scala:487)
       at kafka.network.Processor.run(SocketServer.scala:417)
       at java.lang.Thread.run(Thread.java:745)

      Attachments

        Activity

          People

            Unassigned Unassigned
            bookxiao Wang Shuxiao
            Votes:
            1 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated: