Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-02 Thread emailtokir...@gmail.com



On 2019/06/28 23:29:16, John Roesler  wrote: 
> Hey all,
> 
> If you want to figure it out theoretically, if you print out the
> topology description, you'll have some number of state stores listed
> in there. The number of Rocks instances should just be
> (#global_state_stores +
> sum(#partitions_of_topic_per_local_state_store)) . The number of
> stream threads isn't relevant here.
> 
> You can also figure it out empirically: the first level of
> subdirectories in the state dir are Tasks, and then within that, the
> next level is Stores. You should see the store directory names match
> up with the stores listed in the topology description. The number of
> Store directories is exactly the number of RocksDB instances you have.
> 
> There are also metrics corresponding to each of the state stores, so
> you can compute it from what you find in the metrics.
> 
> Hope that helps,
> -john
> 
> On Thu, Jun 27, 2019 at 6:46 AM Patrik Kleindl  wrote:
> >
> > Hi Kiran
> > Without much research my guess would be "num_stream_threads *
> > (#global_state_stores + sum(#partitions_of_topic_per_local_state_store))"
> > So 10 stores (regardless if explicitly defined or implicitely because of
> > some stateful operation) with 10 partitions each should result in 100
> > Rocksdb instances if you are running at the default of num_stream_threads=1.
> >
> > As I wrote before, start with 100.
> > If the error persists, half the number, if not, double it ;-) Repeat as
> > needed.
> >
> > If you reach the single-digit-range and the error still shows up, start
> > searching for any iterators over a store you might not have closed.
> >
> > br, Patrik
> >
> > On Thu, 27 Jun 2019 at 13:11, emailtokir...@gmail.com <
> > emailtokir...@gmail.com> wrote:
> >
> > >
> > >
> > > On 2019/06/27 09:02:39, Patrik Kleindl  wrote:
> > > > Hello Kiran
> > > >
> > > > First, the value for maxOpenFiles is per RocksDB instance, and the 
> > > > number
> > > > of those can get high if you have a lot of topic partitions etc.
> > > > Check the directory (state dir) to see how many there are.
> > > > Start with a low value (100) and see if that has some effect.
> > > >
> > > > Second, because I just found out, you should use
> > > > BlockBasedTableConfig tableConfig = (BlockBasedTableConfig)
> > > > options.tableFormatConfig();
> > > > tableConfig.setBlockCacheSize(100*1024*1024L);
> > > > tableConfig.setBlockSize(8*1024L);
> > > > instead of creating a new object to prevent accidently messing up
> > > > references.
> > > >
> > > > Hope that helps
> > > > best regards
> > > > Patrik
> > > >
> > > > On Thu, 27 Jun 2019 at 10:46, emailtokir...@gmail.com <
> > > > emailtokir...@gmail.com> wrote:
> > > >
> > > > >
> > > > >
> > > > > On 2019/06/26 21:58:02, Patrik Kleindl  wrote:
> > > > > > 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:
> > > > > > >
> > > > > > >
> > > > >
> > > ---

Re: Kafka streams (2.1.1) - org.rocksdb.RocksDBException:Too many open files

2019-07-02 Thread emailtokir...@gmail.com



On 2019/07/03 05:46:45, Sophie Blee-Goldman  wrote: 
> It sounds like rocksdb *is* honoring your configs -- the max.open.files
> config is an internal restriction that tells rocksdb how many open files it
> is allowed to have, so if that's set to -1 (infinite) it won't ever try to
> limit its open files and you may hit the OS limit.
> 
> Think of it this way: if you have 100 rocksdb instances and a OS limit of
> 500, you should set max.open.files to 5  to avoid hitting this limit
> (assuming there are no other open files on the system, in reality you'd
> want some extra room there)
> 
> On Tue, Jul 2, 2019 at 7:53 PM emailtokir...@gmail.com <
> emailtokir...@gmail.com> wrote:
> 
> >
> >
> > On 2019/06/28 23:29:16, John Roesler  wrote:
> > > Hey all,
> > >
> > > If you want to figure it out theoretically, if you print out the
> > > topology description, you'll have some number of state stores listed
> > > in there. The number of Rocks instances should just be
> > > (#global_state_stores +
> > > sum(#partitions_of_topic_per_local_state_store)) . The number of
> > > stream threads isn't relevant here.
> > >
> > > You can also figure it out empirically: the first level of
> > > subdirectories in the state dir are Tasks, and then within that, the
> > > next level is Stores. You should see the store directory names match
> > > up with the stores listed in the topology description. The number of
> > > Store directories is exactly the number of RocksDB instances you have.
> > >
> > > There are also metrics corresponding to each of the state stores, so
> > > you can compute it from what you find in the metrics.
> > >
> > > Hope that helps,
> > > -john
> > >
> > > On Thu, Jun 27, 2019 at 6:46 AM Patrik Kleindl 
> > wrote:
> > > >
> > > > Hi Kiran
> > > > Without much research my guess would be "num_stream_threads *
> > > > (#global_state_stores +
> > sum(#partitions_of_topic_per_local_state_store))"
> > > > So 10 stores (regardless if explicitly defined or implicitely because
> > of
> > > > some stateful operation) with 10 partitions each should result in 100
> > > > Rocksdb instances if you are running at the default of
> > num_stream_threads=1.
> > > >
> > > > As I wrote before, start with 100.
> > > > If the error persists, half the number, if not, double it ;-) Repeat as
> > > > needed.
> > > >
> > > > If you reach the single-digit-range and the error still shows up, start
> > > > searching for any iterators over a store you might not have closed.
> > > >
> > > > br, Patrik
> > > >
> > > > On Thu, 27 Jun 2019 at 13:11, emailtokir...@gmail.com <
> > > > emailtokir...@gmail.com> wrote:
> > > >
> > > > >
> > > > >
> > > > > On 2019/06/27 09:02:39, Patrik Kleindl  wrote:
> > > > > > Hello Kiran
> > > > > >
> > > > > > First, the value for maxOpenFiles is per RocksDB instance, and the
> > number
> > > > > > of those can get high if you have a lot of topic partitions etc.
> > > > > > Check the directory (state dir) to see how many there are.
> > > > > > Start with a low value (100) and see if that has some effect.
> > > > > >
> > > > > > Second, because I just found out, you should use
> > > > > > BlockBasedTableConfig tableConfig = (BlockBasedTableConfig)
> > > > > > options.tableFormatConfig();
> > > > > > tableConfig.setBlockCacheSize(100*1024*1024L);
> > > > > > tableConfig.setBlockSize(8*1024L);
> > > > > > instead of creating a new object to prevent accidently messing up
> > > > > > references.
> > > > > >
> > > > > > Hope that helps
> > > > > > best regards
> > > > > > Patrik
> > > > > >
> > > > > > On Thu, 27 Jun 2019 at 10:46, emailtokir...@gmail.com <
> > > > > > emailtokir...@gmail.com> wrote:
> > > > > >
> > > > > > >
> > > > > > >
> > > > > > > On 2019/06/26 21:58:02, Patrik Klei