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

SubscriptionWrapper is of an incompatible version (Kafka Streams 3.2.3 -> 3.3.2)

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 3.3.0, 3.3.1, 3.3.2
    • 3.4.0, 3.3.3
    • streams
    • None
    • Kafka Streams 3.2.3 (before update)
      Kafka Streams 3.3.2 (after update)
      Java 17 (Eclipse Temurin 17.0.5), Linux x86_64

    Description

      Hey folks,
       
      we've just updated an application from Kafka Streams 3.2.3 to 3.3.2 and started getting the following exceptions:

      org.apache.kafka.common.errors.UnsupportedVersionException: SubscriptionWrapper is of an incompatible version. 

      After swiftly looking through the code, this exception is potentially thrown in two places:

       
      Is it possible that the second check in SubscriptionStoreReceiveProcessorSupplier<K, KO> was forgotten?
       
      Any hints how to resolve this issue without a downgrade?
      Since this only affects 2 of 15 topologies in the application, I'm hesitant to just downgrade to Kafka 3.2.3 again since the internal topics might already have been updated to use the "new" version of SubscriptionWrapper.
       
      Related discussion in the Confluent Community Slack: https://confluentcommunity.slack.com/archives/C48AHTCUQ/p1674497054507119

      Stack trace

      org.apache.kafka.streams.errors.StreamsException: Exception caught in process. taskId=1_8, processor=XXX-joined-changed-fk-subscription-registration-source, topic=topic.rev7-XXX-joined-changed-fk-subscription-registration-topic, partition=8, offset=12297976, stacktrace=org.apache.kafka.common.errors.UnsupportedVersionException: SubscriptionWrapper is of an incompatible version.
      
          at org.apache.kafka.streams.processor.internals.StreamTask.process(StreamTask.java:750)
          at org.apache.kafka.streams.processor.internals.TaskExecutor.processTask(TaskExecutor.java:100)
          at org.apache.kafka.streams.processor.internals.TaskExecutor.process(TaskExecutor.java:81)
          at org.apache.kafka.streams.processor.internals.TaskManager.process(TaskManager.java:1182)
          at org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:770)
          at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:588)
          at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:550) 

      Attachments

        Activity

          People

            mjsax Matthias J. Sax
            joschi Jochen Schalanda
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: