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

Adrian McCague edited comment on KAFKA-19775 at 11/21/25 3:48 PM:
------------------------------------------------------------------

Using 4.0.1 (CP 8.0.2) after upgrading from 3.X, this error is new. I wonder if 
this is a variant on the same issue?
{code}
   level: ERROR
   logger: org.apache.kafka.streams.processor.internals.TaskManager Error 
flushing caches of dirty task 0_0
   stack: java.lang.IllegalStateException: Stream task 0_0 does not know the 
partition: TOPIC-0
        at 
org.apache.kafka.streams.processor.internals.StreamTask.findOffsetAndMetadata(StreamTask.java:476)
        at 
org.apache.kafka.streams.processor.internals.StreamTask.committableOffsetsAndMetadata(StreamTask.java:507)
        at 
org.apache.kafka.streams.processor.internals.StreamTask.prepareCommit(StreamTask.java:450)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.closeTaskDirty(TaskManager.java:1420)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.closeAndCleanUpTasks(TaskManager.java:1538)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.lambda$shutdown$16(TaskManager.java:1459)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.executeAndMaybeSwallow(TaskManager.java:2120)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.shutdown(TaskManager.java:1457)
        at 
org.apache.kafka.streams.processor.internals.StreamThread.completeShutdown(StreamThread.java:1577)
        at 
org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:701)
}
{code}


was (Author: amccague):
Using 4.0.2 after upgrading from 3.X, this error is new. I wonder if this is a 
variant on the same issue?
{code}
   level: ERROR
   logger: org.apache.kafka.streams.processor.internals.TaskManager Error 
flushing caches of dirty task 0_0
   stack: java.lang.IllegalStateException: Stream task 0_0 does not know the 
partition: TOPIC-0
        at 
org.apache.kafka.streams.processor.internals.StreamTask.findOffsetAndMetadata(StreamTask.java:476)
        at 
org.apache.kafka.streams.processor.internals.StreamTask.committableOffsetsAndMetadata(StreamTask.java:507)
        at 
org.apache.kafka.streams.processor.internals.StreamTask.prepareCommit(StreamTask.java:450)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.closeTaskDirty(TaskManager.java:1420)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.closeAndCleanUpTasks(TaskManager.java:1538)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.lambda$shutdown$16(TaskManager.java:1459)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.executeAndMaybeSwallow(TaskManager.java:2120)
        at 
org.apache.kafka.streams.processor.internals.TaskManager.shutdown(TaskManager.java:1457)
        at 
org.apache.kafka.streams.processor.internals.StreamThread.completeShutdown(StreamThread.java:1577)
        at 
org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:701)
}
{code}

> Error if an empty topic is created when there is a regex source KS
> ------------------------------------------------------------------
>
>                 Key: KAFKA-19775
>                 URL: https://issues.apache.org/jira/browse/KAFKA-19775
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 4.0.0
>            Reporter: Nikita Shupletsov
>            Assignee: Nikita Shupletsov
>            Priority: Major
>             Fix For: 4.2.0, 4.0.2, 4.1.1
>
>
> If there is a KS application that uses a regex source, and we create a new 
> topic that matches that regex, but produce no messages, the application will 
> get into an {{ERROR}} state.
>  
> if we take 
> {}[RegexSourceIntegrationTest#testRegexRecordsAreProcessedAfterNewTopicCreatedWithMultipleSubtopologies|#L206{}}}],
>  but without producing any messages to {{TEST-TOPIC-2}} the problem will 
> reproduce:
> {quote}{{org.apache.kafka.streams.errors.StreamsException: 
> java.lang.IllegalStateException: Stream task 0_0 does not know the partition: 
> TEST-TOPIC-2-0}}
> {{    at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:981)}}
> {{    at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:898)}}
> {{Caused by: java.lang.IllegalStateException: Stream task 0_0 does not know 
> the partition: TEST-TOPIC-2-0}}
> {{    at 
> org.apache.kafka.streams.processor.internals.StreamTask.findOffsetAndMetadata(StreamTask.java:480)}}
> {{    at 
> org.apache.kafka.streams.processor.internals.StreamTask.committableOffsetsAndMetadata(StreamTask.java:511)}}
> {{    at 
> org.apache.kafka.streams.processor.internals.StreamTask.prepareCommit(StreamTask.java:454)}}
> {{    at 
> org.apache.kafka.streams.processor.internals.TaskExecutor.commitTasksAndMaybeUpdateCommittableOffsets(TaskExecutor.java:145)}}
> {{    at 
> org.apache.kafka.streams.processor.internals.TaskManager.commitTasksAndMaybeUpdateCommittableOffsets(TaskManager.java:2025)}}
> {{    at 
> org.apache.kafka.streams.processor.internals.TaskManager.commit(TaskManager.java:1992)}}
> {{    at 
> org.apache.kafka.streams.processor.internals.StreamThread.maybeCommit(StreamThread.java:1836)}}
> {{    at 
> org.apache.kafka.streams.processor.internals.StreamThread.runOnceWithoutProcessingThreads(StreamThread.java:1288)}}
> {{    at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:938)}}
> {{    ... 1 more}}
> {quote}
>  



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to