kafka-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From bbej...@apache.org
Subject [kafka] branch trunk updated: MINOR: improve RocksDBConfigSetter docs (#7009)
Date Mon, 08 Jul 2019 23:11:33 GMT
This is an automated email from the ASF dual-hosted git repository.

bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 102490e  MINOR: improve RocksDBConfigSetter docs (#7009)
102490e is described below

commit 102490ea0caba93631c4dc7ea05a64c0117331b7
Author: A. Sophie Blee-Goldman <sophie@confluent.io>
AuthorDate: Mon Jul 8 16:11:17 2019 -0700

    MINOR: improve RocksDBConfigSetter docs (#7009)
    
    Users often use the RocksDBConfigSetter to modify parameters such as cache or block size,
which must be set through the BlockBasedTableConfig object. Rather than creating a new object
in the config setter, however, users should most likely retrieve a reference to the existing
one so as to not lose the other defaults (eg the BloomFilter)
    
    There have been notes from the community that it is not obvious this should be done, nor
is it immediately clear how to do so. This PR updates the RocksDBConfigSetter docs to hopefully
improve things.
    
    I also piggybacked a few minor cleanups in the docs
    
    Reviewers: Kamal Chandraprakash, Jim Galasyn <jim.galasyn@confluent.io>,  Bruno
Cadonna <bruno@confluent.io>, Bill Bejeck <bbejeck@gmail.com>
---
 docs/streams/developer-guide/config-streams.html      | 17 ++++++-----------
 docs/streams/developer-guide/memory-mgmt.html         | 19 ++++++++-----------
 .../kafka/streams/state/RocksDBConfigSetter.java      |  5 +++--
 3 files changed, 17 insertions(+), 24 deletions(-)

diff --git a/docs/streams/developer-guide/config-streams.html b/docs/streams/developer-guide/config-streams.html
index 8e7f950..9bd82bd 100644
--- a/docs/streams/developer-guide/config-streams.html
+++ b/docs/streams/developer-guide/config-streams.html
@@ -684,31 +684,27 @@
               <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="c1">// This object should be a member variable so it 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">&lt;</span><span
class="n">String</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">BlockBasedTableConfig</span> <span class="n">tableConfig</span>
<span class="o">=</span> <span class="k">(BlockBasedTableConfig)</span>
<span class="n">options</span><span><span class="o">.</span><span
class="na">tableFormatConfig</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 #5 below.</span>
+         <span class="c1">// See #4 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="c1">// See #5 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>
 
@@ -719,12 +715,11 @@
               <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/2.3/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java#L78">here</a>,
 as the t [...]
+                  <li><code class="docutils literal"><span class="pre">BlockBasedTableConfig
tableConfig = (BlockBasedTableConfig) options.tableFormatConfig();</span></code>
Get a reference to the existing table config rather than create a new one, so you don't accidentally
overwrite defaults such as the <code class="docutils literal"><span class="pre">BloomFilter</span></code>,
which is an important optimization.
                   <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-
[...]
                   <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>
+                  <li><code class="docutils literal"><span class="pre">cache.close();</span></code>
To avoid memory leaks, you must close any objects you constructed that extend org.rocksdb.RocksObject.
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>
diff --git a/docs/streams/developer-guide/memory-mgmt.html b/docs/streams/developer-guide/memory-mgmt.html
index 70f5d91..5f1158f 100644
--- a/docs/streams/developer-guide/memory-mgmt.html
+++ b/docs/streams/developer-guide/memory-mgmt.html
@@ -170,21 +170,20 @@
     </div>
     <div class="section" id="rocksdb">
       <h2><a class="toc-backref" href="#id3">RocksDB</a><a class="headerlink"
href="#rocksdb" title="Permalink to this headline"></a></h2>
-      <p> Each instance of RocksDB allocates off-heap memory for a block cache (with
data), index and filter blocks, and memtable (write buffer). Critical configs (for RocksDB
version 4.1.0) include
+      <p> Each instance of RocksDB allocates off-heap memory for a block cache, index
and filter blocks, and memtable (write buffer). Critical configs (for RocksDB version 4.1.0)
include
         <code class="docutils literal"><span class="pre">block_cache_size</span></code>,
<code class="docutils literal"><span class="pre">write_buffer_size</span></code>
and <code class="docutils literal"><span class="pre">max_write_buffer_number</span></code>.
 These can be specified through the
         <code class="docutils literal"><span class="pre">rocksdb.config.setter</span></code>
configuration.</li>
-      <p> As of 2.3.0 the memory usage across all instances can be bounded, limiting
the total off-heap memory of your Streams app. To do so you must configure RocksDB to cache
the index and filter blocks in the block cache, limit the memtable memory through a shared
<a class="reference external" href="https://github.com/facebook/rocksdb/wiki/Write-Buffer-Manager">WriteBufferManager</a>
and count its memory against the block cache, and then pass the same Cache object to each
instance. Se [...]
+      <p> As of 2.3.0 the memory usage across all instances can be bounded, limiting
the total off-heap memory of your Kafka Streams application. To do so you must configure RocksDB
to cache the index and filter blocks in the block cache, limit the memtable memory through
a shared <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/Write-Buffer-Manager">WriteBufferManager</a>
and count its memory against the block cache, and then pass the same Cache object to eac [...]
 
       <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">BoundedMemoryRocksDBConfig</span>
<span class="kd">implements</span> <span class="n">RocksDBConfigSetter</span>
<span class="o">{</span>
 
        <span class="kd">private</span> <span class="kt">static</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">TOTAL_OFF_HEAP_MEMORY</span><span
class="o">,</span> <span class="n">-1</span><span class="o">,</span>
<span class="n">fal [...]
        <span class="kd">private</span> <span class="kt">static</span>
<span class="n">org.rocksdb.WriteBufferManager</span> <span class="n">writeBufferManager</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">WriteBufferManager</span><span class="o">(</span><span
class="mi">TOTAL_MEMTABLE_MEMORY</span><span class="o">,</span> cache<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">&lt;</span><span
class="n">String</span><span class="o">,</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">BlockBasedTableConfig</span> <span class="n">tableConfig</span>
<span class="o">=</span> <span class="k">(BlockBasedTableConfig)</span>
<span class="n">options</span><span><span class="o">.</span><span
class="na">tableFormatConfig</span><span class="o">();</span>
 
          <span class="c1"> // These three options in combination will limit the memory
used by RocksDB to the size passed to the block cache (TOTAL_OFF_HEAP_MEMORY)</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
class="o">);</span>
@@ -192,7 +191,7 @@
          <span class="n">options</span><span class="o">.</span><span
class="na">setWriteBufferManager</span><span class="o">(</span><span
class="mi">writeBufferManager</span><span class="o">);</span>
 
          <span class="c1"> // These options are recommended to be set when bounding
the total memory</span>
-         <span class="n">tableConfig</span><span class="o">.</span><span
class="na">setCacheIndexAndFilterBlocksWithHighPriority</span><span class="o">(</span><span
class="mi">true</span><span class="o">);</span>
+         <span class="n">tableConfig</span><span class="o">.</span><span
class="na">setCacheIndexAndFilterBlocksWithHighPriority</span><span class="o">(</span><span
class="mi">true</span><span class="o">);</span><sup><a href="#fn2"
id="ref2">2</a></sup>
          <span class="n">tableConfig</span><span class="o">.</span><span
class="na">setPinTopLevelIndexAndFilter</span><span class="o">(</span><span
class="mi">true</span><span class="o">);</span>
          <span class="n">tableConfig</span><span class="o">.</span><span
class="na">setBlockSize</span><span class="o">(</span><span class="mi">BLOCK_SIZE</span><span
class="o">);</span><sup><a href="#fn3" id="ref3">3</a></sup>
          <span class="n">options</span><span class="o">.</span><span
class="na">setMaxWriteBufferNumber</span><span class="o">(</span><span
class="mi">N_MEMTABLES</span><span class="o">);</span>
@@ -204,26 +203,24 @@
        <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">// Cache and WriteBufferManager should not be closed here,
as the same objects are shared by every store instance.</span>
-         <span class="c1">// The filter, however, is not shared and should be closed
to avoid leaking memory.</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>
       </div>
-        <sup id="fn1">1. INDEX_FILTER_BLOCK_RATIO can be used to set a fraction of
the block cache to set aside for "high priority" (aka index and filter) blocks, preventing
them from being evicted by data blocks. See the full signature of the LRUCache constructor
<a class="reference external" href="https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/LRUCache.java#L72">here</a>.
</sup>
+        <sup id="fn1">1. INDEX_FILTER_BLOCK_RATIO can be used to set a fraction of
the block cache to set aside for "high priority" (aka index and filter) blocks, preventing
them from being evicted by data blocks. See the full signature of the <a class="reference
external" href="https://github.com/facebook/rocksdb/blob/master/java/src/main/java/org/rocksdb/LRUCache.java#L72">LRUCache
constructor</a>. </sup>
         <br>
-        <sup id="fn2">2. This must be set in order for INDEX_FILTER_BLOCK_RATIO to
take effect (see footnote 1) as described <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/Block-Cache#caching-index-and-filter-blocks">here</a></sup>
+        <sup id="fn2">2. This must be set in order for INDEX_FILTER_BLOCK_RATIO to
take effect (see footnote 1) as described in the <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/Block-Cache#caching-index-and-filter-blocks">RocksDB
docs</a></sup>
         <br>
         <sup id="fn3">3. You may want to 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>. A larger block size means index blocks will be smaller, but the cached dat
[...]
           <br>
           <dl class="docutils">
             <dt>Note:</dt>
             While we recommend setting at least the above configs, the specific options that
yield the best performance are workload dependent and you should consider experimenting with
these to determine the best choices for your specific use case. Keep in mind that the optimal
configs for one app may not apply to one with a different topology or input topic.
-            In addition to the recommended configs above, you may want to consider using
partitioned index filters as described by the <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters">RocksDB
docs</a>
+            In addition to the recommended configs above, you may want to consider using
partitioned index filters as described by the <a class="reference external" href="https://github.com/facebook/rocksdb/wiki/Partitioned-Index-Filters">RocksDB
docs</a>.
 
           </dl>
       </div>
       <div class="section" id="other-memory-usage">
-      <h2><a class="toc-backref" href="#id3">Other memory usage</a><a
class="headerlink" href="#other-memory-usage" title="Permalink to this headline"></a></h2>
+      <h2><a class="toc-backref" href="#id4">Other memory usage</a><a
class="headerlink" href="#other-memory-usage" title="Permalink to this headline"></a></h2>
       <p>There are other modules inside Apache Kafka that allocate memory during runtime.
They include the following:</p>
       <ul class="simple">
         <li>Producer buffering, managed by the producer config <code class="docutils
literal"><span class="pre">buffer.memory</span></code>.</li>
diff --git a/streams/src/main/java/org/apache/kafka/streams/state/RocksDBConfigSetter.java
b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBConfigSetter.java
index 613d636..61b50ab 100644
--- a/streams/src/main/java/org/apache/kafka/streams/state/RocksDBConfigSetter.java
+++ b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBConfigSetter.java
@@ -27,8 +27,9 @@ import org.slf4j.LoggerFactory;
  * An interface to that allows developers to customize the RocksDB settings for a given Store.
  * Please read the <a href="https://github.com/facebook/rocksdb/wiki/RocksDB-Tuning-Guide">RocksDB
Tuning Guide</a>.
  *
- * Note: if you choose to set options.setTableFormatConfig(tableConfig) with a new BlockBasedTableConfig
you should
- * probably also set the filter for that tableConfig, most likely with tableConfig.setFilter(new
BloomFilter());
+ * Note: if you choose to modify the {@code org.rocksdb.BlockBasedTableConfig} you should
retrieve a reference to
+ * the existing one (rather than create a new BlockBasedTableConfig object) so as to not
lose the other default settings.
+ * This can be done as {@code BlockBasedTableConfig tableConfig = (BlockBasedTableConfig)
options.tableFormatConfig();}
  */
 public interface RocksDBConfigSetter {
 


Mime
View raw message