Re: RocksDB Error on partition assignment
Thanks Sameer. Please stay tuned as we work on back port it to 0.10.2.1. Guozhang On Fri, Jul 28, 2017 at 10:15 PM, Sameer Kumar wrote: > Hi Guozhang, > > I am using 10.2.1. > > -Sameer. > > On Sat, Jul 29, 2017 at 12:05 AM, Guozhang Wang > wrote: > > > Sameer, > > > > This bug should be already fixed in trunk. > > > > Which version of Kafka Streams are you running with? We can consider > > backport it and have a bug-fix release if it turns out to be a common > > issue. > > > > > > Guozhang > > > > > > On Fri, Jul 28, 2017 at 4:57 AM, Damian Guy > wrote: > > > > > It is due to a bug. You should set > > > StreamsConfig.STATE_DIR_CLEANUP_DELAY_MS_CONFIG to Long.MAX_VALUE - > > i.e., > > > disabling it. > > > > > > On Fri, 28 Jul 2017 at 10:38 Sameer Kumar > > wrote: > > > > > > > Hi, > > > > > > > > I am facing this error, no clue why this occurred. No other exception > > in > > > > stacktrace was found. > > > > > > > > Only thing different I did was I ran kafka streams jar on machine2 a > > > couple > > > > of mins after i ran it on machine1. > > > > > > > > Please search for this string in the log below:- > > > > org.apache.kafka.streams.processor.internals.StreamThread$1 for > group > > > > LICSp-4-25k failed on partition assignment > > > > > > > > > > > > 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete > state > > > > directory 2_43 for task 2_43 > > > > 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete > state > > > > directory 1_29 for task 1_29 > > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete > state > > > > directory 2_22 for task 2_22 > > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete > state > > > > directory 0_9 for task 0_9 > > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete > state > > > > directory 0_49 for task 0_49 > > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete > state > > > > directory 2_27 for task 2_27 > > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete > state > > > > directory 2_32 for task 2_32 > > > > 2017-07-28 14:55:52 INFO StreamThread:767 - stream-thread > > > [StreamThread-7] > > > > Committing all tasks because the commit interval 5000ms has elapsed > > > > 2017-07-28 14:55:52 INFO StreamThread:805 - stream-thread > > > [StreamThread-7] > > > > Committing task StreamTask 0_1 > > > > 2017-07-28 14:55:52 ERROR StreamThread:813 - stream-thread > > > [StreamThread-2] > > > > Failed to commit StreamTask 1_35 state: > > > > org.apache.kafka.streams.errors.ProcessorStateException: task [1_35] > > > Failed > > > > to flush state store lic3-deb-ci-25k > > > > at > > > > > > > > org.apache.kafka.streams.processor.internals. > > > ProcessorStateManager.flush(ProcessorStateManager.java:337) > > > > at > > > > > > > > org.apache.kafka.streams.processor.internals. > > > StreamTask$1.run(StreamTask.java:72) > > > > at > > > > > > > > org.apache.kafka.streams.processor.internals.StreamsMetricsImpl. > > > measureLatencyNs(StreamsMetricsImpl.java:188) > > > > at > > > > > > > > org.apache.kafka.streams.processor.internals. > > > StreamTask.commit(StreamTask.java:280) > > > > at > > > > > > > > org.apache.kafka.streams.processor.internals.StreamThread.commitOne( > > > StreamThread.java:807) > > > > at > > > > > > > > org.apache.kafka.streams.processor.internals.StreamThread.commitAll( > > > StreamThread.java:794) > > > > at > > > > > > > > org.apache.kafka.streams.processor.internals. > StreamThread.maybeCommit( > > > StreamThread.java:769) > > > > at > > > > > > > > org.apache.kafka.streams.processor.internals.StreamThread.runLoop( > > > StreamThread.java:647) > > > > at > > > > > > > > org.apache.kafka.streams.processor.internals. > > > StreamThread.run(StreamThread.java:361) > > > > Caused by: org.apache.kafka.streams.errors.ProcessorStateException: > > > Error > > > > while executing flush from store lic3-deb-ci-25k-201707280900 > > > > at > > > > > > > > org.apache.kafka.streams.state.internals.RocksDBStore. > > > flushInternal(RocksDBStore.java:354) > > > > at > > > > > > > > org.apache.kafka.streams.state.internals.RocksDBStore. > > > flush(RocksDBStore.java:345) > > > > at > > > > org.apache.kafka.streams.state.internals.Segments. > > > flush(Segments.java:138) > > > > at > > > > > > > > org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore. > > > flush(RocksDBSegmentedBytesStore.java:117) > > > > at > > > > > > > > org.apache.kafka.streams.state.internals.WrappedStateStore$ > > > AbstractWrappedStateStore.flush(WrappedStateStore.java:80) > > > > at > > > > > > > > org.apache.kafka.streams.state.internals.MeteredSegmentedBytesStore. > > > flush(MeteredSegmentedBytesStore.java:111) > > > > at > > > > > > > > org.apache.kafka.streams.state.internals.RocksDBWindowStore.flush( > > > RocksDBWindowStore.java:92) > > > > at > > > > > > > > org.apache.kafka.streams.state.internals.CachingWindowStore.flush( > > > CachingWindowStore.
Re: RocksDB Error on partition assignment
Hi Guozhang, I am using 10.2.1. -Sameer. On Sat, Jul 29, 2017 at 12:05 AM, Guozhang Wang wrote: > Sameer, > > This bug should be already fixed in trunk. > > Which version of Kafka Streams are you running with? We can consider > backport it and have a bug-fix release if it turns out to be a common > issue. > > > Guozhang > > > On Fri, Jul 28, 2017 at 4:57 AM, Damian Guy wrote: > > > It is due to a bug. You should set > > StreamsConfig.STATE_DIR_CLEANUP_DELAY_MS_CONFIG to Long.MAX_VALUE - > i.e., > > disabling it. > > > > On Fri, 28 Jul 2017 at 10:38 Sameer Kumar > wrote: > > > > > Hi, > > > > > > I am facing this error, no clue why this occurred. No other exception > in > > > stacktrace was found. > > > > > > Only thing different I did was I ran kafka streams jar on machine2 a > > couple > > > of mins after i ran it on machine1. > > > > > > Please search for this string in the log below:- > > > org.apache.kafka.streams.processor.internals.StreamThread$1 for group > > > LICSp-4-25k failed on partition assignment > > > > > > > > > 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete state > > > directory 2_43 for task 2_43 > > > 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete state > > > directory 1_29 for task 1_29 > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > > directory 2_22 for task 2_22 > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > > directory 0_9 for task 0_9 > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > > directory 0_49 for task 0_49 > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > > directory 2_27 for task 2_27 > > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > > directory 2_32 for task 2_32 > > > 2017-07-28 14:55:52 INFO StreamThread:767 - stream-thread > > [StreamThread-7] > > > Committing all tasks because the commit interval 5000ms has elapsed > > > 2017-07-28 14:55:52 INFO StreamThread:805 - stream-thread > > [StreamThread-7] > > > Committing task StreamTask 0_1 > > > 2017-07-28 14:55:52 ERROR StreamThread:813 - stream-thread > > [StreamThread-2] > > > Failed to commit StreamTask 1_35 state: > > > org.apache.kafka.streams.errors.ProcessorStateException: task [1_35] > > Failed > > > to flush state store lic3-deb-ci-25k > > > at > > > > > > org.apache.kafka.streams.processor.internals. > > ProcessorStateManager.flush(ProcessorStateManager.java:337) > > > at > > > > > > org.apache.kafka.streams.processor.internals. > > StreamTask$1.run(StreamTask.java:72) > > > at > > > > > > org.apache.kafka.streams.processor.internals.StreamsMetricsImpl. > > measureLatencyNs(StreamsMetricsImpl.java:188) > > > at > > > > > > org.apache.kafka.streams.processor.internals. > > StreamTask.commit(StreamTask.java:280) > > > at > > > > > > org.apache.kafka.streams.processor.internals.StreamThread.commitOne( > > StreamThread.java:807) > > > at > > > > > > org.apache.kafka.streams.processor.internals.StreamThread.commitAll( > > StreamThread.java:794) > > > at > > > > > > org.apache.kafka.streams.processor.internals.StreamThread.maybeCommit( > > StreamThread.java:769) > > > at > > > > > > org.apache.kafka.streams.processor.internals.StreamThread.runLoop( > > StreamThread.java:647) > > > at > > > > > > org.apache.kafka.streams.processor.internals. > > StreamThread.run(StreamThread.java:361) > > > Caused by: org.apache.kafka.streams.errors.ProcessorStateException: > > Error > > > while executing flush from store lic3-deb-ci-25k-201707280900 > > > at > > > > > > org.apache.kafka.streams.state.internals.RocksDBStore. > > flushInternal(RocksDBStore.java:354) > > > at > > > > > > org.apache.kafka.streams.state.internals.RocksDBStore. > > flush(RocksDBStore.java:345) > > > at > > > org.apache.kafka.streams.state.internals.Segments. > > flush(Segments.java:138) > > > at > > > > > > org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore. > > flush(RocksDBSegmentedBytesStore.java:117) > > > at > > > > > > org.apache.kafka.streams.state.internals.WrappedStateStore$ > > AbstractWrappedStateStore.flush(WrappedStateStore.java:80) > > > at > > > > > > org.apache.kafka.streams.state.internals.MeteredSegmentedBytesStore. > > flush(MeteredSegmentedBytesStore.java:111) > > > at > > > > > > org.apache.kafka.streams.state.internals.RocksDBWindowStore.flush( > > RocksDBWindowStore.java:92) > > > at > > > > > > org.apache.kafka.streams.state.internals.CachingWindowStore.flush( > > CachingWindowStore.java:120) > > > at > > > > > > org.apache.kafka.streams.processor.internals. > > ProcessorStateManager.flush(ProcessorStateManager.java:335) > > > ... 8 more > > > Caused by: org.rocksdb.RocksDBException: s > > > at org.rocksdb.RocksDB.flush(Native Method) > > > at org.rocksdb.RocksDB.flush(RocksDB.java:1642) > > > at > > > > > > org.apache.kafka.streams.state.internals.RocksDBStore. > > flushInternal(RocksDBStore.java:352) > > > ... 16 m
Re: RocksDB Error on partition assignment
Sameer, This bug should be already fixed in trunk. Which version of Kafka Streams are you running with? We can consider backport it and have a bug-fix release if it turns out to be a common issue. Guozhang On Fri, Jul 28, 2017 at 4:57 AM, Damian Guy wrote: > It is due to a bug. You should set > StreamsConfig.STATE_DIR_CLEANUP_DELAY_MS_CONFIG to Long.MAX_VALUE - i.e., > disabling it. > > On Fri, 28 Jul 2017 at 10:38 Sameer Kumar wrote: > > > Hi, > > > > I am facing this error, no clue why this occurred. No other exception in > > stacktrace was found. > > > > Only thing different I did was I ran kafka streams jar on machine2 a > couple > > of mins after i ran it on machine1. > > > > Please search for this string in the log below:- > > org.apache.kafka.streams.processor.internals.StreamThread$1 for group > > LICSp-4-25k failed on partition assignment > > > > > > 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete state > > directory 2_43 for task 2_43 > > 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete state > > directory 1_29 for task 1_29 > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > directory 2_22 for task 2_22 > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > directory 0_9 for task 0_9 > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > directory 0_49 for task 0_49 > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > directory 2_27 for task 2_27 > > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > > directory 2_32 for task 2_32 > > 2017-07-28 14:55:52 INFO StreamThread:767 - stream-thread > [StreamThread-7] > > Committing all tasks because the commit interval 5000ms has elapsed > > 2017-07-28 14:55:52 INFO StreamThread:805 - stream-thread > [StreamThread-7] > > Committing task StreamTask 0_1 > > 2017-07-28 14:55:52 ERROR StreamThread:813 - stream-thread > [StreamThread-2] > > Failed to commit StreamTask 1_35 state: > > org.apache.kafka.streams.errors.ProcessorStateException: task [1_35] > Failed > > to flush state store lic3-deb-ci-25k > > at > > > > org.apache.kafka.streams.processor.internals. > ProcessorStateManager.flush(ProcessorStateManager.java:337) > > at > > > > org.apache.kafka.streams.processor.internals. > StreamTask$1.run(StreamTask.java:72) > > at > > > > org.apache.kafka.streams.processor.internals.StreamsMetricsImpl. > measureLatencyNs(StreamsMetricsImpl.java:188) > > at > > > > org.apache.kafka.streams.processor.internals. > StreamTask.commit(StreamTask.java:280) > > at > > > > org.apache.kafka.streams.processor.internals.StreamThread.commitOne( > StreamThread.java:807) > > at > > > > org.apache.kafka.streams.processor.internals.StreamThread.commitAll( > StreamThread.java:794) > > at > > > > org.apache.kafka.streams.processor.internals.StreamThread.maybeCommit( > StreamThread.java:769) > > at > > > > org.apache.kafka.streams.processor.internals.StreamThread.runLoop( > StreamThread.java:647) > > at > > > > org.apache.kafka.streams.processor.internals. > StreamThread.run(StreamThread.java:361) > > Caused by: org.apache.kafka.streams.errors.ProcessorStateException: > Error > > while executing flush from store lic3-deb-ci-25k-201707280900 > > at > > > > org.apache.kafka.streams.state.internals.RocksDBStore. > flushInternal(RocksDBStore.java:354) > > at > > > > org.apache.kafka.streams.state.internals.RocksDBStore. > flush(RocksDBStore.java:345) > > at > > org.apache.kafka.streams.state.internals.Segments. > flush(Segments.java:138) > > at > > > > org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore. > flush(RocksDBSegmentedBytesStore.java:117) > > at > > > > org.apache.kafka.streams.state.internals.WrappedStateStore$ > AbstractWrappedStateStore.flush(WrappedStateStore.java:80) > > at > > > > org.apache.kafka.streams.state.internals.MeteredSegmentedBytesStore. > flush(MeteredSegmentedBytesStore.java:111) > > at > > > > org.apache.kafka.streams.state.internals.RocksDBWindowStore.flush( > RocksDBWindowStore.java:92) > > at > > > > org.apache.kafka.streams.state.internals.CachingWindowStore.flush( > CachingWindowStore.java:120) > > at > > > > org.apache.kafka.streams.processor.internals. > ProcessorStateManager.flush(ProcessorStateManager.java:335) > > ... 8 more > > Caused by: org.rocksdb.RocksDBException: s > > at org.rocksdb.RocksDB.flush(Native Method) > > at org.rocksdb.RocksDB.flush(RocksDB.java:1642) > > at > > > > org.apache.kafka.streams.state.internals.RocksDBStore. > flushInternal(RocksDBStore.java:352) > > ... 16 more > > 2017-07-28 14:55:52 INFO StreamThread:767 - stream-thread > > [StreamThread-12] Committing all tasks because the commit interval 5000ms > > has elapsed > > 2017-07-28 14:55:52 INFO StreamThread:390 - stream-thread > [StreamThread-2] > > Shutting down > > 2017-07-28 14:55:52 INFO StreamThread:805 - stream-thread > > [StreamThread-12] Committing task StreamTask 1_32 > > 2017
Re: RocksDB Error on partition assignment
It is due to a bug. You should set StreamsConfig.STATE_DIR_CLEANUP_DELAY_MS_CONFIG to Long.MAX_VALUE - i.e., disabling it. On Fri, 28 Jul 2017 at 10:38 Sameer Kumar wrote: > Hi, > > I am facing this error, no clue why this occurred. No other exception in > stacktrace was found. > > Only thing different I did was I ran kafka streams jar on machine2 a couple > of mins after i ran it on machine1. > > Please search for this string in the log below:- > org.apache.kafka.streams.processor.internals.StreamThread$1 for group > LICSp-4-25k failed on partition assignment > > > 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete state > directory 2_43 for task 2_43 > 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete state > directory 1_29 for task 1_29 > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > directory 2_22 for task 2_22 > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > directory 0_9 for task 0_9 > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > directory 0_49 for task 0_49 > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > directory 2_27 for task 2_27 > 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state > directory 2_32 for task 2_32 > 2017-07-28 14:55:52 INFO StreamThread:767 - stream-thread [StreamThread-7] > Committing all tasks because the commit interval 5000ms has elapsed > 2017-07-28 14:55:52 INFO StreamThread:805 - stream-thread [StreamThread-7] > Committing task StreamTask 0_1 > 2017-07-28 14:55:52 ERROR StreamThread:813 - stream-thread [StreamThread-2] > Failed to commit StreamTask 1_35 state: > org.apache.kafka.streams.errors.ProcessorStateException: task [1_35] Failed > to flush state store lic3-deb-ci-25k > at > > org.apache.kafka.streams.processor.internals.ProcessorStateManager.flush(ProcessorStateManager.java:337) > at > > org.apache.kafka.streams.processor.internals.StreamTask$1.run(StreamTask.java:72) > at > > org.apache.kafka.streams.processor.internals.StreamsMetricsImpl.measureLatencyNs(StreamsMetricsImpl.java:188) > at > > org.apache.kafka.streams.processor.internals.StreamTask.commit(StreamTask.java:280) > at > > org.apache.kafka.streams.processor.internals.StreamThread.commitOne(StreamThread.java:807) > at > > org.apache.kafka.streams.processor.internals.StreamThread.commitAll(StreamThread.java:794) > at > > org.apache.kafka.streams.processor.internals.StreamThread.maybeCommit(StreamThread.java:769) > at > > org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:647) > at > > org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:361) > Caused by: org.apache.kafka.streams.errors.ProcessorStateException: Error > while executing flush from store lic3-deb-ci-25k-201707280900 > at > > org.apache.kafka.streams.state.internals.RocksDBStore.flushInternal(RocksDBStore.java:354) > at > > org.apache.kafka.streams.state.internals.RocksDBStore.flush(RocksDBStore.java:345) > at > org.apache.kafka.streams.state.internals.Segments.flush(Segments.java:138) > at > > org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.flush(RocksDBSegmentedBytesStore.java:117) > at > > org.apache.kafka.streams.state.internals.WrappedStateStore$AbstractWrappedStateStore.flush(WrappedStateStore.java:80) > at > > org.apache.kafka.streams.state.internals.MeteredSegmentedBytesStore.flush(MeteredSegmentedBytesStore.java:111) > at > > org.apache.kafka.streams.state.internals.RocksDBWindowStore.flush(RocksDBWindowStore.java:92) > at > > org.apache.kafka.streams.state.internals.CachingWindowStore.flush(CachingWindowStore.java:120) > at > > org.apache.kafka.streams.processor.internals.ProcessorStateManager.flush(ProcessorStateManager.java:335) > ... 8 more > Caused by: org.rocksdb.RocksDBException: s > at org.rocksdb.RocksDB.flush(Native Method) > at org.rocksdb.RocksDB.flush(RocksDB.java:1642) > at > > org.apache.kafka.streams.state.internals.RocksDBStore.flushInternal(RocksDBStore.java:352) > ... 16 more > 2017-07-28 14:55:52 INFO StreamThread:767 - stream-thread > [StreamThread-12] Committing all tasks because the commit interval 5000ms > has elapsed > 2017-07-28 14:55:52 INFO StreamThread:390 - stream-thread [StreamThread-2] > Shutting down > 2017-07-28 14:55:52 INFO StreamThread:805 - stream-thread > [StreamThread-12] Committing task StreamTask 1_32 > 2017-07-28 14:55:52 INFO StreamThread:1075 - stream-thread > [StreamThread-2] Closing task 0_0 > 2017-07-28 14:55:53 INFO StreamThread:767 - stream-thread > [StreamThread-15] Committing all tasks because the commit interval 5000ms > has elapsed > 2017-07-28 14:55:53 INFO StreamThread:805 - stream-thread > [StreamThread-15] Committing task StreamTask 0_32 > 2017-07-28 14:55:53 INFO StreamThread:767 - stream-thread [StreamThread-5] > Committing all tasks because the commit interval 5000ms has elapsed > 2017-07-28 14:55:53 INFO StreamThread:805 - stream-
RocksDB Error on partition assignment
Hi, I am facing this error, no clue why this occurred. No other exception in stacktrace was found. Only thing different I did was I ran kafka streams jar on machine2 a couple of mins after i ran it on machine1. Please search for this string in the log below:- org.apache.kafka.streams.processor.internals.StreamThread$1 for group LICSp-4-25k failed on partition assignment 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete state directory 2_43 for task 2_43 2017-07-28 14:55:51 INFO StateDirectory:213 - Deleting obsolete state directory 1_29 for task 1_29 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state directory 2_22 for task 2_22 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state directory 0_9 for task 0_9 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state directory 0_49 for task 0_49 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state directory 2_27 for task 2_27 2017-07-28 14:55:52 INFO StateDirectory:213 - Deleting obsolete state directory 2_32 for task 2_32 2017-07-28 14:55:52 INFO StreamThread:767 - stream-thread [StreamThread-7] Committing all tasks because the commit interval 5000ms has elapsed 2017-07-28 14:55:52 INFO StreamThread:805 - stream-thread [StreamThread-7] Committing task StreamTask 0_1 2017-07-28 14:55:52 ERROR StreamThread:813 - stream-thread [StreamThread-2] Failed to commit StreamTask 1_35 state: org.apache.kafka.streams.errors.ProcessorStateException: task [1_35] Failed to flush state store lic3-deb-ci-25k at org.apache.kafka.streams.processor.internals.ProcessorStateManager.flush(ProcessorStateManager.java:337) at org.apache.kafka.streams.processor.internals.StreamTask$1.run(StreamTask.java:72) at org.apache.kafka.streams.processor.internals.StreamsMetricsImpl.measureLatencyNs(StreamsMetricsImpl.java:188) at org.apache.kafka.streams.processor.internals.StreamTask.commit(StreamTask.java:280) at org.apache.kafka.streams.processor.internals.StreamThread.commitOne(StreamThread.java:807) at org.apache.kafka.streams.processor.internals.StreamThread.commitAll(StreamThread.java:794) at org.apache.kafka.streams.processor.internals.StreamThread.maybeCommit(StreamThread.java:769) at org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:647) at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:361) Caused by: org.apache.kafka.streams.errors.ProcessorStateException: Error while executing flush from store lic3-deb-ci-25k-201707280900 at org.apache.kafka.streams.state.internals.RocksDBStore.flushInternal(RocksDBStore.java:354) at org.apache.kafka.streams.state.internals.RocksDBStore.flush(RocksDBStore.java:345) at org.apache.kafka.streams.state.internals.Segments.flush(Segments.java:138) at org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.flush(RocksDBSegmentedBytesStore.java:117) at org.apache.kafka.streams.state.internals.WrappedStateStore$AbstractWrappedStateStore.flush(WrappedStateStore.java:80) at org.apache.kafka.streams.state.internals.MeteredSegmentedBytesStore.flush(MeteredSegmentedBytesStore.java:111) at org.apache.kafka.streams.state.internals.RocksDBWindowStore.flush(RocksDBWindowStore.java:92) at org.apache.kafka.streams.state.internals.CachingWindowStore.flush(CachingWindowStore.java:120) at org.apache.kafka.streams.processor.internals.ProcessorStateManager.flush(ProcessorStateManager.java:335) ... 8 more Caused by: org.rocksdb.RocksDBException: s at org.rocksdb.RocksDB.flush(Native Method) at org.rocksdb.RocksDB.flush(RocksDB.java:1642) at org.apache.kafka.streams.state.internals.RocksDBStore.flushInternal(RocksDBStore.java:352) ... 16 more 2017-07-28 14:55:52 INFO StreamThread:767 - stream-thread [StreamThread-12] Committing all tasks because the commit interval 5000ms has elapsed 2017-07-28 14:55:52 INFO StreamThread:390 - stream-thread [StreamThread-2] Shutting down 2017-07-28 14:55:52 INFO StreamThread:805 - stream-thread [StreamThread-12] Committing task StreamTask 1_32 2017-07-28 14:55:52 INFO StreamThread:1075 - stream-thread [StreamThread-2] Closing task 0_0 2017-07-28 14:55:53 INFO StreamThread:767 - stream-thread [StreamThread-15] Committing all tasks because the commit interval 5000ms has elapsed 2017-07-28 14:55:53 INFO StreamThread:805 - stream-thread [StreamThread-15] Committing task StreamTask 0_32 2017-07-28 14:55:53 INFO StreamThread:767 - stream-thread [StreamThread-5] Committing all tasks because the commit interval 5000ms has elapsed 2017-07-28 14:55:53 INFO StreamThread:805 - stream-thread [StreamThread-5] Committing task StreamTask 2_31 2017-07-28 14:55:53 INFO StreamThread:767 - stream-thread [StreamThread-14] Committing all tasks because the commit interval 5000ms has elapsed 2017-07-28 14:55:53 INFO StreamThread:805 - stream-thread [StreamThread-14] Committing task StreamTask 0_34 2017-07-28 14:55:53 INFO StreamThread:805 - stream-thread [StreamThread-14] Committing ta