kafka-users mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Ara Ebrahimi <ara.ebrah...@argyledata.com>
Subject kafka streaming rocks db lock bug?
Date Sun, 23 Oct 2016 20:24:47 GMT
Hi,

This happens when I hammer our 5 kafka streaming nodes (each with 4 streaming threads) hard
enough for an hour or so:

2016-10-23 13:04:17 ERROR StreamThread:324 - stream-thread [StreamThread-2] Failed to flush
state for StreamTask 3_8:
org.apache.kafka.streams.errors.ProcessorStateException: task [3_8] Failed to flush state
store streams-data-record-stats-avro-br-store
at org.apache.kafka.streams.processor.internals.ProcessorStateManager.flush(ProcessorStateManager.java:322)
at org.apache.kafka.streams.processor.internals.AbstractTask.flushState(AbstractTask.java:181)
at org.apache.kafka.streams.processor.internals.StreamThread$4.apply(StreamThread.java:360)
at org.apache.kafka.streams.processor.internals.StreamThread.performOnAllTasks(StreamThread.java:322)
at org.apache.kafka.streams.processor.internals.StreamThread.flushAllState(StreamThread.java:357)
at org.apache.kafka.streams.processor.internals.StreamThread.shutdownTasksAndState(StreamThread.java:295)
at org.apache.kafka.streams.processor.internals.StreamThread.shutdown(StreamThread.java:262)
at org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:245)
Caused by: org.apache.kafka.streams.errors.ProcessorStateException: Error opening store streams-data-record-stats-avro-br-store-201505160000
at location /tmp/kafka-streams/argyle-streams/3_8/streams-data-record-stats-avro-br-store/streams-data-record-stats-avro-br-store-201505160000
at org.apache.kafka.streams.state.internals.RocksDBStore.openDB(RocksDBStore.java:196)
at org.apache.kafka.streams.state.internals.RocksDBStore.openDB(RocksDBStore.java:158)
at org.apache.kafka.streams.state.internals.RocksDBWindowStore$Segment.openDB(RocksDBWindowStore.java:72)
at org.apache.kafka.streams.state.internals.RocksDBWindowStore.getOrCreateSegment(RocksDBWindowStore.java:402)
at org.apache.kafka.streams.state.internals.RocksDBWindowStore.putAndReturnInternalKey(RocksDBWindowStore.java:310)
at org.apache.kafka.streams.state.internals.RocksDBWindowStore.put(RocksDBWindowStore.java:292)
at org.apache.kafka.streams.state.internals.MeteredWindowStore.put(MeteredWindowStore.java:101)
at org.apache.kafka.streams.state.internals.CachingWindowStore$1.apply(CachingWindowStore.java:87)
at org.apache.kafka.streams.state.internals.NamedCache.flush(NamedCache.java:117)
at org.apache.kafka.streams.state.internals.ThreadCache.flush(ThreadCache.java:100)
at org.apache.kafka.streams.state.internals.CachingWindowStore.flush(CachingWindowStore.java:118)
at org.apache.kafka.streams.processor.internals.ProcessorStateManager.flush(ProcessorStateManager.java:320)
... 7 more
Caused by: org.rocksdb.RocksDBException: IO error: lock /tmp/kafka-streams/argyle-streams/3_8/streams-data-record-stats-avro-br-store/streams-data-record-stats-avro-br-store-201505160000/LOCK:
No locks available
at org.rocksdb.RocksDB.open(Native Method)
at org.rocksdb.RocksDB.open(RocksDB.java:184)
at org.apache.kafka.streams.state.internals.RocksDBStore.openDB(RocksDBStore.java:189)
... 18 more

Some sort of a locking bug?

Note that when this happen this node stops processing anything and the other nodes seem to
want to pick up the load, which brings the whole streaming cluster to a stand still. That’s
very worrying. Is a document somewhere describing *in detail* how failover for streaming works?

Ara.



________________________________

This message is for the designated recipient only and may contain privileged, proprietary,
or otherwise confidential information. If you have received it in error, please notify the
sender immediately and delete the original. Any other use of the e-mail by you is prohibited.
Thank you in advance for your cooperation.

________________________________
Mime
  • Unnamed multipart/alternative (inline, None, 0 bytes)
View raw message