[ 
https://issues.apache.org/jira/browse/KAFKA-4455?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Davor Poldrugo updated KAFKA-4455:
----------------------------------
    Description: 
h2. Problem description
>From time to time a rebalance in Kafka Streams causes the commit to throw 
>CommitFailedException. When this exception is thrown, the tasks and processors 
>are not closed. If some processor contains a state store (RocksDB), the 
>RocksDB is not closed, which leads to not relasead LOCK's on OS level, and 
>when the Kafka Streams app is trying to open tasks and their respective 
>processors and state stores the {{org.rocksdb.RocksDBException: IO error: lock 
>.../LOCK: No locks available}} is thrown. If the the jvm/process is restarted 
>the locks are released.

h2. Additional info
I have been running 3 Kafka Streams instances on separate machines with 
{{num.stream.threads=1}} and each with it's own state directory. Other Kafka 
Streams apps were running on the same machines but they had separate 
directories for state stores. In the attached logs you can see 
{{StreamThread-1895}}, I'm not running 1895 StreamThreads, I have implemented 
some Kafka Streams restart policy in my {{UncaughtExceptionHandler}} which on 
some transient exceptions restarts the 
{{org.apache.kafka.streams.KafkaStreams}} topology, by calling 
{{org.apache.kafka.streams.KafkaStreams.stop()}} and then 
{{org.apache.kafka.streams.KafkaStreams.start()}}. This causes the thread names 
to have bigger numbers.

h2. Stacktrace
[^RocksDBException_IO-error_stacktrace.txt] 

h2. Suggested solution
To avoid restarting the jvm, modify Kafka Streams to close tasks, which will 
lead to release of resources - in this case - filesystem LOCK files.

h2. Possible solution code
Branch: https://github.com/dpoldrugo/kafka/commits/infobip-fork
Commit: [BUGFIX: When commit fails during rebalance - release 
resources|https://github.com/dpoldrugo/kafka/commit/af0d16fc5f8629ab0583c94edf3dbf41158b73f3]
I have been running this fork in production for 3 days and the error doesn't 
come-up.

h2. Note
This could be related this issues: KAFKA-3708 and KAFKA-3938
Additinal conversation can be found here: [stream shut down due to no locks for 
state 
store|https://groups.google.com/forum/#!topic/confluent-platform/i5cwYhpUtx4]

  was:
h2. Problem description
>From time to time a rebalance in Kafka Streams causes the commit to throw 
>CommitFailedException. When this exception is thrown, the tasks and processors 
>are not closed. If some processor contains a state store (RocksDB), the 
>RocksDB is not closed, which leads to not relasead LOCK's on OS level, and 
>when the Kafka Streams app is trying to open tasks and their respective 
>processors and state stores the {{org.rocksdb.RocksDBException: IO error: lock 
>.../LOCK: No locks available}} is thrown. If the the jvm/process is restarted 
>the locks are released.

h2. Additional info
I have been running 3 Kafka Streams instances on separate machines with 
{{num.stream.threads=1}} and each with it's own state directory. Other Kafka 
Streams apps were running on the same machines but they had separate 
directories for state stores. In the attached logs you can see 
{{StreamThread-1895}}, I'm not running 1895 StreamThreads, I have implemented 
some Kafka Streams restart policy in my {{UncaughtExceptionHandler}} which on 
some transient exceptions restarts the 
{{org.apache.kafka.streams.KafkaStreams}} topology, by calling 
{{org.apache.kafka.streams.KafkaStreams.stop()}} and then 
{{org.apache.kafka.streams.KafkaStreams.start()}}.

h2. Stacktrace
[^RocksDBException_IO-error_stacktrace.txt] 

h2. Suggested solution
To avoid restarting the jvm, modify Kafka Streams to close tasks, which will 
lead to release of resources - in this case - filesystem LOCK files.

h2. Possible solution code
Branch: https://github.com/dpoldrugo/kafka/commits/infobip-fork
Commit: [BUGFIX: When commit fails during rebalance - release 
resources|https://github.com/dpoldrugo/kafka/commit/af0d16fc5f8629ab0583c94edf3dbf41158b73f3]
I have been running this fork in production for 3 days and the error doesn't 
come-up.

h2. Note
This could be related this issues: KAFKA-3708 and KAFKA-3938
Additinal conversation can be found here: [stream shut down due to no locks for 
state 
store|https://groups.google.com/forum/#!topic/confluent-platform/i5cwYhpUtx4]


> Commit during rebalance does not close RocksDB which later causes: 
> org.rocksdb.RocksDBException: IO error: lock .../LOCK: No locks available
> --------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: KAFKA-4455
>                 URL: https://issues.apache.org/jira/browse/KAFKA-4455
>             Project: Kafka
>          Issue Type: Bug
>          Components: streams
>    Affects Versions: 0.10.1.0
>         Environment: Kafka Streams were running on CentOS - I have observed 
> this - after some time the locks were released even if the jvm/process wasn't 
> restarted, so I guess CentOS has some lock cleaning policy.
>            Reporter: Davor Poldrugo
>              Labels: stacktrace
>         Attachments: RocksDBException_IO-error_stacktrace.txt
>
>
> h2. Problem description
> From time to time a rebalance in Kafka Streams causes the commit to throw 
> CommitFailedException. When this exception is thrown, the tasks and 
> processors are not closed. If some processor contains a state store 
> (RocksDB), the RocksDB is not closed, which leads to not relasead LOCK's on 
> OS level, and when the Kafka Streams app is trying to open tasks and their 
> respective processors and state stores the {{org.rocksdb.RocksDBException: IO 
> error: lock .../LOCK: No locks available}} is thrown. If the the jvm/process 
> is restarted the locks are released.
> h2. Additional info
> I have been running 3 Kafka Streams instances on separate machines with 
> {{num.stream.threads=1}} and each with it's own state directory. Other Kafka 
> Streams apps were running on the same machines but they had separate 
> directories for state stores. In the attached logs you can see 
> {{StreamThread-1895}}, I'm not running 1895 StreamThreads, I have implemented 
> some Kafka Streams restart policy in my {{UncaughtExceptionHandler}} which on 
> some transient exceptions restarts the 
> {{org.apache.kafka.streams.KafkaStreams}} topology, by calling 
> {{org.apache.kafka.streams.KafkaStreams.stop()}} and then 
> {{org.apache.kafka.streams.KafkaStreams.start()}}. This causes the thread 
> names to have bigger numbers.
> h2. Stacktrace
> [^RocksDBException_IO-error_stacktrace.txt] 
> h2. Suggested solution
> To avoid restarting the jvm, modify Kafka Streams to close tasks, which will 
> lead to release of resources - in this case - filesystem LOCK files.
> h2. Possible solution code
> Branch: https://github.com/dpoldrugo/kafka/commits/infobip-fork
> Commit: [BUGFIX: When commit fails during rebalance - release 
> resources|https://github.com/dpoldrugo/kafka/commit/af0d16fc5f8629ab0583c94edf3dbf41158b73f3]
> I have been running this fork in production for 3 days and the error doesn't 
> come-up.
> h2. Note
> This could be related this issues: KAFKA-3708 and KAFKA-3938
> Additinal conversation can be found here: [stream shut down due to no locks 
> for state 
> store|https://groups.google.com/forum/#!topic/confluent-platform/i5cwYhpUtx4]



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

Reply via email to