Hi Kiran
You can use the RocksDBConfigSetter and pass

options.setMaxOpenFiles(100);

to all RocksDBs for the Streams application which limits how many are
kept open at the same time.

best regards

Patrik


On Wed, 26 Jun 2019 at 16:14, emailtokir...@gmail.com <
emailtokir...@gmail.com> wrote:

> Hi,
>
> We are using Kafka streams DSL APIs for doing some counter aggregations
> (running on OpenJDK 11.0.2). Our topology has some 400 sub topologies & we
> are using 8 partitions in source topic. When we start pumping more load, we
> start getting RockDBException stating "too many open files".
>
> Here are the stack trace samples:
>
> ------------------------------------------------------------------------------------------
> Caused by: org.rocksdb.RocksDBException: while open a file for lock:
> PPPPPPPPPPP.1512000000/LOCK: Too many open files
>         at org.rocksdb.RocksDB.open(Native Method)
>         at org.rocksdb.RocksDB.open(RocksDB.java:235)
>         at
> org.apache.kafka.streams.state.internals.RocksDBStore.openDB(RocksDBStore.java:156)
>         ... 24 common frames omitted
>
>
> Caused by: org.apache.kafka.streams.errors.ProcessorStateException: Error
> while executing flush from store XXXXXXXXXXX.1512000000
>         at
> org.apache.kafka.streams.state.internals.RocksDBStore.flushInternal(RocksDBStore.java:397)
>         at
> org.apache.kafka.streams.state.internals.RocksDBStore.flush(RocksDBStore.java:388)
>         at
> org.apache.kafka.streams.state.internals.Segments.flush(Segments.java:163)
>         at
> org.apache.kafka.streams.state.internals.RocksDBSegmentedBytesStore.flush(RocksDBSegmentedBytesStore.java:178)
>         at
> org.apache.kafka.streams.state.internals.WrappedStateStore$AbstractStateStore.flush(WrappedStateStore.java:85)
>         at
> org.apache.kafka.streams.state.internals.WrappedStateStore$AbstractStateStore.flush(WrappedStateStore.java:85)
>         at
> org.apache.kafka.streams.state.internals.CachingWindowStore.flush(CachingWindowStore.java:130)
>         at
> org.apache.kafka.streams.state.internals.MeteredWindowStore.flush(MeteredWindowStore.java:177)
>         at
> org.apache.kafka.streams.processor.internals.ProcessorStateManager.flush(ProcessorStateManager.java:217)
>         ... 10 more
> Caused by: org.rocksdb.RocksDBException: While open a file for appending:
> YYYYYYYYYYYYY.1512000000/000007.dbtmp: Too many open files
>         at org.rocksdb.RocksDB.flush(Native Method)
>         at org.rocksdb.RocksDB.flush(RocksDB.java:3401)
>         at org.rocksdb.RocksDB.flush(RocksDB.java:3361)
>         at
> org.apache.kafka.streams.state.internals.RocksDBStore.flushInternal(RocksDBStore.java:395)
>
> ------------------------------------------------------------------------------------------
>
> We tried increasing the open files limit at OS level to some decent
> number.. but still no luck. Obviously we don't want to have boundless open
> files..
>
> We also tried to play with commit interval(kafka.commit.interval.ms) and
> cache size (kafka.cache.max.bytes.buffering) .. but no luck there either.
>
> KAFKA-3904 talks about it.. but it was resolved long back..
>
> Any other config tuning we have to do?
>
> Appreciate any help in this regard!
>
> Thanks,
> Kiran
>
>

Reply via email to