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

BufferUnderflowException during Kafka Streams Upgrade

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Blocker
    • Resolution: Fixed
    • 2.5.0
    • 2.6.0, 2.4.2, 2.5.1
    • streams

    Description

      I migrated a Kafka Streams application from version 2.3.1 to 2.5.0. I followed the steps described in the upgrade guide and set the property migrate.from=2.3. On my dev system with just one running instance I got the following exception:

      stream-thread [0-StreamThread-2] Encountered the following error during processing:
      java.nio.BufferUnderflowException: null
      	at java.base/java.nio.HeapByteBuffer.get(Unknown Source)
      	at java.base/java.nio.ByteBuffer.get(Unknown Source)
      	at org.apache.kafka.streams.state.internals.BufferValue.extractValue(BufferValue.java:94)
      	at org.apache.kafka.streams.state.internals.BufferValue.deserialize(BufferValue.java:83)
      	at org.apache.kafka.streams.state.internals.InMemoryTimeOrderedKeyValueBuffer.restoreBatch(InMemoryTimeOrderedKeyValueBuffer.java:368)
      	at org.apache.kafka.streams.processor.internals.CompositeRestoreListener.restoreBatch(CompositeRestoreListener.java:89)
      	at org.apache.kafka.streams.processor.internals.StateRestorer.restore(StateRestorer.java:92)
      	at org.apache.kafka.streams.processor.internals.StoreChangelogReader.processNext(StoreChangelogReader.java:350)
      	at org.apache.kafka.streams.processor.internals.StoreChangelogReader.restore(StoreChangelogReader.java:94)
      	at org.apache.kafka.streams.processor.internals.TaskManager.updateNewAndRestoringTasks(TaskManager.java:401)
      	at org.apache.kafka.streams.processor.internals.StreamThread.runOnce(StreamThread.java:779)
      	at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:697)
      	at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:670)
      

      I figured out, that this problem only occurs for stores, where I use the suppress feature. If I rename the changelog topics during the migration, the problem will not occur. 

      Attachments

        Issue Links

          Activity

            People

              vvcephei John Roesler
              karsten.schnitter Karsten Schnitter
              Votes:
              0 Vote for this issue
              Watchers:
              8 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: