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

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


The following commit(s) were added to refs/heads/master by this push:
     new 54e4ca3b39 HDDS-8313. Support rocksdb WAL ttl and size limit 
configuration (#4494)
54e4ca3b39 is described below

commit 54e4ca3b39911f8afaab5c5d1138ad6273210a55
Author: Sammi Chen <[email protected]>
AuthorDate: Fri Mar 31 11:28:24 2023 +0800

    HDDS-8313. Support rocksdb WAL ttl and size limit configuration (#4494)
---
 .../hadoop/hdds/utils/db/DBStoreBuilder.java       |  4 +++
 .../hadoop/hdds/utils/db/RocksDBConfiguration.java | 42 +++++++++++++++++++---
 2 files changed, 42 insertions(+), 4 deletions(-)

diff --git 
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java
 
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java
index 5cd20ab237..3d5d785f86 100644
--- 
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java
+++ 
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/DBStoreBuilder.java
@@ -399,6 +399,10 @@ public final class DBStoreBuilder {
       dbOptions.setLogger(logger);
     }
 
+    // Apply WAL settings.
+    dbOptions.setWalTtlSeconds(rocksDBConfiguration.getWalTTL());
+    dbOptions.setWalSizeLimitMB(rocksDBConfiguration.getWalSizeLimit());
+
     // Create statistics.
     if (!rocksDbStat.equals(OZONE_METADATA_STORE_ROCKSDB_STATISTICS_OFF)) {
       ManagedStatistics statistics = new ManagedStatistics();
diff --git 
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBConfiguration.java
 
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBConfiguration.java
index 31c6dbb81e..7864946188 100644
--- 
a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBConfiguration.java
+++ 
b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDBConfiguration.java
@@ -20,9 +20,12 @@ package org.apache.hadoop.hdds.utils.db;
 
 import org.apache.hadoop.hdds.conf.Config;
 import org.apache.hadoop.hdds.conf.ConfigGroup;
-import org.apache.hadoop.hdds.conf.ConfigTag;
 import org.apache.hadoop.hdds.conf.ConfigType;
 
+import static org.apache.hadoop.hdds.conf.ConfigTag.DATANODE;
+import static org.apache.hadoop.hdds.conf.ConfigTag.OM;
+import static org.apache.hadoop.hdds.conf.ConfigTag.SCM;
+
 /**
  * Holds configuration items for OM RocksDB.
  */
@@ -32,26 +35,42 @@ public class RocksDBConfiguration {
   @Config(key = "rocksdb.logging.enabled",
       type = ConfigType.BOOLEAN,
       defaultValue = "false",
-      tags = {ConfigTag.OM},
+      tags = {OM, SCM, DATANODE},
       description = "Enable/Disable RocksDB logging for OM.")
   private boolean rocksdbLogEnabled;
 
   @Config(key = "rocksdb.logging.level",
       type = ConfigType.STRING,
       defaultValue = "INFO",
-      tags = {ConfigTag.OM},
+      tags = {OM, SCM, DATANODE},
       description = "OM RocksDB logging level (INFO/DEBUG/WARN/ERROR/FATAL)")
   private String rocksdbLogLevel;
 
   @Config(key = "rocksdb.writeoption.sync",
       type = ConfigType.BOOLEAN,
       defaultValue = "false",
-      tags = {ConfigTag.OM},
+      tags = {OM, SCM, DATANODE},
       description = "Enable/Disable Sync option. If true write will be " +
           "considered complete, once flushed to persistent storage. If false," 
+
           " writes are flushed asynchronously.")
   private boolean syncOption;
 
+  @Config(key = "rocksdb.WAL_ttl_seconds",
+      type = ConfigType.LONG,
+      defaultValue = "1200",
+      tags = {OM, SCM, DATANODE},
+      description = "The lifetime of WAL log files. Default 1200 seconds.")
+  private long walTTL = 1200;
+
+  @Config(key = "rocksdb.WAL_size_limit_MB",
+      type = ConfigType.SIZE,
+      defaultValue = "0MB",
+      tags = {OM, SCM, DATANODE},
+      description = "The total size limit of WAL log files. Once the total log"
+          + " file size exceeds this limit, the earliest files will be 
deleted."
+          + "Default 0 means no limit.")
+  private long walSizeLimit = 0;
+
   public void setRocksdbLoggingEnabled(boolean enabled) {
     this.rocksdbLogEnabled = enabled;
   }
@@ -76,4 +95,19 @@ public class RocksDBConfiguration {
     return syncOption;
   }
 
+  public void setWalTTL(long ttl) {
+    this.walTTL = ttl;
+  }
+
+  public long getWalTTL() {
+    return walTTL;
+  }
+
+  public void setWalSizeLimit(long limit) {
+    this.walSizeLimit = limit;
+  }
+
+  public long getWalSizeLimit() {
+    return walSizeLimit;
+  }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to