Hi, I decided to go for the option in which metrics are exposed for each logical state store. I revisited the KIP correspondingly and added a section on how to aggregate metrics over multiple physical RocksDB instances within one logical state store. Would be great, if you could take a look and give feedback. If nobody has complaints about the chosen option I would proceed with voting on this KIP since this was the last open question.
Best, Bruno On Fri, Jun 7, 2019 at 9:38 PM Patrik Kleindl <pklei...@gmail.com> wrote: > > Hi Sophie > This will be a good change, I have been thinking about proposing something > similar or even passing the properties per store. > RocksDB should probably know how much memory was reserved but maybe does not > expose it. > We are limiting it already as you suggested but this is a rather crude tool. > Especially in a larger topology with mixed loads par topic it would be > helpful to get more insights which store puts a lot of load on memory. > Regarding the limiting capability, I think I remember reading that those only > affect some parts of the memory and others can still exceed this limit. I‘ll > try to look up the difference. > Best regards > Patrik > > > Am 07.06.2019 um 21:03 schrieb Sophie Blee-Goldman <sop...@confluent.io>: > > > > Hi Patrik, > > > > As of 2.3 you will be able to use the RocksDBConfigSetter to effectively > > bound the total memory used by RocksDB for a single app instance. You > > should already be able to limit the memory used per rocksdb store, though > > as you mention there can be a lot of them. I'm not sure you can monitor the > > memory usage if you are not limiting it though. > > > >> On Fri, Jun 7, 2019 at 2:06 AM Patrik Kleindl <pklei...@gmail.com> wrote: > >> > >> Hi > >> Thanks Bruno for the KIP, this is a very good idea. > >> > >> I have one question, are there metrics available for the memory consumption > >> of RocksDB? > >> As they are running outside the JVM we have run into issues because they > >> were using all the other memory. > >> And with multiple streams applications on the same machine, each with > >> several KTables and 10+ partitions per topic the number of stores can get > >> out of hand pretty easily. > >> Or did I miss something obvious how those can be monitored better? > >> > >> best regards > >> > >> Patrik > >> > >>> On Fri, 17 May 2019 at 23:54, Bruno Cadonna <br...@confluent.io> wrote: > >>> > >>> Hi all, > >>> > >>> this KIP describes the extension of the Kafka Streams' metrics to include > >>> RocksDB's internal statistics. > >>> > >>> Please have a look at it and let me know what you think. Since I am not a > >>> RocksDB expert, I am thankful for any additional pair of eyes that > >>> evaluates this KIP. > >>> > >>> > >>> > >> https://cwiki.apache.org/confluence/display/KAFKA/KIP-471:+Expose+RocksDB+Metrics+in+Kafka+Streams > >>> > >>> Best regards, > >>> Bruno > >>> > >>