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

Guozhang Wang commented on KAFKA-4919:
--------------------------------------

[~elevy] There is a PR open for this ticket but not yet linked to it 
automatically: https://github.com/apache/kafka/pull/2725

Please feel free to share your opinions on that PR if you want.

> Document that stores must not be closed when Processors are closed
> ------------------------------------------------------------------
>
>                 Key: KAFKA-4919
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4919
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 0.10.2.0
>            Reporter: Elias Levy
>
> I have a streams job, that previously worked, that consumes and writes to a 
> large number of topics with many partitions and that uses many threads.  I 
> upgraded the job to 0.10.2.0.  The job now fails after a short time running, 
> seemingly after a rebalance.
> {quote}
> WARN  2017-03-19 18:03:20,432 [StreamThread-10][StreamThread.java:160] : 
> Unexpected state transition from RUNNING to NOT_RUNNING
> {quote}
> The first observation is that Streams is no longer outputting exceptions and 
> backtraces.  I had to add code to get this information.
> The exception:
> {quote}
> Exception: org.apache.kafka.streams.errors.StreamsException: Exception caught 
> in process. taskId=1_225, processor=KSTREAM-SOURCE-0000000003, 
> topic=some_topic, partition=225, offset=266411
> org.apache.kafka.streams.errors.StreamsException: Exception caught in 
> process. taskId=1_225, processor=KSTREAM-SOURCE-0000000003, topic=some_topic, 
> partition=225, offset=266411
>       at 
> org.apache.kafka.streams.processor.internals.StreamTask.process(StreamTask.java:216)
>       at 
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:641)
>       at 
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:368)
> Caused by: org.apache.kafka.streams.errors.InvalidStateStoreException: Store 
> someStore-201701060400 is currently closed
>       at 
> org.apache.kafka.streams.state.internals.RocksDBStore.validateStoreOpen(RocksDBStore.java:205)
>       at 
> org.apache.kafka.streams.state.internals.RocksDBStore.put(RocksDBStore.java:221)
>       at 
> org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.put(RocksDBSegmentedBytesStore.java:74)
>       at 
> org.apache.kafka.streams.state.internals.ChangeLoggingSegmentedBytesStore.put(ChangeLoggingSegmentedBytesStore.java:54)
>       at 
> org.apache.kafka.streams.state.internals.MeteredSegmentedBytesStore.put(MeteredSegmentedBytesStore.java:101)
>       at 
> org.apache.kafka.streams.state.internals.RocksDBWindowStore.put(RocksDBWindowStore.java:109)
>       ... X more
> {quote}
> The error occurs for many partitions.
> This was preceded by (for this partition):
> {quote}
> INFO  2017-03-19 18:03:16,403 [StreamThread-10][ConsumerCoordinator.java:393] 
> : Revoking previously assigned partitions [some_topic-225] for group some_job
> INFO  2017-03-19 18:03:16,403 [StreamThread-10][StreamThread.java:254] : 
> stream-thread [StreamThread-10] partitions [[some_topic-225]] revoked at the 
> beginning of consumer rebalance.
> INFO  2017-03-19 18:03:16,403 [StreamThread-10][StreamThread.java:1056] : 
> stream-thread [StreamThread-10] Closing a task's topology 1_225
> INFO  2017-03-19 18:03:17,887 [StreamThread-10][StreamThread.java:544] : 
> stream-thread [StreamThread-10] Flushing state stores of task 1_225
> INFO  2017-03-19 18:03:17,887 [StreamThread-10][StreamThread.java:534] : 
> stream-thread [StreamThread-10] Committing consumer offsets of task 1_225
> INFO  2017-03-19 18:03:17,891 [StreamThread-10][StreamThread.java:1012] : 
> stream-thread [StreamThread-10] Updating suspended tasks to contain active 
> tasks [[1_225, 0_445, 0_30]]
> INFO  2017-03-19 18:03:17,891 [StreamThread-10][StreamThread.java:1019] : 
> stream-thread [StreamThread-10] Removing all active tasks [[1_225, 0_445, 
> 0_30]]
> INFO  2017-03-19 18:03:19,925 [StreamThread-10][ConsumerCoordinator.java:252] 
> : Setting newly assigned partitions [some_tpoic-225] for group some_job
> INFO  2017-03-19 18:03:19,927 [StreamThread-10][StreamThread.java:228] : 
> stream-thread [StreamThread-10] New partitions [[some_topic-225]] assigned at 
> the end of consumer rebalance.
> INFO  2017-03-19 18:03:19,929 [StreamThread-10][StreamTask.java:333] : task 
> [1_225] Initializing processor nodes of the topology
> Something happens.  What ???
> INFO  2017-03-19 18:03:20,135 [StreamThread-10][StreamThread.java:1045] : 
> stream-thread [StreamThread-10] Closing a task 1_225
> INFO  2017-03-19 18:03:20,355 [StreamThread-10][StreamThread.java:544] : 
> stream-thread [StreamThread-10] Flushing state stores of task 1_225
> INFO  2017-03-19 18:03:20,355 [StreamThread-10][StreamThread.java:523] : 
> stream-thread [StreamThread-10] Closing the state manager of task 1_225
> INFO  2017-03-19 18:03:20,432 [StreamThread-10][StreamThread.java:1019] : 
> stream-thread [StreamThread-10] Removing all active tasks [[1_225, 0_445, 
> 0_30]]
> INFO  2017-03-19 18:03:20,432 [StreamThread-10][StreamThread.java:1034] : 
> stream-thread [StreamThread-10] Removing all standby tasks [[]]
> INFO  2017-03-19 18:03:20,432 [StreamThread-10][StreamThread.java:427] : 
> stream-thread [StreamThread-10] Stream thread shutdown complete
> {quote}



--
This message was sent by Atlassian JIRA
(v6.3.15#6346)

Reply via email to