[ 
https://issues.apache.org/jira/browse/KAFKA-10173?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17140863#comment-17140863
 ] 

John Roesler commented on KAFKA-10173:
--------------------------------------

Hi [~karsten.schnitter] ,

I've had a chance now to look into this. It's actually quite strange, although 
I'm probably missing something.

That stacktrace means that Streams is actually having trouble deserializing 
data written with changelog schema version 2, which IIRC was introduced in 
version 2.4.0. If you only ran the application with version 2.3.1, stopped it, 
then started it with version 2.5.0, it shouldn't be possible to hit this 
branch, as it means reading data that 2.3.1 couldn't possibly have written.

The only thing I can imagine is if you had previously run Streams with a 
version _less than_ 2.3.0 AND if some of your data happened to contain a header 
with the name "v", then it could lead to this kind of corruption. But, in that 
case, you should have gotten an exception while running with version 2.3.1 
already. So, this doesn't sound like it fits.

The only thing that seems to fit your report is that the 
serialization/deserialization logic is actually wrong. But it's quite heavily 
tested, and has been exercised in production for quite a while also. There is 
always the possibility of a rare edge case, though. I've just given the logic 
another pass, and I'm not seeing what could be the problem. Is there any chance 
you could find the changelog record that caused this exception and just send me 
the bytes for it?

Short of that, I'm not sure how to proceed.

-John

P.s., while reviewing the code, I did find one flaw, which I've submitted 
[https://github.com/apache/kafka/pull/8905] for. I doubt that was your problem, 
but we might as well fix it anyway.

> BufferUnderflowException during Kafka Streams Upgrade
> -----------------------------------------------------
>
>                 Key: KAFKA-10173
>                 URL: https://issues.apache.org/jira/browse/KAFKA-10173
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 2.5.0
>            Reporter: Karsten Schnitter
>            Assignee: John Roesler
>            Priority: Blocker
>             Fix For: 2.5.1
>
>
> 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:
> {noformat}
> 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)
> {noformat}
> 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. 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

Reply via email to