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

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

commit 798b22cb258579b3ef981eb444397abd7c81f291
Author: Nico Kruber <n...@ververica.com>
AuthorDate: Thu Dec 5 10:35:44 2019 +0100

    [FLINK-15068][rocksdb] stop writing to RocksDB logs by default
    
    This commit changes the default RocksDB configuration for all 
PredefinedOptions
    so that they use log level HEADER_LEVEL and disable periodic statistics 
dumps
    to the LOG file.
    Please note that there is no need to also change
    DefaultConfigurableOptionsFactory since this is only applied after any
    PredefinedOptions, and there is always one - at least 
PredefinedOptions#DEFAULT.
    
    The problem with this file is that is will grow indefinitely until it is 
deleted
    when the job is cancelled/restarted since it lives in RocksDB's local 
directory.
    Therefore, it cannot be used for troubleshooting errors. For looking into
    performance, metrics are probably better in the first place.
    
    Note: Theoretically, we could even set the log level to NUM_INFO_LOG_LEVELS
    which even removes (most of) the headers, but although that is working, it 
is
    practically an invalid value for the log level and would be a bit hacky.
    
    This closes #10437
---
 .../contrib/streaming/state/PredefinedOptions.java | 33 +++++++++++++++++++---
 1 file changed, 29 insertions(+), 4 deletions(-)

diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
index 73dc0be..24fde98 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/main/java/org/apache/flink/contrib/streaming/state/PredefinedOptions.java
@@ -23,6 +23,7 @@ import org.rocksdb.BloomFilter;
 import org.rocksdb.ColumnFamilyOptions;
 import org.rocksdb.CompactionStyle;
 import org.rocksdb.DBOptions;
+import org.rocksdb.InfoLogLevel;
 
 /**
  * The {@code PredefinedOptions} are configuration settings for the {@link 
RocksDBStateBackend}.
@@ -31,6 +32,9 @@ import org.rocksdb.DBOptions;
  *
  * <p>Some of these settings are based on experiments by the Flink community, 
some follow
  * guides from the RocksDB project.
+ *
+ * <p>All of them effectively disable the RocksDB log by default because this 
file would grow
+ * indefinitely and will be deleted with the TM anyway.
  */
 public enum PredefinedOptions {
 
@@ -40,13 +44,22 @@ public enum PredefinedOptions {
         *
         * <p>Note: Because Flink does not rely on RocksDB data on disk for 
recovery,
         * there is no need to sync data to stable storage.
+        *
+        * <p>The following options are set:
+        * <ul>
+        *     <li>setUseFsync(false)</li>
+        *     <li>setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)</li>
+        *     <li>setStatsDumpPeriodSec(0)</li>
+        * </ul>
         */
        DEFAULT {
 
                @Override
                public DBOptions createDBOptions() {
                        return new DBOptions()
-                                       .setUseFsync(false);
+                                       .setUseFsync(false)
+                                       
.setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)
+                                       .setStatsDumpPeriodSec(0);
                }
 
                @Override
@@ -71,6 +84,8 @@ public enum PredefinedOptions {
         *     <li>setUseFsync(false)</li>
         *     <li>setDisableDataSync(true)</li>
         *     <li>setMaxOpenFiles(-1)</li>
+        *     <li>setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)</li>
+        *     <li>setStatsDumpPeriodSec(0)</li>
         * </ul>
         *
         * <p>Note: Because Flink does not rely on RocksDB data on disk for 
recovery,
@@ -84,7 +99,9 @@ public enum PredefinedOptions {
                        return new DBOptions()
                                        .setIncreaseParallelism(4)
                                        .setUseFsync(false)
-                                       .setMaxOpenFiles(-1);
+                                       .setMaxOpenFiles(-1)
+                                       
.setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)
+                                       .setStatsDumpPeriodSec(0);
                }
 
                @Override
@@ -114,6 +131,8 @@ public enum PredefinedOptions {
         *     <li>setMaxWriteBufferNumber(4)</li>
         *     <li>setUseFsync(false)</li>
         *     <li>setMaxOpenFiles(-1)</li>
+        *     <li>setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)</li>
+        *     <li>setStatsDumpPeriodSec(0)</li>
         *     <li>BlockBasedTableConfig.setBlockCacheSize(256 MBytes)</li>
         *     <li>BlockBasedTableConfigsetBlockSize(128 KBytes)</li>
         * </ul>
@@ -129,7 +148,9 @@ public enum PredefinedOptions {
                        return new DBOptions()
                                        .setIncreaseParallelism(4)
                                        .setUseFsync(false)
-                                       .setMaxOpenFiles(-1);
+                                       .setMaxOpenFiles(-1)
+                                       
.setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)
+                                       .setStatsDumpPeriodSec(0);
                }
 
                @Override
@@ -169,6 +190,8 @@ public enum PredefinedOptions {
         *     <li>setUseFsync(false)</li>
         *     <li>setDisableDataSync(true)</li>
         *     <li>setMaxOpenFiles(-1)</li>
+        *     <li>setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)</li>
+        *     <li>setStatsDumpPeriodSec(0)</li>
         * </ul>
         *
         * <p>Note: Because Flink does not rely on RocksDB data on disk for 
recovery,
@@ -181,7 +204,9 @@ public enum PredefinedOptions {
                        return new DBOptions()
                                        .setIncreaseParallelism(4)
                                        .setUseFsync(false)
-                                       .setMaxOpenFiles(-1);
+                                       .setMaxOpenFiles(-1)
+                                       
.setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)
+                                       .setStatsDumpPeriodSec(0);
                }
 
                @Override

Reply via email to