Hello Ashok

Adding to what Sophie wrote, if you use a custom RocksDBConfigSetter then
override the BlockBasedTableConfig like following and call
options.setTableFormatConfig(tableConfig)
at the end.

BlockBasedTableConfig tableConfig = (BlockBasedTableConfig)
options.tableFormatConfig();
tableConfig.setBlockCacheSize(100*1024*1024L);
tableConfig.setBlockSize(8*1024L);
options.setTableFormatConfig(tableConfig);

best regards

Patrik


On Tue, 16 Jul 2019 at 23:23, Sophie Blee-Goldman <sop...@confluent.io>
wrote:

> Hi Ashok,
>
> 1) RocksDB uses memory in four ways, one of which (iterators) *should* be
> negligible -- however if you have a very large number of them open at any
> one time, they can consume a lot of memory (until they are closed). If you
> are opening many iterators throughout the day, consider closing them more
> frequently than once a day.
>
> 2) The other three users of memory are: index & filter blocks, block cache,
> and memtable (write buffer). You can limit the memory usage of index/filter
> blocks by setting tableConfig.setCacheIndexAndFilterBlocks(true) -- this
> will cause these blocks to be stored in the block cache alongside data
> blocks (and be evicted to disk when full). If you do this I would suggest
> also setting tableConfig.setPinL0FilterAndIndexBlocksInCache(true). You can
> then control the off-heap memory usage by setting the block cache size, and
> write buffer size + write buffer number.
>
> 3) By disabling the RocksDBConfigSetter, you aren't disabling the RocksDB
> cache (or other users of memory), you are reverting to the defaults set by
> Streams (not sure if that's what you expect or not) -- Rocks will continue
> to use off-heap memory
>
> 4) RocksDB objects are backed by C++ objects, so you need to actually close
> some objects you construct to free the memory. Since you construct a
> BloomFilter in your config setter but never close it, you are leaking
> memory. Unfortunately we did not have a RocksDBConfigSetter#close method in
> 1.0.0, but there is one as of 2.3 -- you should either remove the
> BloomFilter from your config setter or consider upgrading to 2.3 (or 2.2,
> which already uses a BloomFilter -- you can just get/update the existing
> BlockBasedTableConfig instead of creating a new one to utilize the
> BloomFilter)
>
> 5) The settings from the config setter (or Streams defaults) are actually
> per rocksdb instance, not per Streams instance. If you have a very large
> number of stores, you may hit OOM even with the relatively conservative
> defaults Streams uses. If you have a large number of stores in your
> subtopology, or a large number of partitions all being read by the same
> instance, the total off-heap memory will be quite large. (If you're able to
> upgrade to 2.3, you can actually use the config setter to limit the total
> memory across all stores rather than on a per-store basis)
>
> If you don't have a large number of stores on an instance, don't open a
> large number of iterators at a time, and still hit OOM over 100GB even with
> the default rocks configs, there may be a memory leak. But I would first
> try setting the configs suggested in 2) above, with a smaller write
> buffer/block cache size (and no bloom filter)
>
> Cheers,
> Sophie
>
> On Tue, Jul 16, 2019 at 12:21 PM Jayaraman, AshokKumar (CCI-Atlanta-CON) <
> ashokkumar.jayara...@cox.com> wrote:
>
> > Hi,
> >
> > In our streaming instance, the internal caching has been disabled and
> > RocksDB caching has been enabled, with the override as shown below.
> > Although the heap is restricted to 36GB, the memory utilization is going
> > over 100GB in a week and eventually runs out of memory.  As part of the
> > profiling, we have confirmed that the garbage collection process is
> within
> > the within the limit 36GB (on-heap).  However, the additional memory
> > utilization is not appearing within the profiling and is the one we
> > suspect, growing unbounded (off-heap).
> >
> > We have also tried enabling the streams caching (5GB) and disabling the
> > RocksDB config setter (commented as below).  However, we are still seeing
> > the similar behaviour where the memory is growing unlimited overtime.
>  We
> > process 20 million records each 20 minutes (a message size - 1KB) on an
> > average.  Can you please review and advise what could cause this
> behavior?
> > We have ensured that the iterators are closed (which happens once a day).
> >
> > //streamsConfig.put(StreamsConfig.ROCKSDB_CONFIG_SETTER_CLASS_CONFIG,
> > RocksDBOverride.class)
> >
> > Kafka Broker / Kafka Stream version: 1.0.0
> > Rocks DB: 5.7.3
> >
> > Command:
> > java -Xms12g -Xmx36g -XX:MetaspaceSize=576m -XX:+UseG1GC
> > -XX:ParallelGCThreads=8 -XX:MaxGCPauseMillis=80
> > -XX:InitiatingHeapOccupancyPercent=35 -XX:G1HeapRegionSize=16M
> > -XX:MinMetaspaceFreeRatio=50 -XX:MaxMetaspaceFreeRatio=80 -cp
> > /scripts/device_metrics.jar:/libs/kafka/*
> > -Dlog4j.configuration=file:/cfg/device_metrics_log4j.properties
> > org.ssd.devicemetrics /cfg/device_metrics.properties
> >
> >
> > Rocks DB Config setter
> >              BlockBasedTableConfig tableConfig = new
> > org.rocksdb.BlockBasedTableConfig();
> >              BloomFilter bloomFilter = new BloomFilter();
> >              tableConfig.setBlockCacheSize(512MB);
> >              tableConfig.setBlockSize(64KB);
> >              tableConfig.setCacheIndexAndFilterBlocks(false);
> >              tableConfig.setFilter(bloomFilter);
> >
> >              options.setTableFormatConfig(tableConfig);
> >              options.setWriteBufferSize(512MB);
> >              options.setMaxWriteBufferNumber(5);
> >              options.setCompressionType(CompressionType.LZ4_COMPRESSION);
> >
> > Thanks,
> > Ashok
> >
> >
>

Reply via email to