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

ERROR "SubscriptionInfo - unable to decode subscription data: version=2" when upgrading from 0.10.0.0 to 0.10.2.1

    XMLWordPrintableJSON

    Details

      Description

      KIP: https://cwiki.apache.org/confluence/display/KAFKA/KIP-268%3A+Simplify+Kafka+Streams+Rebalance+Metadata+Upgrade

      We upgraded an app from kafka-streams 0.10.0.0 to 0.10.2.1. We did a rolling upgrade of the app, so that one point, there were both 0.10.0.0-based instances and 0.10.2.1-based instances running.

      We observed the following stack trace:

      2017-10-11 07:02:19.964 [StreamThread-3] ERROR o.a.k.s.p.i.a.SubscriptionInfo -
      unable to decode subscription data: version=2
      org.apache.kafka.streams.errors.TaskAssignmentException: unable to decode
      subscription data: version=2
              at org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo.decode(SubscriptionInfo.java:113)
              at org.apache.kafka.streams.processor.internals.StreamPartitionAssignor.assign(StreamPartitionAssignor.java:235)
              at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.performAssignment(ConsumerCoordinator.java:260)
              at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.onJoinLeader(AbstractCoordinator.java:404)
              at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.access$900(AbstractCoordinator.java:81)
              at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:358)
              at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$JoinGroupResponseHandler.handle(AbstractCoordinator.java:340)
              at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:679)
              at org.apache.kafka.clients.consumer.internals.AbstractCoordinator$CoordinatorResponseHandler.onSuccess(AbstractCoordinator.java:658)
              at org.apache.kafka.clients.consumer.internals.RequestFuture$1.onSuccess(RequestFuture.java:167)
              at org.apache.kafka.clients.consumer.internals.RequestFuture.fireSuccess(RequestFuture.java:133)
              at org.apache.kafka.clients.consumer.internals.RequestFuture.complete(RequestFuture.java:107)
              at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient$RequestFutureCompletionHandler.onComplete(ConsumerNetworkClient.java:426)
              at org.apache.kafka.clients.NetworkClient.poll(NetworkClient.java:278)
              at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.clientPoll(ConsumerNetworkClient.java:360)
              at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:224)
              at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:192)
              at org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient.poll(ConsumerNetworkClient.java:163)
              at org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:243)
              at org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.ensurePartitionAssignment(ConsumerCoordinator.java:345)
              at org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:977)
              at org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:937)
              at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:295)
              at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:218)
              
      

      I spoke with Matthias J. Sax and he said this is a known issue that happens when you have both 0.10.0.0 instances and 0.10.2.1 instances running at the same time, because the internal version number of the protocol changed when adding Interactive Queries. Matthias asked me to file this JIRA>

        Attachments

          Issue Links

            Activity

              People

              • Assignee:
                mjsax Matthias J. Sax
                Reporter:
                wushujames James Cheng
              • Votes:
                0 Vote for this issue
                Watchers:
                3 Start watching this issue

                Dates

                • Created:
                  Updated:
                  Resolved: