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

Chris Toomey commented on KAFKA-9141:
-------------------------------------

Here's the log message and stack trace. It comes from [this 
line|https://github.com/apache/kafka/blob/2.4/streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java#L248]
 in {{GlobalStreamThread}}.

{code:java}
2019-10-31 17:12:15.894 - - ERROR o.a.k.s.p.i.GlobalStreamThread$StateConsumer 
service-platformAPI-revokedJwtsCache-91814a54-b18b-47a8-9d35-926e6abcb5f6-GlobalStreamThread
 - global-stream-thread 
[service-platformAPI-revokedJwtsCache-91814a54-b18b-47a8-9d35-926e6abcb5f6-GlobalStreamThread]
 Updating global state failed. You can restart KafkaStreams to recover from 
this error.
org.apache.kafka.clients.consumer.OffsetOutOfRangeException: Offsets out of 
range with no configured reset policy for partitions: 
{livongo.public.entity.system.auth.jwt.revoked-0=15579}
        at 
org.apache.kafka.clients.consumer.internals.Fetcher.initializePartitionRecords(Fetcher.java:1266)
        at 
org.apache.kafka.clients.consumer.internals.Fetcher.fetchedRecords(Fetcher.java:605)
        at 
org.apache.kafka.clients.consumer.KafkaConsumer.pollForFetches(KafkaConsumer.java:1283)
        at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1214)
        at 
org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:1190)
        at 
org.apache.kafka.streams.processor.internals.GlobalStreamThread$StateConsumer.pollAndUpdate(GlobalStreamThread.java:239)
        at 
org.apache.kafka.streams.processor.internals.GlobalStreamThread.run(GlobalStreamThread.java:290)
{code}

The context in which this occurred was switching the application to point to a 
different kafka broker in a development environment, which caused the offset 
problem. Several of us on my team got this error and took the message "You can 
restart KafkaStreams to recover from this error" to mean "restarting 
KafkaStreams will fix this error", which it doesn't, hence this ticket.

How could an application detect and self-recover by calling {{cleanUp()}} from 
this, given that it happens asynchronously on the global stream thread?

> Global state update error: missing recovery or wrong log message
> ----------------------------------------------------------------
>
>                 Key: KAFKA-9141
>                 URL: https://issues.apache.org/jira/browse/KAFKA-9141
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 2.3.0
>            Reporter: Chris Toomey
>            Priority: Major
>
> I'm getting an {{OffsetOutOfRangeException}} accompanied by the log message 
> "Updating global state failed. You can restart KafkaStreams to recover from 
> this error." But I've restarted the app several times and it's not 
> recovering, it keeps failing the same way.
>  
> I see there's a {{cleanUp()}} method on {{KafkaStreams}} that looks like it's 
> what's needed, but it's not called anywhere in the streams source code. So 
> either that's a bug and the call should be added to do the recovery, or the 
> log message is wrong and should be fixed.



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

Reply via email to