mirror of https://github.com/apache/kafka.git
MINOR: Updated configuration docs with RocksDBConfigSetter#close (#6784)
The old docs here used a now deprecated method to set the block cache size. In switching over to the new one we would now need to construct a Cache object and therefore also need to close it, so this is a good opportunity to demonstrate the RocksDBConfigSetter#close method that will need to be implemented by users. Reviewers: Guozhang Wang <wangguoz@gmail.com>
This commit is contained in:
parent
d286051a21
commit
105264b5cd
|
@ -684,19 +684,32 @@
|
|||
<p>Here is an example that adjusts the memory size consumed by RocksDB.</p>
|
||||
<div class="highlight-java"><div class="highlight"><pre><span></span> <span class="kd">public</span> <span class="kd">static</span> <span class="kd">class</span> <span class="nc">CustomRocksDBConfig</span> <span class="kd">implements</span> <span class="n">RocksDBConfigSetter</span> <span class="o">{</span>
|
||||
|
||||
<span class="c1">// These objects should be member variables so they can be closed in RocksDBConfigSetter#close.</span>
|
||||
<span class="kd">private</span> <span class="n">org.rocksdb.Cache</span> <span class="n">cache</span> <span class="o">=</span> <span class="k">new</span> <span class="n">org</span><span class="o">.</span><span class="na">rocksdb</span><span class="o">.</span><span class="na">LRUCache</span><span class="o">(</span><span class="mi">16</span> <span class="o">*</span> <span class="mi">1024L</span> <span class="o">*</span> <span class="mi">1024L</span><span class="o">);</span>
|
||||
<span class="kd">private</span> <span class="n">org.rocksdb.Filter</span> <span class="n">filter</span> <span class="o">=</span> <span class="k">new</span> <span class="n">org</span><span class="o">.</span><span class="na">rocksdb</span><span class="o">.</span><span class="na">BloomFilter</span><span class="o">();</span>
|
||||
|
||||
<span class="nd">@Override</span>
|
||||
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">setConfig</span><span class="o">(</span><span class="kd">final</span> <span class="n">String</span> <span class="n">storeName</span><span class="o">,</span> <span class="kd">final</span> <span class="n">Options</span> <span class="n">options</span><span class="o">,</span> <span class="kd">final</span> <span class="n">Map</span><span class="o"><</span><span class="n">String</span><span class="o">,</span> <span class="n">Object</span><span class="o">></span> <span class="n">configs</span><span class="o">)</span> <span class="o">{</span>
|
||||
<span class="c1">// See #1 below.</span>
|
||||
<span class="n">BlockBasedTableConfig</span> <span class="n">tableConfig</span> <span class="o">=</span> <span class="k">new</span> <span class="n">org</span><span class="o">.</span><span class="na">rocksdb</span><span class="o">.</span><span class="na">BlockBasedTableConfig</span><span class="o">();</span>
|
||||
<span class="n">tableConfig</span><span class="o">.</span><span class="na">setBlockCacheSize</span><span class="o">(</span><span class="mi">16</span> <span class="o">*</span> <span class="mi">1024</span> <span class="o">*</span> <span class="mi">1024L</span><span class="o">);</span>
|
||||
<span class="n">tableConfig</span><span class="o">.</span><span class="na">setBlockCache</span><span class="o">(</span><span class="mi">cache</span></span><span class="o">);</span>
|
||||
<span class="c1">// See #2 below.</span>
|
||||
<span class="n">tableConfig</span><span class="o">.</span><span class="na">setBlockSize</span><span class="o">(</span><span class="mi">16</span> <span class="o">*</span> <span class="mi">1024L</span><span class="o">);</span>
|
||||
<span class="c1">// See #3 below.</span>
|
||||
<span class="n">tableConfig</span><span class="o">.</span><span class="na">setCacheIndexAndFilterBlocks</span><span class="o">(</span><span class="kc">true</span><span class="o">);</span>
|
||||
<span class="c1">// See #4 below.</span>
|
||||
<span class="n">tableConfig</span><span class="o">.</span><span class="na">setFilter</span><span class="o">(</span><span class="kc">filter</span><span class="o">);</span>
|
||||
<span class="n">options</span><span class="o">.</span><span class="na">setTableFormatConfig</span><span class="o">(</span><span class="n">tableConfig</span><span class="o">);</span>
|
||||
<span class="c1">// See #4 below.</span>
|
||||
<span class="c1">// See #5 below.</span>
|
||||
<span class="n">options</span><span class="o">.</span><span class="na">setMaxWriteBufferNumber</span><span class="o">(</span><span class="mi">2</span><span class="o">);</span>
|
||||
<span class="o">}</span>
|
||||
|
||||
<span class="nd">@Override</span>
|
||||
<span class="kd">public</span> <span class="kt">void</span> <span class="nf">close</span><span class="o">(</span><span class="kd">final</span> <span class="n">String</span> <span class="n">storeName</span><span class="o">,</span> <span class="kd">final</span> <span class="n">Options</span> <span class="n">options</span><span class="o">)</span> <span class="o">{</span>
|
||||
<span class="c1">// See #6 below.</span>
|
||||
<span class="n">cache</span><span class="o">.</span><span class="na">close</span><span class="o">();</span>
|
||||
<span class="n">filter</span><span class="o">.</span><span class="na">close</span><span class="o">();</span>
|
||||
<span class="o">}</span>
|
||||
<span class="o">}</span>
|
||||
|
||||
<span class="n">Properties</span> <span class="n">streamsSettings</span> <span class="o">=</span> <span class="k">new</span> <span class="n">Properties</span><span class="o">();</span>
|
||||
|
@ -706,10 +719,12 @@
|
|||
<dl class="docutils">
|
||||
<dt>Notes for example:</dt>
|
||||
<dd><ol class="first last arabic simple">
|
||||
<li><code class="docutils literal"><span class="pre">BlockBasedTableConfig</span> <span class="pre">tableConfig</span> <span class="pre">=</span> <span class="pre">new</span> <span class="pre">org.rocksdb.BlockBasedTableConfig();</span></code> Reduce block cache size from the default, shown <a class="reference external" href="https://github.com/apache/kafka/blob/1.0/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java#L81">here</a>, as the total number of store RocksDB databases is partitions (40) * segments (3) = 120.</li>
|
||||
<li><code class="docutils literal"><span class="pre">tableConfig.setBlockSize(16</span> <span class="pre">*</span> <span class="pre">1024L);</span></code> Modify the default <a class="reference external" href="https://github.com/apache/kafka/blob/1.0/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java#L82">block size</a> per these instructions from the <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB#indexes-and-filter-blocks">RocksDB GitHub</a>.</li>
|
||||
<li><code class="docutils literal"><span class="pre">BlockBasedTableConfig</span> <span class="pre">tableConfig</span> <span class="pre">=</span> <span class="pre">new</span> <span class="pre">org.rocksdb.BlockBasedTableConfig();</span></code> Reduce block cache size from the default, shown <a class="reference external" href="https://github.com/apache/kafka/blob/2.3/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java#L78">here</a>, as the total number of store RocksDB databases is partitions (40) * segments (3) = 120.</li>
|
||||
<li><code class="docutils literal"><span class="pre">tableConfig.setBlockSize(16</span> <span class="pre">*</span> <span class="pre">1024L);</span></code> Modify the default <a class="reference external" href="https://github.com/apache/kafka/blob/2.3/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java#L79">block size</a> per these instructions from the <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/Memory-usage-in-RocksDB#indexes-and-filter-blocks">RocksDB GitHub</a>.</li>
|
||||
<li><code class="docutils literal"><span class="pre">tableConfig.setCacheIndexAndFilterBlocks(true);</span></code> Do not let the index and filter blocks grow unbounded. For more information, see the <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/Block-Cache#caching-index-and-filter-blocks">RocksDB GitHub</a>.</li>
|
||||
<li><code class="docutils literal"><span class="pre">tableConfig.setFilter(filter);</span></code> Creating/setting a new BlockBasedTableConfig overwrites the BloomFilter Streams uses by default. This is an important optimization and you should consider (re)setting it if you construct your own tableConfig. See <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/RocksDB-Bloom-Filter">RocksDB Bloom Filter docs</a> for details.</li>
|
||||
<li><code class="docutils literal"><span class="pre">options.setMaxWriteBufferNumber(2);</span></code> See the advanced options in the <a class="reference external" href="https://github.com/facebook/rocksdb/blob/8dee8cad9ee6b70fd6e1a5989a8156650a70c04f/include/rocksdb/advanced_options.h#L103">RocksDB GitHub</a>.</li>
|
||||
<li><code class="docutils literal"><span class="pre">cache.close();</span></code> You must implement close to free the memory of any objects that extend org.rocksdb.RocksObject that you constructed. See <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/RocksJava-Basics#memory-management">RocksJava docs</a> for more details.</li>
|
||||
</ol>
|
||||
</dd>
|
||||
</dl>
|
||||
|
|
Loading…
Reference in New Issue