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

2019-07-12 Thread Thameem Ansari
Actually in our topology 394 different counters get set and with 8 partitions 
it totals to 3152 tasks. 
Btw, just to give you an update. I was able to run the stream application in 2 
separate instances in 2 VMs and it is working fine. As I suspected in my 
previous email rocksdb wasn’t able to work properly when there are too many 
tasks from single streaming application instance or we are missing some 
configuration to handle it with single instance. 
After few more tests next week I can conclude my findings. 


Sent from my iPhone

> On Jul 12, 2019, at 9:10 PM, Bill Bejeck  wrote:
> 
> Hi Thameem,
> 
>> Our topology has 3100 tasks with 3 brokers and 8 partitions.
> 
> I have a question that is somewhat orthogonal to your original item.  I may
> be missing something, but are you saying your streams application has 3,100
> tasks from 8 input partitions? If that is the case, do you mind sharing
> your topology? It seems like you have several repartition operations and
> you may be able to reduce the number of repartition operations depending on
> what you are doing.
> 
> Thanks,
> Bill
> 
>> On Thu, Jul 11, 2019 at 11:24 PM Thameem Ansari  wrote:
>> 
>> Open files in the system was set to 10Million and users limit is 1million.
>> When the process was active i was closely watching the open files and it
>> was around 400K so its well within the set limit. Rocksdb open files we
>> tried setting 100, 50, -1 but the results are same.
>> I am using Rocksdb config setter and not options (not sure what it is)
>> file.
>> 
>> Quick question: Our topology has 3100 tasks with 3 brokers and 8
>> partitions. But we are using only one streaming application instance with
>> single thread. Does this cause any issue? Today i am going to try with 2 or
>> 3 instances and see whether we can get rid of the issue.
>> 
>> Thanks
>> Thameem
>> 
>> 
>>> On Jul 12, 2019, at 1:57 AM, Sophie Blee-Goldman 
>> wrote:
>>> 
>>> I believe the "resource temporarily unavailable" actually is related to
>> the
>>> open files, most likely you are hitting the total file descriptor limit.
>>> Sorry if you mentioned this and I missed it, but what was the
>>> max.open.files in your RocksDBConfigSetter when you ran this? Actually,
>>> could you just include your full RocksDBConfigSetter implementation?
>>> 
>>> Not sure why Rocks would be spawning so many child processes...it does
>> use
>>> background threads for compaction but this sounds like something totally
>>> different. I also notice a read "This is a RocksDB options file" -- are
>> you
>>> trying to use an options file vs setting the options with the config
>> setter
>>> (or otherwise using Rocksdb outside of Streams?)
>>> 
>>> Have you tried cleaning up the state between runs? Are you using
>>> interactive queries?
>>> 
>>> Regarding the .checkpoint file not found thing -- this is an annoying but
>>> pretty much harmless bug we only recently figured out. There's an open PR
>>> for it but for now you can just ignore the warning. See KAFKA-5998
>>> > https://issues.apache.org/jira/browse/KAFKA-5998>> for a long history (but
>>> scroll to the bottom for the actual explanation)
>>> 
>>> 
>>> On Thu, Jul 4, 2019 at 11:02 AM Thameem Ansari > > wrote:
>>> 
 i have few more details to share from today’s testing.
 
 Attached strace to the process and noticed that there are thousands of
 short lived child processes have been created by the stream application.
 Not sure whether rocksdb is playing any role here. Noticed more than
>> 73000
 child processes were created and this is after i increased the default
 linux system limit from 32767 to 4million per PID. Appreciate if someone
 answer the following questions.
 1. Why rocksdb creates these many processes? Each process is trying to
 read the contents of the file and getting EAGAIN (Resource temporarily
 unavailable)
 2. Noticed very high number of .checkpoint files missing and hence “No
 such file or directory” message
 3. The child processes were created using “clone” system call. Not sure
 whether there is any significance of this but wanna mention. Moreover
>> these
 child processes were keep on increasing when the load is getting applied
 continuously. Not sure under what condition it will hit the ceiling.
 4. Noticed very high VIRT memory usage (around 31G) but RES memory was
 only 1.9G. Isn’t every file open/close consume memory? But it is not
>> going
 up even though the number of processes is increased.
 
 Thanks
 Thameem
 
> On Jul 4, 2019, at 9:28 AM, Thameem Ansari  wrote:
> 
> Sorry for reposting the previous message as the images didn’t come
 thru.. pasting as text.
> 
> I have changed both system and user limits.
> 
> To completely isolate the problem, I have tried the application in
 Centos 7 environment. Set the ulimit to

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

2019-07-12 Thread Bill Bejeck
Hi Thameem,

> Our topology has 3100 tasks with 3 brokers and 8 partitions.

I have a question that is somewhat orthogonal to your original item.  I may
be missing something, but are you saying your streams application has 3,100
tasks from 8 input partitions? If that is the case, do you mind sharing
your topology? It seems like you have several repartition operations and
you may be able to reduce the number of repartition operations depending on
what you are doing.

Thanks,
Bill

On Thu, Jul 11, 2019 at 11:24 PM Thameem Ansari  wrote:

> Open files in the system was set to 10Million and users limit is 1million.
> When the process was active i was closely watching the open files and it
> was around 400K so its well within the set limit. Rocksdb open files we
> tried setting 100, 50, -1 but the results are same.
> I am using Rocksdb config setter and not options (not sure what it is)
> file.
>
> Quick question: Our topology has 3100 tasks with 3 brokers and 8
> partitions. But we are using only one streaming application instance with
> single thread. Does this cause any issue? Today i am going to try with 2 or
> 3 instances and see whether we can get rid of the issue.
>
> Thanks
> Thameem
>
>
> > On Jul 12, 2019, at 1:57 AM, Sophie Blee-Goldman 
> wrote:
> >
> > I believe the "resource temporarily unavailable" actually is related to
> the
> > open files, most likely you are hitting the total file descriptor limit.
> > Sorry if you mentioned this and I missed it, but what was the
> > max.open.files in your RocksDBConfigSetter when you ran this? Actually,
> > could you just include your full RocksDBConfigSetter implementation?
> >
> > Not sure why Rocks would be spawning so many child processes...it does
> use
> > background threads for compaction but this sounds like something totally
> > different. I also notice a read "This is a RocksDB options file" -- are
> you
> > trying to use an options file vs setting the options with the config
> setter
> > (or otherwise using Rocksdb outside of Streams?)
> >
> > Have you tried cleaning up the state between runs? Are you using
> > interactive queries?
> >
> > Regarding the .checkpoint file not found thing -- this is an annoying but
> > pretty much harmless bug we only recently figured out. There's an open PR
> > for it but for now you can just ignore the warning. See KAFKA-5998
> >  https://issues.apache.org/jira/browse/KAFKA-5998>> for a long history (but
> > scroll to the bottom for the actual explanation)
> >
> >
> > On Thu, Jul 4, 2019 at 11:02 AM Thameem Ansari  > wrote:
> >
> >> i have few more details to share from today’s testing.
> >>
> >> Attached strace to the process and noticed that there are thousands of
> >> short lived child processes have been created by the stream application.
> >> Not sure whether rocksdb is playing any role here. Noticed more than
> 73000
> >> child processes were created and this is after i increased the default
> >> linux system limit from 32767 to 4million per PID. Appreciate if someone
> >> answer the following questions.
> >> 1. Why rocksdb creates these many processes? Each process is trying to
> >> read the contents of the file and getting EAGAIN (Resource temporarily
> >> unavailable)
> >> 2. Noticed very high number of .checkpoint files missing and hence “No
> >> such file or directory” message
> >> 3. The child processes were created using “clone” system call. Not sure
> >> whether there is any significance of this but wanna mention. Moreover
> these
> >> child processes were keep on increasing when the load is getting applied
> >> continuously. Not sure under what condition it will hit the ceiling.
> >> 4. Noticed very high VIRT memory usage (around 31G) but RES memory was
> >> only 1.9G. Isn’t every file open/close consume memory? But it is not
> going
> >> up even though the number of processes is increased.
> >>
> >> Thanks
> >> Thameem
> >>
> >>> On Jul 4, 2019, at 9:28 AM, Thameem Ansari  wrote:
> >>>
> >>> Sorry for reposting the previous message as the images didn’t come
> >> thru.. pasting as text.
> >>>
> >>> I have changed both system and user limits.
> >>>
> >>> To completely isolate the problem, I have tried the application in
> >> Centos 7 environment. Set the ulimit to 1million and system limits to
> >> 10million open files. Now 3 kafka nodes are running in separate servers
> >> each and streaming application is running in a dedicated VM. Now the
> >> application is not explicitly throwing “too many open files” error but
> >> automatically aborted with the message
> >>>
> >>> terminate called after throwing an instance of 'std::system_error'
> >>>  what():  Resource temporarily unavailable
> >>> ./bin/start.sh: line 42:  1100 Aborted
> >>>
> >>> Here is the last few lines from strace output which shows the aborted
> >> message.
> >>>
> >>> 25721 14:58:35
> >>
> open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.I

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

2019-07-11 Thread Thameem Ansari
Open files in the system was set to 10Million and users limit is 1million. When 
the process was active i was closely watching the open files and it was around 
400K so its well within the set limit. Rocksdb open files we tried setting 100, 
50, -1 but the results are same. 
I am using Rocksdb config setter and not options (not sure what it is) file. 

Quick question: Our topology has 3100 tasks with 3 brokers and 8 partitions. 
But we are using only one streaming application instance with single thread. 
Does this cause any issue? Today i am going to try with 2 or 3 instances and 
see whether we can get rid of the issue. 

Thanks
Thameem


> On Jul 12, 2019, at 1:57 AM, Sophie Blee-Goldman  wrote:
> 
> I believe the "resource temporarily unavailable" actually is related to the
> open files, most likely you are hitting the total file descriptor limit.
> Sorry if you mentioned this and I missed it, but what was the
> max.open.files in your RocksDBConfigSetter when you ran this? Actually,
> could you just include your full RocksDBConfigSetter implementation?
> 
> Not sure why Rocks would be spawning so many child processes...it does use
> background threads for compaction but this sounds like something totally
> different. I also notice a read "This is a RocksDB options file" -- are you
> trying to use an options file vs setting the options with the config setter
> (or otherwise using Rocksdb outside of Streams?)
> 
> Have you tried cleaning up the state between runs? Are you using
> interactive queries?
> 
> Regarding the .checkpoint file not found thing -- this is an annoying but
> pretty much harmless bug we only recently figured out. There's an open PR
> for it but for now you can just ignore the warning. See KAFKA-5998
>  > for a long history (but
> scroll to the bottom for the actual explanation)
> 
> 
> On Thu, Jul 4, 2019 at 11:02 AM Thameem Ansari  > wrote:
> 
>> i have few more details to share from today’s testing.
>> 
>> Attached strace to the process and noticed that there are thousands of
>> short lived child processes have been created by the stream application.
>> Not sure whether rocksdb is playing any role here. Noticed more than 73000
>> child processes were created and this is after i increased the default
>> linux system limit from 32767 to 4million per PID. Appreciate if someone
>> answer the following questions.
>> 1. Why rocksdb creates these many processes? Each process is trying to
>> read the contents of the file and getting EAGAIN (Resource temporarily
>> unavailable)
>> 2. Noticed very high number of .checkpoint files missing and hence “No
>> such file or directory” message
>> 3. The child processes were created using “clone” system call. Not sure
>> whether there is any significance of this but wanna mention. Moreover these
>> child processes were keep on increasing when the load is getting applied
>> continuously. Not sure under what condition it will hit the ceiling.
>> 4. Noticed very high VIRT memory usage (around 31G) but RES memory was
>> only 1.9G. Isn’t every file open/close consume memory? But it is not going
>> up even though the number of processes is increased.
>> 
>> Thanks
>> Thameem
>> 
>>> On Jul 4, 2019, at 9:28 AM, Thameem Ansari  wrote:
>>> 
>>> Sorry for reposting the previous message as the images didn’t come
>> thru.. pasting as text.
>>> 
>>> I have changed both system and user limits.
>>> 
>>> To completely isolate the problem, I have tried the application in
>> Centos 7 environment. Set the ulimit to 1million and system limits to
>> 10million open files. Now 3 kafka nodes are running in separate servers
>> each and streaming application is running in a dedicated VM. Now the
>> application is not explicitly throwing “too many open files” error but
>> automatically aborted with the message
>>> 
>>> terminate called after throwing an instance of 'std::system_error'
>>>  what():  Resource temporarily unavailable
>>> ./bin/start.sh: line 42:  1100 Aborted
>>> 
>>> Here is the last few lines from strace output which shows the aborted
>> message.
>>> 
>>> 25721 14:58:35
>> open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN.155520/MANIFEST-07",
>> O_RDONLY|O_CLOEXEC) = 12505
>>> 25721 14:58:35
>> open("/sys/devices/virtual/block/dm-2/queue/logical_block_size", O_RDONLY)
>> = 12506
>>> 25721 14:58:35 read(12506, "512\n", 4096) = 4
>>> 25721 14:58:35 close(12506) = 0
>>> 25721 14:58:35 write(12502, "s.advise_random_on_open: 0\n2019/"...,
>> 4096) = 4096
>>> 25721 14:58:35 write(12502, "ions.comparator: leveldb.Bytewis"..., 4096)
>> = 4096
>>> 25721 14:58:35 read(12505,
>> "V\371\270\370\34\0\1\1\32le

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

2019-07-11 Thread Sophie Blee-Goldman
I believe the "resource temporarily unavailable" actually is related to the
open files, most likely you are hitting the total file descriptor limit.
Sorry if you mentioned this and I missed it, but what was the
max.open.files in your RocksDBConfigSetter when you ran this? Actually,
could you just include your full RocksDBConfigSetter implementation?

Not sure why Rocks would be spawning so many child processes...it does use
background threads for compaction but this sounds like something totally
different. I also notice a read "This is a RocksDB options file" -- are you
trying to use an options file vs setting the options with the config setter
(or otherwise using Rocksdb outside of Streams?)

 Have you tried cleaning up the state between runs? Are you using
interactive queries?

Regarding the .checkpoint file not found thing -- this is an annoying but
pretty much harmless bug we only recently figured out. There's an open PR
for it but for now you can just ignore the warning. See KAFKA-5998
 for a long history (but
scroll to the bottom for the actual explanation)


On Thu, Jul 4, 2019 at 11:02 AM Thameem Ansari  wrote:

> i have few more details to share from today’s testing.
>
> Attached strace to the process and noticed that there are thousands of
> short lived child processes have been created by the stream application.
> Not sure whether rocksdb is playing any role here. Noticed more than 73000
> child processes were created and this is after i increased the default
> linux system limit from 32767 to 4million per PID. Appreciate if someone
> answer the following questions.
> 1. Why rocksdb creates these many processes? Each process is trying to
> read the contents of the file and getting EAGAIN (Resource temporarily
> unavailable)
> 2. Noticed very high number of .checkpoint files missing and hence “No
> such file or directory” message
> 3. The child processes were created using “clone” system call. Not sure
> whether there is any significance of this but wanna mention. Moreover these
> child processes were keep on increasing when the load is getting applied
> continuously. Not sure under what condition it will hit the ceiling.
> 4. Noticed very high VIRT memory usage (around 31G) but RES memory was
> only 1.9G. Isn’t every file open/close consume memory? But it is not going
> up even though the number of processes is increased.
>
> Thanks
> Thameem
>
> > On Jul 4, 2019, at 9:28 AM, Thameem Ansari  wrote:
> >
> > Sorry for reposting the previous message as the images didn’t come
> thru.. pasting as text.
> >
> > I have changed both system and user limits.
> >
> > To completely isolate the problem, I have tried the application in
> Centos 7 environment. Set the ulimit to 1million and system limits to
> 10million open files. Now 3 kafka nodes are running in separate servers
> each and streaming application is running in a dedicated VM. Now the
> application is not explicitly throwing “too many open files” error but
> automatically aborted with the message
> >
> > terminate called after throwing an instance of 'std::system_error'
> >   what():  Resource temporarily unavailable
> > ./bin/start.sh: line 42:  1100 Aborted
> >
> > Here is the last few lines from strace output which shows the aborted
> message.
> >
> > 25721 14:58:35
> open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN.155520/MANIFEST-07",
> O_RDONLY|O_CLOEXEC) = 12505
> > 25721 14:58:35
> open("/sys/devices/virtual/block/dm-2/queue/logical_block_size", O_RDONLY)
> = 12506
> > 25721 14:58:35 read(12506, "512\n", 4096) = 4
> > 25721 14:58:35 close(12506) = 0
> > 25721 14:58:35 write(12502, "s.advise_random_on_open: 0\n2019/"...,
> 4096) = 4096
> > 25721 14:58:35 write(12502, "ions.comparator: leveldb.Bytewis"..., 4096)
> = 4096
> > 25721 14:58:35 read(12505,
> "V\371\270\370\34\0\1\1\32leveldb.BytewiseCompara"..., 32768) = 192
> > 25721 14:58:35 read(12505, "", 28672)   = 0
> > 25721 14:58:35 close(12505) = 0
> > 17701 14:58:35
> open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN.155520/06.sst",
> O_RDONLY|O_CLOEXEC) = 12505
> > 17702 14:58:35 +++ exited with 0 +++
> > 25721 14:58:35 write(2, "terminate called after throwing "..., 48) = 48
> > 25721 14:58:35 write(2, "std::system_error", 17) = 17
> > 25721 14:58:35 write(2, "'\n", 2)   = 2
> > 25721 14:58:35 write(2, "  what():  ", 11) = 11
> > 25721 14:58:35 write(2, "Resource temporarily unavailable", 32) = 32
> > 25721 14:58:35 write(2, "\n", 1)= 1
> > 177

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

2019-07-04 Thread Thameem Ansari
i have few more details to share from today’s testing. 

Attached strace to the process and noticed that there are thousands of short 
lived child processes have been created by the stream application. Not sure 
whether rocksdb is playing any role here. Noticed more than 73000 child 
processes were created and this is after i increased the default linux system 
limit from 32767 to 4million per PID. Appreciate if someone answer the 
following questions. 
1. Why rocksdb creates these many processes? Each process is trying to read the 
contents of the file and getting EAGAIN (Resource temporarily unavailable) 
2. Noticed very high number of .checkpoint files missing and hence “No such 
file or directory” message 
3. The child processes were created using “clone” system call. Not sure whether 
there is any significance of this but wanna mention. Moreover these child 
processes were keep on increasing when the load is getting applied 
continuously. Not sure under what condition it will hit the ceiling.
4. Noticed very high VIRT memory usage (around 31G) but RES memory was only 
1.9G. Isn’t every file open/close consume memory? But it is not going up even 
though the number of processes is increased. 

Thanks
Thameem 

> On Jul 4, 2019, at 9:28 AM, Thameem Ansari  wrote:
> 
> Sorry for reposting the previous message as the images didn’t come thru.. 
> pasting as text. 
> 
> I have changed both system and user limits. 
> 
> To completely isolate the problem, I have tried the application in Centos 7 
> environment. Set the ulimit to 1million and system limits to 10million open 
> files. Now 3 kafka nodes are running in separate servers each and streaming 
> application is running in a dedicated VM. Now the application is not 
> explicitly throwing “too many open files” error but automatically aborted 
> with the message 
> 
> terminate called after throwing an instance of 'std::system_error'
>   what():  Resource temporarily unavailable
> ./bin/start.sh: line 42:  1100 Aborted 
> 
> Here is the last few lines from strace output which shows the aborted 
> message. 
> 
> 25721 14:58:35 
> open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN.155520/MANIFEST-07",
>  O_RDONLY|O_CLOEXEC) = 12505
> 25721 14:58:35 
> open("/sys/devices/virtual/block/dm-2/queue/logical_block_size", O_RDONLY) = 
> 12506
> 25721 14:58:35 read(12506, "512\n", 4096) = 4
> 25721 14:58:35 close(12506) = 0
> 25721 14:58:35 write(12502, "s.advise_random_on_open: 0\n2019/"..., 4096) = 
> 4096
> 25721 14:58:35 write(12502, "ions.comparator: leveldb.Bytewis"..., 4096) = 
> 4096
> 25721 14:58:35 read(12505, 
> "V\371\270\370\34\0\1\1\32leveldb.BytewiseCompara"..., 32768) = 192
> 25721 14:58:35 read(12505, "", 28672)   = 0
> 25721 14:58:35 close(12505) = 0
> 17701 14:58:35 
> open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN.155520/06.sst",
>  O_RDONLY|O_CLOEXEC) = 12505
> 17702 14:58:35 +++ exited with 0 +++
> 25721 14:58:35 write(2, "terminate called after throwing "..., 48) = 48
> 25721 14:58:35 write(2, "std::system_error", 17) = 17
> 25721 14:58:35 write(2, "'\n", 2)   = 2
> 25721 14:58:35 write(2, "  what():  ", 11) = 11
> 25721 14:58:35 write(2, "Resource temporarily unavailable", 32) = 32
> 25721 14:58:35 write(2, "\n", 1)= 1
> 17701 14:58:35 
> open("/sys/devices/virtual/block/dm-2/queue/logical_block_size", O_RDONLY) = 
> 12506
> 25721 14:58:35 write(12502, "ction_dynamic_level_bytes: 0\n201"..., 3672) = 
> 3672
> 25721 14:58:35 --- SIGABRT {si_signo=SIGABRT, si_code=SI_TKILL, si_pid=25697, 
> si_uid=1000} ---
> 17701 14:58:35 read(12506,  
> 17701 14:58:36 +++ killed by SIGABRT +++
> 17700 14:58:36 +++ killed by SIGABRT +++
> 17699 14:58:36 +++ killed by SIGABRT +++
> 
> 
> As I can see from the open files they are way lower (45201) than the max 
> limit and hence can we rule out the “open files” is the root cause? 
> 
> I have also noticed there is lot of EAGAIN (count 3334) and associated 
> “Resource temporarily unavailable” messages as well
> 
> 
> 
> 25732 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) 
> = 59
> 25732 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
> 25732 14:49:23 close(59)= 0
> 25709 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) 
> = 59
> 25709 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
> 25709 14:49:23 close(59)= 0
> 25721 14:49:23 write(50, "\0", 1)   = 1
> 25709 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONL

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

2019-07-03 Thread Thameem Ansari
Sorry for reposting the previous message as the images didn’t come thru.. 
pasting as text. 

I have changed both system and user limits. 

To completely isolate the problem, I have tried the application in Centos 7 
environment. Set the ulimit to 1million and system limits to 10million open 
files. Now 3 kafka nodes are running in separate servers each and streaming 
application is running in a dedicated VM. Now the application is not explicitly 
throwing “too many open files” error but automatically aborted with the message 

terminate called after throwing an instance of 'std::system_error'
  what():  Resource temporarily unavailable
./bin/start.sh: line 42:  1100 Aborted 

Here is the last few lines from strace output which shows the aborted message. 

25721 14:58:35 
open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN.155520/MANIFEST-07",
 O_RDONLY|O_CLOEXEC) = 12505
25721 14:58:35 open("/sys/devices/virtual/block/dm-2/queue/logical_block_size", 
O_RDONLY) = 12506
25721 14:58:35 read(12506, "512\n", 4096) = 4
25721 14:58:35 close(12506) = 0
25721 14:58:35 write(12502, "s.advise_random_on_open: 0\n2019/"..., 4096) = 4096
25721 14:58:35 write(12502, "ions.comparator: leveldb.Bytewis"..., 4096) = 4096
25721 14:58:35 read(12505, 
"V\371\270\370\34\0\1\1\32leveldb.BytewiseCompara"..., 32768) = 192
25721 14:58:35 read(12505, "", 28672)   = 0
25721 14:58:35 close(12505) = 0
17701 14:58:35 
open("/home/devops/data/kafka-streams/RawLog_Processor/393_7/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN/MAIL.INCOMING_FILTER_BY_USER_MAIL_INCOMING_DOMAIN_TRAFFIC_DETAIL.DETECTED_CONTENT_FILTER_RECIPIENT_DOMAIN.155520/06.sst",
 O_RDONLY|O_CLOEXEC) = 12505
17702 14:58:35 +++ exited with 0 +++
25721 14:58:35 write(2, "terminate called after throwing "..., 48) = 48
25721 14:58:35 write(2, "std::system_error", 17) = 17
25721 14:58:35 write(2, "'\n", 2)   = 2
25721 14:58:35 write(2, "  what():  ", 11) = 11
25721 14:58:35 write(2, "Resource temporarily unavailable", 32) = 32
25721 14:58:35 write(2, "\n", 1)= 1
17701 14:58:35 open("/sys/devices/virtual/block/dm-2/queue/logical_block_size", 
O_RDONLY) = 12506
25721 14:58:35 write(12502, "ction_dynamic_level_bytes: 0\n201"..., 3672) = 3672
25721 14:58:35 --- SIGABRT {si_signo=SIGABRT, si_code=SI_TKILL, si_pid=25697, 
si_uid=1000} ---
17701 14:58:35 read(12506,  
17701 14:58:36 +++ killed by SIGABRT +++
17700 14:58:36 +++ killed by SIGABRT +++
17699 14:58:36 +++ killed by SIGABRT +++


As I can see from the open files they are way lower (45201) than the max limit 
and hence can we rule out the “open files” is the root cause? 

I have also noticed there is lot of EAGAIN (count 3334) and associated 
“Resource temporarily unavailable” messages as well



25732 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 
59
25732 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
25732 14:49:23 close(59)= 0
25709 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 
59
25709 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
25709 14:49:23 close(59)= 0
25721 14:49:23 write(50, "\0", 1)   = 1
25709 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 
59
25709 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
25709 14:49:23 close(59)= 0
25721 14:49:23 read(17, 
"PK\3\4\n\0\0\10\10\0\343SHNM\n4>Q\2\0\0\265\5\0\0A\0\0\0", 30) = 30
25721 14:49:23 read(17, 
"\245T]O\23A\24=\323\226\326-\253\255\250\365\v\4\265ji\221E\344\305\200M\f\201\304\244j"...,
 593) = 593
25709 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 
59
25709 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
25709 14:49:23 close(59)= 0
25732 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 
59
25732 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
25732 14:49:23 close(59)= 0
25721 14:49:23 read(49, "\0", 16)   = 1
25709 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 
59
25709 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
25709 14:49:23 close(59)= 0
25732 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 
59
25732 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
25732 14:49:23 close(59)= 0
25709 14:49:23 open("/sys/fs/cgroup/memory/memory.limit_in_bytes", O_RDONLY) = 
59
25709 14:49:23 read(59, "9223372036854775807\n", 4096) = 20
25709 14:49:23 close(59)= 0
25721 14:49:23 read(54, "\0\1dx", 4)= 4
25721 14:49:23 read(54, 
"\0\0\0\v\0\0\0\0\0\0\0\1dj\0\0\0\0\1\212\0\201RawLog_Pro"...,

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

2019-07-03 Thread Sophie Blee-Goldman
Also, when you say you changed the OS limit are you referring to the system
limit or the user limit or both? If you increased one but not the other you
may still be hitting the lower limit.

On Wed, Jul 3, 2019 at 1:53 AM Patrik Kleindl  wrote:

> Hi
> Try to set it really low like Sophie suggested.
> You can verify if the settings take effect by checking the files in the
> rocksdb directories, I think it should be somewhere in OPTIONS or LOG
> br, Patrik
>
> > Am 03.07.2019 um 09:37 schrieb Thameem Ansari :
> >
> > Tried setting the open files to 100 and 50 but the results are same. I
> checked the total open files while the streaming application was busy
> running just before getting the “too many open files” message it was around
> 41756 which is same as what we have got when we set to -1.
> >
> > VisualVM shows that there is no abnormality with the threads / memory or
> heap.
> >
> > Thanks
> > Thameem
> >
> >> On Jul 3, 2019, at 11:50 AM, Sophie Blee-Goldman 
> wrote:
> >>
> >> How sure are you that the open file count never goes beyond 50K? Are
> those
> >> numbers just from a snapshot after it crashed? It's possible rocks is
> >> creating a large number of files just for a short period of time (maybe
> >> while compacting) that causes the open file count to spike and go back
> down.
> >>
> >> For things to try, you should set the rocks config max.open.files to
> >> something less than infinity...if you're OS limit is 1 million and you
> have
> >> (rounding up) ~5k rocks instances, set this to 1 million / 5k = 200. If
> you
> >> set a lower limit and still hit this error, we can go from there
> >>
> >> On Tue, Jul 2, 2019 at 11:10 PM emailtokir...@gmail.com <
> >> emailtokir...@gmail.com> wrote:
> >>
> >>>
> >>>
>  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 m

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

2019-07-03 Thread Patrik Kleindl
Hi
Try to set it really low like Sophie suggested.
You can verify if the settings take effect by checking the files in the rocksdb 
directories, I think it should be somewhere in OPTIONS or LOG
br, Patrik 

> Am 03.07.2019 um 09:37 schrieb Thameem Ansari :
> 
> Tried setting the open files to 100 and 50 but the results are same. I 
> checked the total open files while the streaming application was busy running 
> just before getting the “too many open files” message it was around 41756 
> which is same as what we have got when we set to -1. 
> 
> VisualVM shows that there is no abnormality with the threads / memory or 
> heap. 
> 
> Thanks
> Thameem
> 
>> On Jul 3, 2019, at 11:50 AM, Sophie Blee-Goldman  wrote:
>> 
>> How sure are you that the open file count never goes beyond 50K? Are those
>> numbers just from a snapshot after it crashed? It's possible rocks is
>> creating a large number of files just for a short period of time (maybe
>> while compacting) that causes the open file count to spike and go back down.
>> 
>> For things to try, you should set the rocks config max.open.files to
>> something less than infinity...if you're OS limit is 1 million and you have
>> (rounding up) ~5k rocks instances, set this to 1 million / 5k = 200. If you
>> set a lower limit and still hit this error, we can go from there
>> 
>> On Tue, Jul 2, 2019 at 11:10 PM emailtokir...@gmail.com <
>> emailtokir...@gmail.com> wrote:
>> 
>>> 
>>> 
 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
>>> u

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

2019-07-03 Thread Thameem Ansari
Tried setting the open files to 100 and 50 but the results are same. I checked 
the total open files while the streaming application was busy running just 
before getting the “too many open files” message it was around 41756 which is 
same as what we have got when we set to -1. 

VisualVM shows that there is no abnormality with the threads / memory or heap. 

Thanks
Thameem

> On Jul 3, 2019, at 11:50 AM, Sophie Blee-Goldman  wrote:
> 
> How sure are you that the open file count never goes beyond 50K? Are those
> numbers just from a snapshot after it crashed? It's possible rocks is
> creating a large number of files just for a short period of time (maybe
> while compacting) that causes the open file count to spike and go back down.
> 
> For things to try, you should set the rocks config max.open.files to
> something less than infinity...if you're OS limit is 1 million and you have
> (rounding up) ~5k rocks instances, set this to 1 million / 5k = 200. If you
> set a lower limit and still hit this error, we can go from there
> 
> On Tue, Jul 2, 2019 at 11:10 PM emailtokir...@gmail.com <
> emailtokir...@gmail.com> wrote:
> 
>> 
>> 
>> 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 Kleindl 
 wrote:
>> Hi Kiran
>> You can use the RocksDBConfigSetter and pass
>

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

2019-07-02 Thread Sophie Blee-Goldman
How sure are you that the open file count never goes beyond 50K? Are those
numbers just from a snapshot after it crashed? It's possible rocks is
creating a large number of files just for a short period of time (maybe
while compacting) that causes the open file count to spike and go back down.

For things to try, you should set the rocks config max.open.files to
something less than infinity...if you're OS limit is 1 million and you have
(rounding up) ~5k rocks instances, set this to 1 million / 5k = 200. If you
set a lower limit and still hit this error, we can go from there

On Tue, Jul 2, 2019 at 11:10 PM emailtokir...@gmail.com <
emailtokir...@gmail.com> wrote:

>
>
> 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 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.
> > > > > > > > >
> >

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 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 Thameem Ansari
As I mentioned, tried setting the OS limit to 600K & 1Million on the shell and 
tried to start the application on the same shell but still the problem exists. 
Tried rebooting the laptop and the results are same. So, need a way to find out 
what exactly is causing this issue when we hit close to 42K system limit. 

Thanks
Thameem


> On Jul 3, 2019, at 11:16 AM, 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 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
>>>

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

2019-07-02 Thread Sophie Blee-Goldman
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 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:
> > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> --
> > > > > > > > Caused by: org.rocksdb.RocksDBException: while open a file
> for
> > > > lock:
> > > > > > > > PPP.151200/LOCK: Too many open files
> > > > > > > > at org.rocksdb.RocksDB.open(Native Method)
> > > > > > > > at org.rocksdb.RocksDB.open(RocksDB.java:235)
> > >

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

2019-07-02 Thread Thameem Ansari
Many places it is mentioned that closing the iterator is fixing the issue but 
this is true only if we use Processor APIs. But in DSL there is no iterator 
explicitly available and we are using wrapper methods like aggregate, map, 
groupBy, etc. 

Here is the snapshot of the issue with exact statistics observed from the 
recent run in Mac with Mojave. 
 
- Number of state directories are 3154 and hence there will be 3154 rocksdb 
instances 
 - OS openfiles limit was set to 1Million, here is the break out on number of 
open files: 
Total open files from the system: 41645
Open files from stream application: 16545
Open files related to state directories: 16025
So, if we do the math 16025/3154 ~ 5 files per instance 

 - Following parameters were used but still problem exists 
cache.index.and.filter.blocks=false 
block.cache.size=100MB
code block.size=8MB
max.write.buffer.number=2 
table.cache.numshardbits=8 
max.open.files=-1 
compaction.readahead.size=256MB 
skip.stats.update_on_db_open=true 
write.buffer.size=32MB
 - Topic has 8 partitions and the streaming application is running as SINGLE 
instance with SINGLE thread 
 - Noticed these rocksdb properties have been consumed by the app but not 
working as expected (or defined in the documentation)
 - Observed no issues reported related to memory  

Thanks
Thameem


On 2019/07/03 02:53:20, "e...@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 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> 
> > > > > > >

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:
> > > > > > >
> > > > > > >
> > > > >
> > > --
> > > > > > > Caused by: org.rocksdb.RocksDBException: while open a file for
> > > lock:
> > > > > > > PPP.151200/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 XXX.151200
> > > > > > > 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)
> > > > 

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

2019-07-02 Thread Sophie Blee-Goldman
This can also happen if you have any open iterators that you forget to
close (for example using IQ), although that's probably not what's going on
here since 3152 is certainly a lot of rocks instances for a single fs.

There's no default number of open files per instance, since rocks creates
new files on new levels as you add more data. To understand the impact you
should check out the description of *max_open_files* here

--
but you will probably want to increase your system limit in addition to
constraining the number of open files per instance

On Mon, Jul 1, 2019 at 9:58 AM 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 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:
> > > > > > > >
> > > > > > > >
> > > > > >
> > > >
> --
> > > > > > > > Caused by: org.rocksdb.RocksDBException: while open a file
> for
> > > > lock:
> > > > > > > > PPP.151200/LOCK: Too many open files
> > > > > > > > at or

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

2019-07-01 Thread emailtokirank



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:
> > > > > > >
> > > > > > >
> > > > >
> > > --
> > > > > > > Caused by: org.rocksdb.RocksDBException: while open a file for
> > > lock:
> > > > > > > PPP.151200/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 XXX.151200
> > > > > > > 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)
> > > > 

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

2019-06-28 Thread John Roesler
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:
> > > > > >
> > > > > >
> > > >
> > --
> > > > > > Caused by: org.rocksdb.RocksDBException: while open a file for
> > lock:
> > > > > > PPP.151200/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 XXX.151200
> > > > > > 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
> > > > > >
>

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

2019-06-27 Thread Patrik Kleindl
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:
> > > > >
> > > > >
> > >
> --
> > > > > Caused by: org.rocksdb.RocksDBException: while open a file for
> lock:
> > > > > PPP.151200/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 XXX.151200
> > > > > 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:
> > > > > Y.151200/07.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)
> > > > >
> > > > >
> > >
> 

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

2019-06-27 Thread emailtokirank



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:
> > > >
> > > >
> > --
> > > > Caused by: org.rocksdb.RocksDBException: while open a file for lock:
> > > > PPP.151200/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 XXX.151200
> > > > 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:
> > > > Y.151200/07.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
> > > >
> > > >
> > >
> >
> > Hi Patrik/All,
> >
> > Thanks for providing some valuable pointer!
> >
> > I did that & it doesn't seems to work.
> >
> > Here is how my custom config setter looks like:
> >
> >
> > 

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

2019-06-27 Thread Patrik Kleindl
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:
> > >
> > >
> --
> > > Caused by: org.rocksdb.RocksDBException: while open a file for lock:
> > > PPP.151200/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 XXX.151200
> > > 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:
> > > Y.151200/07.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
> > >
> > >
> >
>
> Hi Patrik/All,
>
> Thanks for providing some valuable pointer!
>
> I did that & it doesn't seems to work.
>
> Here is how my custom config setter looks like:
>
>
> 
>
>
> @Override
>   public void setConfig(final String storeName, final Options
> options, final Map configs) {
> // See #1 below.
> BlockBasedTableConfig tableConfig = new
> org.rocksdb.BlockBasedTableConfig();
>
> tableConfig.setBlockCacheSize(16 * 1024 * 1024L);
> // See 

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

2019-06-27 Thread emailtokirank



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:
> >
> > --
> > Caused by: org.rocksdb.RocksDBException: while open a file for lock:
> > PPP.151200/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 XXX.151200
> > 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:
> > Y.151200/07.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
> >
> >
> 

Hi Patrik/All,

Thanks for providing some valuable pointer!

I did that & it doesn't seems to work. 

Here is how my custom config setter looks like:




@Override
  public void setConfig(final String storeName, final Options options, 
final Map configs) {
// See #1 below.
BlockBasedTableConfig tableConfig = new 
org.rocksdb.BlockBasedTableConfig();
   
tableConfig.setBlockCacheSize(16 * 1024 * 1024L);
// See #2 below.
tableConfig.setBlockSize(16 * 1024L);
// See #3 below.
tableConfig.setCacheIndexAndFilterBlocks(false);
   // tableConfig.setPinL0FilterAndIndexBlocksInCache(true);
options.setMaxOpenFiles(-1);
options.setTableFormatConfig(tableConfig);
// See #4 below.
options.setMaxWriteBufferNumber(2);
  }

I tried many options with this:
1. tableConfig.setCacheIndexAndFilterBlocks(true);  > as per docs 
(https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB#indexes-and-filter-blocks)
 if we set to true, the max_open_files shouldn't play a role. But I was still 
getting too many open files exception from Rocksdb

2. tableConfig.setCacheIndexAndFilterBlocks(true); 
tableConfig.setPinL0FilterAndIndexBlocksInCache(true); 

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

2019-06-26 Thread Patrik Kleindl
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:
> PPP.151200/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 XXX.151200
> 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:
> Y.151200/07.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
>
>


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

2019-06-26 Thread emailtokirank
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: 
PPP.151200/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 XXX.151200
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: 
Y.151200/07.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