You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ta...@apache.org on 2021/12/02 12:18:32 UTC

[flink] branch master updated: [FLINK-23791] Enable RocksDB info log by default

This is an automated email from the ASF dual-hosted git repository.

tangyun pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/flink.git


The following commit(s) were added to refs/heads/master by this push:
     new 754914a  [FLINK-23791] Enable RocksDB info log by default
754914a is described below

commit 754914a98df8791693245d889e195818c3bf1e49
Author: Zakelly <za...@gmail.com>
AuthorDate: Thu Dec 2 14:39:57 2021 +0800

    [FLINK-23791] Enable RocksDB info log by default
---
 .../generated/rocksdb_configurable_configuration.html     | 12 ++++++------
 .../streaming/state/RocksDBConfigurableOptions.java       | 15 ++++++++-------
 2 files changed, 14 insertions(+), 13 deletions(-)

diff --git a/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html b/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
index a8a974b..2eaf473 100644
--- a/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
+++ b/docs/layouts/shortcodes/generated/rocksdb_configurable_configuration.html
@@ -76,21 +76,21 @@
         </tr>
         <tr>
             <td><h5>state.backend.rocksdb.log.file-num</h5></td>
-            <td style="word-wrap: break-word;">1000</td>
+            <td style="word-wrap: break-word;">4</td>
             <td>Integer</td>
-            <td>The maximum number of files RocksDB should keep for information logging (Default setting: 1000).</td>
+            <td>The maximum number of files RocksDB should keep for information logging (Default setting: 4).</td>
         </tr>
         <tr>
             <td><h5>state.backend.rocksdb.log.level</h5></td>
-            <td style="word-wrap: break-word;">HEADER_LEVEL</td>
+            <td style="word-wrap: break-word;">INFO_LEVEL</td>
             <td><p>Enum</p></td>
-            <td>The specified information logging level for RocksDB. If unset, Flink will use <code class="highlighter-rouge">HEADER_LEVEL</code>.<br />Note: RocksDB info logs will not be written to the TaskManager logs and there is no rolling strategy, unless you configure <code class="highlighter-rouge">state.backend.rocksdb.log.dir</code>, <code class="highlighter-rouge">state.backend.rocksdb.log.max-file-size</code>, and <code class="highlighter-rouge">state.backend.rocksdb.log.file- [...]
+            <td>The specified information logging level for RocksDB. If unset, Flink will use <code class="highlighter-rouge">INFO_LEVEL</code>.<br />Note: RocksDB info logs will not be written to the TaskManager logs and there is no rolling strategy, unless you configure <code class="highlighter-rouge">state.backend.rocksdb.log.dir</code>, <code class="highlighter-rouge">state.backend.rocksdb.log.max-file-size</code>, and <code class="highlighter-rouge">state.backend.rocksdb.log.file-nu [...]
         </tr>
         <tr>
             <td><h5>state.backend.rocksdb.log.max-file-size</h5></td>
-            <td style="word-wrap: break-word;">0 bytes</td>
+            <td style="word-wrap: break-word;">25 mb</td>
             <td>MemorySize</td>
-            <td>The maximum size of RocksDB's file used for information logging. If the log files becomes larger than this, a new file will be created. If 0 (Flink default setting), all logs will be written to one log file.</td>
+            <td>The maximum size of RocksDB's file used for information logging. If the log files becomes larger than this, a new file will be created. If 0, all logs will be written to one log file. The default maximum file size is '25MB'. </td>
         </tr>
         <tr>
             <td><h5>state.backend.rocksdb.thread.num</h5></td>
diff --git a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBConfigurableOptions.java b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBConfigurableOptions.java
index 8084ef9..25a2464 100644
--- a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBConfigurableOptions.java
+++ b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/RocksDBConfigurableOptions.java
@@ -39,7 +39,7 @@ import static org.rocksdb.CompactionStyle.FIFO;
 import static org.rocksdb.CompactionStyle.LEVEL;
 import static org.rocksdb.CompactionStyle.NONE;
 import static org.rocksdb.CompactionStyle.UNIVERSAL;
-import static org.rocksdb.InfoLogLevel.HEADER_LEVEL;
+import static org.rocksdb.InfoLogLevel.INFO_LEVEL;
 
 /**
  * This class contains the configuration options for the {@link EmbeddedRocksDBStateBackend}.
@@ -73,18 +73,19 @@ public class RocksDBConfigurableOptions implements Serializable {
     public static final ConfigOption<MemorySize> LOG_MAX_FILE_SIZE =
             key("state.backend.rocksdb.log.max-file-size")
                     .memoryType()
-                    .defaultValue(MemorySize.ZERO)
+                    .defaultValue(MemorySize.parse("25mb"))
                     .withDescription(
                             "The maximum size of RocksDB's file used for information logging. "
                                     + "If the log files becomes larger than this, a new file will be created. "
-                                    + "If 0 (Flink default setting), all logs will be written to one log file.");
+                                    + "If 0, all logs will be written to one log file. "
+                                    + "The default maximum file size is '25MB'. ");
 
     public static final ConfigOption<Integer> LOG_FILE_NUM =
             key("state.backend.rocksdb.log.file-num")
                     .intType()
-                    .defaultValue(1000)
+                    .defaultValue(4)
                     .withDescription(
-                            "The maximum number of files RocksDB should keep for information logging (Default setting: 1000).");
+                            "The maximum number of files RocksDB should keep for information logging (Default setting: 4).");
 
     public static final ConfigOption<String> LOG_DIR =
             key("state.backend.rocksdb.log.dir")
@@ -98,13 +99,13 @@ public class RocksDBConfigurableOptions implements Serializable {
     public static final ConfigOption<InfoLogLevel> LOG_LEVEL =
             key("state.backend.rocksdb.log.level")
                     .enumType(InfoLogLevel.class)
-                    .defaultValue(HEADER_LEVEL)
+                    .defaultValue(INFO_LEVEL)
                     .withDescription(
                             Description.builder()
                                     .text(
                                             "The specified information logging level for RocksDB. "
                                                     + "If unset, Flink will use %s.",
-                                            code(HEADER_LEVEL.name()))
+                                            code(INFO_LEVEL.name()))
                                     .linebreak()
                                     .text(
                                             "Note: RocksDB info logs will not be written to the TaskManager logs and there "