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

Replication Broken between Kafka 0.8.2.1 and 0.9 - NetworkClient.java uses wrong protocol version

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 0.9.0.0
    • 0.9.0.0
    • None
    • None

    Description

      During a rolling upgrade from 0.8.2.1 to 0.9.0.0, replication between 0.9.0.0 and 0.8.2.1 fails due to
      org.apache.kafka.clients.networkClient:handleCompletedReceives always using the latest version of the API Key available instead of the one specified by inter.broker.protocol.version.

      This line should not use ProtoUtils.currentResponseSchema and instead call ProtoUtils.ResponseSchema and specify a version explicitly:

      Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload());
      

      This results in WARN messages like the following in the server.log file as the responses are decoded with the wrong Schema:

      [2015-11-05 19:13:10,309] WARN [ReplicaFetcherThread-0-182050600], Error in fetch kafka.server.ReplicaFetcherThread$FetchRequest@6cc18858. Possible cause: org.apache.kafka.common.protocol.types.SchemaException: Error reading field 'responses': Error reading field 'topic': java.nio.BufferUnderflowException (kafka.server.ReplicaFetcherThread)
      
      [2015-11-03 16:55:15,178] WARN [ReplicaFetcherThread-1-182050600], Error in fetch kafka.server.ReplicaFetcherThread$FetchRequest@224388b2. Possible cause: org.apache.kafka.common.protocol.types.SchemaException: Error reading field 'responses': Error reading field 'partition_responses': Error reading field 'record_set': java.lang.IllegalArgumentException (kafka.server.ReplicaFetcherThread)
      

      Attachments

        1. KAFKA-2756.patch
          1 kB
          Matthew Bruce

        Issue Links

          Activity

            People

              mbruce@blackberry.com Matthew Bruce
              mbruce@blackberry.com Matthew Bruce
              Votes:
              0 Vote for this issue
              Watchers:
              4 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: