[ https://issues.apache.org/jira/browse/KAFKA-17098?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17863975#comment-17863975 ]
Matthias J. Sax commented on KAFKA-17098: ----------------------------------------- [~cadonna] – do you think you recent fix for state updated (ie from today) addresses this issues: [https://github.com/apache/kafka/commit/25230b538841a5e7256b1b51725361dd59435901] We did discuss internally about potential lock issues, which you did rule out for the error we say in our own soak, but maybe this one is different? But I would assume you might need logs (\cc [~eduwerc] do you happen to have logs?) to judge? > Error Opening RocksDBStore > -------------------------- > > Key: KAFKA-17098 > URL: https://issues.apache.org/jira/browse/KAFKA-17098 > Project: Kafka > Issue Type: Bug > Components: streams > Affects Versions: 3.8.0 > Reporter: Eduwer Camacaro > Priority: Minor > > I'm getting this exception on many of my state stores: > > {code:java} > 03:35:33 [1;31mERROR[m [LH] KafkaStreamsServerImpl - Uncaught exception > for Error opening store core-repartition-store at location > /lh-data/streams/lh-core/2_8/rocksdb/core-repartition-store 03:35:33 > [1;31mERROR[m [KAFKA] KafkaStreams - stream-client [lh-0-core] Encountered > the following exception during processing and the registered exception > handler opted to SHUTDOWN_CLIENT. The streams client is going to shut down > now. org.apache.kafka.streams.errors.ProcessorStateException: Error opening > store core-repartition-store at location > /lh-data/streams/lh-core/2_8/rocksdb/core-repartition-store at > org.apache.kafka.streams.state.internals.RocksDBStore.openRocksDB(RocksDBStore.java:329) > ~[server.jar:?] at > org.apache.kafka.streams.state.internals.RocksDBStore.openRocksDB(RocksDBStore.java:291) > ~[server.jar:?] at > org.apache.kafka.streams.state.internals.RocksDBStore.openDB(RocksDBStore.java:254) > ~[server.jar:?] at > org.apache.kafka.streams.state.internals.RocksDBStore.init(RocksDBStore.java:175) > ~[server.jar:?] at > org.apache.kafka.streams.state.internals.WrappedStateStore.init(WrappedStateStore.java:71) > ~[server.jar:?] at > org.apache.kafka.streams.state.internals.ChangeLoggingKeyValueBytesStore.init(ChangeLoggingKeyValueBytesStore.java:56) > ~[server.jar:?] at > org.apache.kafka.streams.state.internals.WrappedStateStore.init(WrappedStateStore.java:71) > ~[server.jar:?] at > org.apache.kafka.streams.state.internals.MeteredKeyValueStore.lambda$init$3(MeteredKeyValueStore.java:158) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.metrics.StreamsMetricsImpl.maybeMeasureLatency(StreamsMetricsImpl.java:872) > ~[server.jar:?] at > org.apache.kafka.streams.state.internals.MeteredKeyValueStore.init(MeteredKeyValueStore.java:158) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.ProcessorStateManager.registerStateStores(ProcessorStateManager.java:232) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.StateManagerUtil.registerStateStores(StateManagerUtil.java:102) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.StreamTask.initializeIfNeeded(StreamTask.java:258) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.TaskManager.addTaskToStateUpdater(TaskManager.java:1008) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.TaskManager.addTasksToStateUpdater(TaskManager.java:995) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.TaskManager.checkStateUpdater(TaskManager.java:911) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.StreamThread.checkStateUpdater(StreamThread.java:1188) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.StreamThread.runOnceWithoutProcessingThreads(StreamThread.java:996) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:711) > ~[server.jar:?] at > org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:670) > [server.jar:?] Caused by: org.rocksdb.RocksDBException: lock hold by current > process, acquire time 1720323060 acquiring thread 139820785784576: > /lh-data/streams/lh-core/2_8/rocksdb/core-repartition-store/LOCK: No locks > available at org.rocksdb.RocksDB.open(Native Method) ~[server.jar:?] at > org.rocksdb.RocksDB.open(RocksDB.java:307) ~[server.jar:?] at > org.apache.kafka.streams.state.internals.RocksDBStore.openRocksDB(RocksDBStore.java:323) > ~[server.jar:?] ... 19 more {code} > > Some other details: > - State updater is enabled > - I'm using 5 stream threads > - This usually happends during either normal processing or during state > store restoration -- This message was sent by Atlassian Jira (v8.20.10#820010)