[
https://issues.apache.org/jira/browse/KAFKA-5070?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16035471#comment-16035471
]
Matthias J. Sax commented on KAFKA-5070:
----------------------------------------
[~kchen] At a regular rebalance, this exception is expected. That's why we log
it at {{WARN}} level -- the stack trace is a little miss leading -- it's not a
real issues and we removed the stack trace for upcoming {{0.11}} release. So as
long as the exception does resolve itself, note the {{"Will retry."}} clause,
it's all fine. It's only a problem is the exception does not resolve itself and
get stuck retrying.
Thus, the {{rocskdb.flush()}} exception seems to be a real problem. Are you
using {{0.10.2.0}} or {{0.10.2.1}}? We fixed a couple of issues for regard to
state dir locking in {{0.10.2.1}} so I would highly recommend to upgrade to the
bug fix release. More fixes are coming in {{0.11}} -- the target release date
is mid June. If you are already on {{0.10.2.1}} can you maybe share some logs
or stacktrace for the {{rocskdb.flush()}} exception?
Hope this helps.
> org.apache.kafka.streams.errors.LockException: task [0_18] Failed to lock the
> state directory: /opt/rocksdb/pulse10/0_18
> ------------------------------------------------------------------------------------------------------------------------
>
> Key: KAFKA-5070
> URL: https://issues.apache.org/jira/browse/KAFKA-5070
> Project: Kafka
> Issue Type: Bug
> Components: streams
> Affects Versions: 0.10.2.0
> Environment: Linux Version
> Reporter: Dhana
> Assignee: Matthias J. Sax
> Attachments: RocksDB_LockStateDirec.7z
>
>
> Notes: we run two instance of consumer in two difference machines/nodes.
> we have 400 partitions. 200 stream threads/consumer, with 2 consumer.
> We perform HA test(on rebalance - shutdown of one of the consumer/broker), we
> see this happening
> Error:
> 2017-04-05 11:36:09.352 WARN StreamThread:1184 StreamThread-66 - Could not
> create task 0_115. Will retry.
> org.apache.kafka.streams.errors.LockException: task [0_115] Failed to lock
> the state directory: /opt/rocksdb/pulse10/0_115
> at
> org.apache.kafka.streams.processor.internals.ProcessorStateManager.<init>(ProcessorStateManager.java:102)
> at
> org.apache.kafka.streams.processor.internals.AbstractTask.<init>(AbstractTask.java:73)
> at
> org.apache.kafka.streams.processor.internals.StreamTask.<init>(StreamTask.java:108)
> at
> org.apache.kafka.streams.processor.internals.StreamThread.createStreamTask(StreamThread.java:834)
> at
> org.apache.kafka.streams.processor.internals.StreamThread$TaskCreator.createTask(StreamThread.java:1207)
> at
> org.apache.kafka.streams.processor.internals.StreamThread$AbstractTaskCreator.retryWithBackoff(StreamThread.java:1180)
> at
> org.apache.kafka.streams.processor.internals.StreamThread.addStreamTasks(StreamThread.java:937)
> at
> org.apache.kafka.streams.processor.internals.StreamThread.access$500(StreamThread.java:69)
> at
> org.apache.kafka.streams.processor.internals.StreamThread$1.onPartitionsAssigned(StreamThread.java:236)
> at
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.onJoinComplete(ConsumerCoordinator.java:255)
> at
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.joinGroupIfNeeded(AbstractCoordinator.java:339)
> at
> org.apache.kafka.clients.consumer.internals.AbstractCoordinator.ensureActiveGroup(AbstractCoordinator.java:303)
> at
> org.apache.kafka.clients.consumer.internals.ConsumerCoordinator.poll(ConsumerCoordinator.java:286)
> at
> org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(KafkaConsumer.java:1030)
> at
> org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaConsumer.java:995)
> at
> org.apache.kafka.streams.processor.internals.StreamThread.runLoop(StreamThread.java:582)
> at
> org.apache.kafka.streams.processor.internals.StreamThread.run(StreamThread.java:368)
--
This message was sent by Atlassian JIRA
(v6.3.15#6346)