Jon,

How many StreamThreads do you have running?
How many application instances?
Do you have more than one instance per machine? If yes, are they sharing
the same State Directory?
Do you have full logs that can be provided so we can try and see how/what
is happening?

Thanks,
Damian

On Mon, 12 Dec 2016 at 10:17 Jon Yeargers <jon.yearg...@cedexis.com> wrote:

> No luck here. Moved all state storage to a non-tmp folder and restarted.
> Still hitting the 'No locks available' error quite frequently.
>
> On Sun, Dec 11, 2016 at 3:45 PM, Jon Yeargers <jon.yearg...@cedexis.com>
> wrote:
>
> > I moved the state folder to a separate drive and linked out to it.
> >
> > I'll try your suggestion and point directly.
> >
> > On Sun, Dec 11, 2016 at 2:20 PM, Matthias J. Sax <matth...@confluent.io>
> > wrote:
> >
> >> I am not sure, but this might be related with your state directory.
> >>
> >> You use default directory that is located in /tmp -- could it be, that
> >> /tmp gets clean up and thus you loose files/directories?
> >>
> >> Try to reconfigure your state directory via StreamsConfig:
> >> http://docs.confluent.io/current/streams/developer-guide.
> >> html#optional-configuration-parameters
> >>
> >>
> >> -Matthias
> >>
> >> On 12/11/16 1:28 AM, Jon Yeargers wrote:
> >> > Seeing this appearing somewhat frequently -
> >> >
> >> > org.apache.kafka.streams.errors.ProcessorStateException: Error opening
> >> > store minute_agg_stream-201612100812 at location
> >> > /tmp/kafka-streams/MinuteAgg/1_9/minute_agg_stream/minute_ag
> >> g_stream-201612100812
> >> >
> >> >         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.
> >> putInternal(RocksDBWindowStore.java:333)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.state.internals.RocksDBWindowStore.
> >> access$100(RocksDBWindowStore.java:51)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.state.internals.RocksDBWindowStore$
> >> 2.restore(RocksDBWindowStore.java:212)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.ProcessorStateM
> >> anager.restoreActiveState(ProcessorStateManager.java:235)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.ProcessorStateM
> >> anager.register(ProcessorStateManager.java:198)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.ProcessorContex
> >> tImpl.register(ProcessorContextImpl.java:123)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.state.internals.RocksDBWindowStore.
> >> init(RocksDBWindowStore.java:206)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.state.internals.MeteredWindowStore.
> >> init(MeteredWindowStore.java:66)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.state.internals.CachingWindowStore.
> >> init(CachingWindowStore.java:64)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.AbstractTask.in
> >> itializeStateStores(AbstractTask.java:81)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.StreamTask.<
> >> init>(StreamTask.java:120)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.StreamThread.
> >> createStreamTask(StreamThread.java:633)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.StreamThread.
> >> addStreamTasks(StreamThread.java:660)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.StreamThread.
> >> access$100(StreamThread.java:69)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.StreamThread$1.
> >> onPartitionsAssigned(StreamThread.java:124)
> >> >
> >> >         at
> >> > org.apache.kafka.clients.consumer.internals.ConsumerCoordina
> >> tor.onJoinComplete(ConsumerCoordinator.java:228)
> >> >
> >> >         at
> >> > org.apache.kafka.clients.consumer.internals.AbstractCoordina
> >> tor.joinGroupIfNeeded(AbstractCoordinator.java:313)
> >> >
> >> >         at
> >> > org.apache.kafka.clients.consumer.internals.AbstractCoordina
> >> tor.ensureActiveGroup(AbstractCoordinator.java:277)
> >> >
> >> >         at
> >> > org.apache.kafka.clients.consumer.internals.ConsumerCoordina
> >> tor.poll(ConsumerCoordinator.java:259)
> >> >
> >> >         at
> >> > org.apache.kafka.clients.consumer.KafkaConsumer.pollOnce(
> >> KafkaConsumer.java:1013)
> >> >
> >> >         at
> >> > org.apache.kafka.clients.consumer.KafkaConsumer.poll(KafkaCo
> >> nsumer.java:979)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.StreamThread.
> >> runLoop(StreamThread.java:407)
> >> >
> >> >         at
> >> > org.apache.kafka.streams.processor.internals.StreamThread.
> >> run(StreamThread.java:242)
> >> >
> >> > Caused by: org.rocksdb.RocksDBException: IO error: lock
> >> > /tmp/kafka-streams/MinuteAgg/1_9/minute_agg_stream/minute_ag
> >> g_stream-201612100812/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)
> >> >
> >> >         ... 26 common frames omitted
> >> >
> >>
> >>
> >
>

Reply via email to