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 1ae3d34  [FLINK-26998][state] Activate default log-level options in 
RocksDB state backend
1ae3d34 is described below

commit 1ae3d343f54b7697dd079f9de4229160b40a3a7d
Author: Zakelly <[email protected]>
AuthorDate: Sat Apr 2 11:33:52 2022 +0800

    [FLINK-26998][state] Activate default log-level options in RocksDB state 
backend
    
    this closes #19335.
---
 .../contrib/streaming/state/PredefinedOptions.java      | 17 ++---------------
 .../streaming/state/RocksDBStateBackendConfigTest.java  |  3 +++
 2 files changed, 5 insertions(+), 15 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 25e5fa0..d69b543 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
@@ -22,7 +22,6 @@ import org.apache.flink.configuration.ConfigOption;
 import org.apache.flink.configuration.MemorySize;
 
 import org.rocksdb.CompactionStyle;
-import org.rocksdb.InfoLogLevel;
 
 import javax.annotation.Nullable;
 
@@ -54,15 +53,9 @@ 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>setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)
-     * </ul>
+     * <p>There are no specified options here.
      */
-    DEFAULT(
-            Collections.singletonMap(
-                    RocksDBConfigurableOptions.LOG_LEVEL, 
InfoLogLevel.HEADER_LEVEL)),
+    DEFAULT(Collections.emptyMap()),
 
     /**
      * Pre-defined options for regular spinning hard disks.
@@ -78,7 +71,6 @@ public enum PredefinedOptions {
      *   <li>setMaxBackgroundJobs(4)
      *   <li>setDisableDataSync(true)
      *   <li>setMaxOpenFiles(-1)
-     *   <li>setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)
      * </ul>
      *
      * <p>Note: Because Flink does not rely on RocksDB data on disk for 
recovery, there is no need
@@ -91,7 +83,6 @@ public enum PredefinedOptions {
                 {
                     put(RocksDBConfigurableOptions.MAX_BACKGROUND_THREADS, 4);
                     put(RocksDBConfigurableOptions.MAX_OPEN_FILES, -1);
-                    put(RocksDBConfigurableOptions.LOG_LEVEL, 
InfoLogLevel.HEADER_LEVEL);
                     put(RocksDBConfigurableOptions.COMPACTION_STYLE, 
CompactionStyle.LEVEL);
                     put(RocksDBConfigurableOptions.USE_DYNAMIC_LEVEL_SIZE, 
true);
                 }
@@ -116,7 +107,6 @@ public enum PredefinedOptions {
      *   <li>setMinWriteBufferNumberToMerge(3)
      *   <li>setMaxWriteBufferNumber(4)
      *   <li>setMaxOpenFiles(-1)
-     *   <li>setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)
      *   <li>BlockBasedTableConfig.setBlockCacheSize(256 MBytes)
      *   <li>BlockBasedTableConfig.setBlockSize(128 KBytes)
      * </ul>
@@ -131,7 +121,6 @@ public enum PredefinedOptions {
                 {
                     put(RocksDBConfigurableOptions.MAX_BACKGROUND_THREADS, 4);
                     put(RocksDBConfigurableOptions.MAX_OPEN_FILES, -1);
-                    put(RocksDBConfigurableOptions.LOG_LEVEL, 
InfoLogLevel.HEADER_LEVEL);
                     put(RocksDBConfigurableOptions.COMPACTION_STYLE, 
CompactionStyle.LEVEL);
                     put(RocksDBConfigurableOptions.USE_DYNAMIC_LEVEL_SIZE, 
true);
                     put(
@@ -159,7 +148,6 @@ public enum PredefinedOptions {
      *   <li>setMaxBackgroundJobs(4)
      *   <li>setDisableDataSync(true)
      *   <li>setMaxOpenFiles(-1)
-     *   <li>setInfoLogLevel(InfoLogLevel.HEADER_LEVEL)
      * </ul>
      *
      * <p>Note: Because Flink does not rely on RocksDB data on disk for 
recovery, there is no need
@@ -172,7 +160,6 @@ public enum PredefinedOptions {
                 {
                     put(RocksDBConfigurableOptions.MAX_BACKGROUND_THREADS, 4);
                     put(RocksDBConfigurableOptions.MAX_OPEN_FILES, -1);
-                    put(RocksDBConfigurableOptions.LOG_LEVEL, 
InfoLogLevel.HEADER_LEVEL);
                 }
             });
 
diff --git 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
index a3be47c..6e65f81 100644
--- 
a/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
+++ 
b/flink-state-backends/flink-statebackend-rocksdb/src/test/java/org/apache/flink/contrib/streaming/state/RocksDBStateBackendConfigTest.java
@@ -102,6 +102,9 @@ public class RocksDBStateBackendConfigTest {
         // set the environment variable 'log.file' with the Flink log file 
location
         System.setProperty("log.file", logFile.getPath());
         try (RocksDBResourceContainer container = 
backend.createOptionsAndResourceContainer()) {
+            assertEquals(
+                    RocksDBConfigurableOptions.LOG_LEVEL.defaultValue(),
+                    container.getDbOptions().infoLogLevel());
             assertEquals(logFile.getParent(), 
container.getDbOptions().dbLogDir());
         } finally {
             logFile.delete();

Reply via email to