You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by ay...@apache.org on 2022/03/10 00:06:06 UTC

[bookkeeper] branch master updated: change rocksdb init: use OptionsUtil

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 3edbc98  change rocksdb init: use OptionsUtil
3edbc98 is described below

commit 3edbc98824bbdf574d97510aec783d5b9355548e
Author: StevenLuMT <42...@users.noreply.github.com>
AuthorDate: Thu Mar 10 08:05:59 2022 +0800

    change rocksdb init: use OptionsUtil
    
    Descriptions of the changes in this PR:
    
    ### Motivation
    
    1. some old parameters in rocksDB is not configurable
    2. for all the tuning of rocksdb in the future, there is no need to update the code or introduce configuration to bookie
    
    ### Changes
    
    1)   rocks all old parameter change to be configurable
    2)  use OptionsUtil to init all params for rocksdb
    
    the old pr #3006  has some rebase error,open a new pr
    
    Reviewers: Andrey Yegorov <None>, LinChen <None>
    
    This closes #3056 from StevenLuMT/master_improveRocksDB
---
 bookkeeper-dist/bkctl/build.gradle                 |   3 +
 bookkeeper-dist/src/assemble/bkctl.xml             |   3 +
 .../bookie/storage/ldb/DbLedgerStorage.java        |   2 +-
 .../bookie/storage/ldb/EntryLocationIndex.java     |   2 +-
 .../bookie/storage/ldb/KeyValueStorageFactory.java |   5 +-
 .../bookie/storage/ldb/KeyValueStorageRocksDB.java | 139 ++++-----------------
 .../bookie/storage/ldb/LedgerMetadataIndex.java    |   2 +-
 .../bookie/storage/ldb/LedgersIndexCheckOp.java    |   2 +-
 .../bookie/storage/ldb/LedgersIndexRebuildOp.java  |   2 +-
 .../storage/ldb/LocationsIndexRebuildOp.java       |   2 +-
 .../storage/ldb/PersistentEntryLogMetadataMap.java |   2 +-
 .../bookkeeper/conf/ServerConfiguration.java       |  82 ++++++++++++
 .../bookie/storage/ldb/KeyValueStorageTest.java    |   2 +-
 .../src/test/resources/conf/default_rocksdb.conf   |  29 +++++
 .../resources/conf/entry_location_rocksdb.conf     |  69 ++++++++++
 .../resources/conf/ledger_metadata_rocksdb.conf    |  29 +++++
 conf/default_rocksdb.conf                          |  29 +++++
 conf/entry_location_rocksdb.conf                   |  69 ++++++++++
 conf/ledger_metadata_rocksdb.conf                  |  29 +++++
 19 files changed, 378 insertions(+), 124 deletions(-)

diff --git a/bookkeeper-dist/bkctl/build.gradle b/bookkeeper-dist/bkctl/build.gradle
index 27d2753..af4cce6 100644
--- a/bookkeeper-dist/bkctl/build.gradle
+++ b/bookkeeper-dist/bkctl/build.gradle
@@ -72,6 +72,9 @@ distributions {
                             "log4j.cli.properties",
                             "log4j.shell.properties",
                             "nettyenv.sh",
+                            "default_rocksdb.conf",
+                            "entry_location_rocksdb.conf",
+                            "ledger_metadata_rocksdb.conf",
                     )
                 }
             }
diff --git a/bookkeeper-dist/src/assemble/bkctl.xml b/bookkeeper-dist/src/assemble/bkctl.xml
index cb4ea8f..1e7d518 100644
--- a/bookkeeper-dist/src/assemble/bkctl.xml
+++ b/bookkeeper-dist/src/assemble/bkctl.xml
@@ -41,6 +41,9 @@
         <include>log4j.cli.properties</include>
         <include>log4j.shell.properties</include>
         <include>nettyenv.sh</include>
+        <include>default_rocksdb.conf</include>
+        <include>entry_location_rocksdb.conf</include>
+        <include>ledger_metadata_rocksdb.conf</include>
       </includes>
     </fileSet>
     <fileSet>
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java
index 2ba63cc..0a563a1 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/DbLedgerStorage.java
@@ -379,7 +379,7 @@ public class DbLedgerStorage implements LedgerStorage {
 
         EntryLocationIndex entryLocationIndex = new EntryLocationIndex(serverConf,
                 (basePath, subPath, dbConfigType, conf1) ->
-                        new KeyValueStorageRocksDB(basePath, subPath, DbConfigType.Small, conf1, true),
+                        new KeyValueStorageRocksDB(basePath, subPath, DbConfigType.Default, conf1, true),
                 ledgerBasePath, NullStatsLogger.INSTANCE);
         try {
             long lastEntryId = entryLocationIndex.getLastEntryInLedger(ledgerId);
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java
index 792d96d..613ec24 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/EntryLocationIndex.java
@@ -53,7 +53,7 @@ public class EntryLocationIndex implements Closeable {
 
     public EntryLocationIndex(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath,
             StatsLogger stats) throws IOException {
-        locationsDb = storageFactory.newKeyValueStorage(basePath, "locations", DbConfigType.Huge, conf);
+        locationsDb = storageFactory.newKeyValueStorage(basePath, "locations", DbConfigType.EntryLocation, conf);
 
         this.stats = new EntryLocationIndexStats(
             stats,
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageFactory.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageFactory.java
index f3023c1..08f68fe 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageFactory.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageFactory.java
@@ -33,8 +33,9 @@ public interface KeyValueStorageFactory {
      * Enum used to specify different config profiles in the underlying storage.
      */
     enum DbConfigType {
-        Small, // Used for ledgers db, doesn't need particular configuration
-        Huge // Used for location index, lots of writes and much bigger dataset
+        Default, // Used for default,command until or test case
+        LedgerMetadata, // Used for ledgers db, doesn't need particular configuration
+        EntryLocation // Used for location index, lots of writes and much bigger dataset
     }
 
     KeyValueStorage newKeyValueStorage(String defaultBasePath, String subPath, DbConfigType dbConfigType,
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java
index e6eb197..293004c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageRocksDB.java
@@ -22,27 +22,21 @@ package org.apache.bookkeeper.bookie.storage.ldb;
 
 import static com.google.common.base.Preconditions.checkState;
 
-//CHECKSTYLE.OFF: IllegalImport
-import io.netty.util.internal.PlatformDependent;
-//CHECKSTYLE.ON: IllegalImport
-
 import java.io.IOException;
 import java.nio.file.FileSystems;
 import java.nio.file.Files;
 import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map.Entry;
-import java.util.concurrent.TimeUnit;
 
 import org.apache.bookkeeper.bookie.storage.ldb.KeyValueStorageFactory.DbConfigType;
 import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.rocksdb.BlockBasedTableConfig;
-import org.rocksdb.BloomFilter;
-import org.rocksdb.Cache;
-import org.rocksdb.ChecksumType;
-import org.rocksdb.CompressionType;
-import org.rocksdb.InfoLogLevel;
-import org.rocksdb.LRUCache;
-import org.rocksdb.Options;
+import org.rocksdb.ColumnFamilyDescriptor;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.DBOptions;
+import org.rocksdb.Env;
+import org.rocksdb.OptionsUtil;
 import org.rocksdb.ReadOptions;
 import org.rocksdb.RocksDB;
 import org.rocksdb.RocksDBException;
@@ -65,25 +59,12 @@ public class KeyValueStorageRocksDB implements KeyValueStorage {
 
     private final WriteOptions optionSync;
     private final WriteOptions optionDontSync;
-    private final Cache cache;
 
     private final ReadOptions optionCache;
     private final ReadOptions optionDontCache;
-
     private final WriteBatch emptyBatch;
 
     private static final String ROCKSDB_LOG_PATH = "dbStorage_rocksDB_logPath";
-    private static final String ROCKSDB_LOG_LEVEL = "dbStorage_rocksDB_logLevel";
-    private static final String ROCKSDB_LZ4_COMPRESSION_ENABLED = "dbStorage_rocksDB_lz4CompressionEnabled";
-    private static final String ROCKSDB_WRITE_BUFFER_SIZE_MB = "dbStorage_rocksDB_writeBufferSizeMB";
-    private static final String ROCKSDB_SST_SIZE_MB = "dbStorage_rocksDB_sstSizeInMB";
-    private static final String ROCKSDB_BLOCK_SIZE = "dbStorage_rocksDB_blockSize";
-    private static final String ROCKSDB_BLOOM_FILTERS_BITS_PER_KEY = "dbStorage_rocksDB_bloomFilterBitsPerKey";
-    private static final String ROCKSDB_BLOCK_CACHE_SIZE = "dbStorage_rocksDB_blockCacheSize";
-    private static final String ROCKSDB_NUM_LEVELS = "dbStorage_rocksDB_numLevels";
-    private static final String ROCKSDB_NUM_FILES_IN_LEVEL0 = "dbStorage_rocksDB_numFilesInLevel0";
-    private static final String ROCKSDB_MAX_SIZE_IN_LEVEL1_MB = "dbStorage_rocksDB_maxSizeInLevel1MB";
-    private static final String ROCKSDB_FORMAT_VERSION = "dbStorage_rocksDB_format_version";
 
     public KeyValueStorageRocksDB(String basePath, String subPath, DbConfigType dbConfigType, ServerConfiguration conf)
             throws IOException {
@@ -105,104 +86,37 @@ public class KeyValueStorageRocksDB implements KeyValueStorage {
         this.optionDontCache = new ReadOptions();
         this.emptyBatch = new WriteBatch();
 
-        try (Options options = new Options()) {
-            options.setCreateIfMissing(true);
-
-            if (dbConfigType == DbConfigType.Huge) {
-                // Set default RocksDB block-cache size to 10% / numberOfLedgers of direct memory, unless override
-                int ledgerDirsSize = conf.getLedgerDirNames().length;
-                long defaultRocksDBBlockCacheSizeBytes = PlatformDependent.maxDirectMemory() / ledgerDirsSize / 10;
-                long blockCacheSize = DbLedgerStorage.getLongVariableOrDefault(conf, ROCKSDB_BLOCK_CACHE_SIZE,
-                        defaultRocksDBBlockCacheSizeBytes);
-
-                long writeBufferSizeMB = conf.getInt(ROCKSDB_WRITE_BUFFER_SIZE_MB, 64);
-                long sstSizeMB = conf.getInt(ROCKSDB_SST_SIZE_MB, 64);
-                int numLevels = conf.getInt(ROCKSDB_NUM_LEVELS, -1);
-                int numFilesInLevel0 = conf.getInt(ROCKSDB_NUM_FILES_IN_LEVEL0, 4);
-                long maxSizeInLevel1MB = conf.getLong(ROCKSDB_MAX_SIZE_IN_LEVEL1_MB, 256);
-                int blockSize = conf.getInt(ROCKSDB_BLOCK_SIZE, 64 * 1024);
-                int bloomFilterBitsPerKey = conf.getInt(ROCKSDB_BLOOM_FILTERS_BITS_PER_KEY, 10);
-                boolean lz4CompressionEnabled = conf.getBoolean(ROCKSDB_LZ4_COMPRESSION_ENABLED, true);
-                int formatVersion = conf.getInt(ROCKSDB_FORMAT_VERSION, 2);
-
-                if (lz4CompressionEnabled) {
-                    options.setCompressionType(CompressionType.LZ4_COMPRESSION);
-                }
-                options.setWriteBufferSize(writeBufferSizeMB * 1024 * 1024);
-                options.setMaxWriteBufferNumber(4);
-                if (numLevels > 0) {
-                    options.setNumLevels(numLevels);
-                }
-                options.setLevelZeroFileNumCompactionTrigger(numFilesInLevel0);
-                options.setMaxBytesForLevelBase(maxSizeInLevel1MB * 1024 * 1024);
-                options.setMaxBackgroundJobs(32);
-                options.setIncreaseParallelism(32);
-                options.setMaxTotalWalSize(512 * 1024 * 1024);
-                options.setMaxOpenFiles(-1);
-                options.setTargetFileSizeBase(sstSizeMB * 1024 * 1024);
-                options.setDeleteObsoleteFilesPeriodMicros(TimeUnit.HOURS.toMicros(1));
-
-                this.cache = new LRUCache(blockCacheSize);
-                BlockBasedTableConfig tableOptions = new BlockBasedTableConfig();
-                tableOptions.setBlockSize(blockSize);
-                tableOptions.setBlockCache(cache);
-                tableOptions.setFormatVersion(formatVersion);
-                tableOptions.setChecksumType(ChecksumType.kxxHash);
-                if (bloomFilterBitsPerKey > 0) {
-                    tableOptions.setFilterPolicy(new BloomFilter(bloomFilterBitsPerKey, false));
-                }
-
-                // Options best suited for HDDs
-                tableOptions.setCacheIndexAndFilterBlocks(true);
-                options.setLevelCompactionDynamicLevelBytes(true);
-
-                options.setTableFormatConfig(tableOptions);
+        String dbFilePath = "";
+        DBOptions dbOptions = new DBOptions();
+        final List<ColumnFamilyDescriptor> cfDescs = new ArrayList<>();
+        final List<ColumnFamilyHandle> cfHandles = new ArrayList<>();
+        try {
+            if (dbConfigType == DbConfigType.EntryLocation) {
+                dbFilePath = conf.getEntryLocationRocksdbConf();
+            } else if (dbConfigType == DbConfigType.LedgerMetadata) {
+                dbFilePath = conf.getLedgerMetadataRocksdbConf();
             } else {
-                this.cache = null;
+                dbFilePath = conf.getDefaultRocksDBConf();
             }
 
+            OptionsUtil.loadOptionsFromFile(dbFilePath, Env.getDefault(), dbOptions, cfDescs, false);
             // Configure file path
             String logPath = conf.getString(ROCKSDB_LOG_PATH, "");
             if (!logPath.isEmpty()) {
                 Path logPathSetting = FileSystems.getDefault().getPath(logPath, subPath);
                 Files.createDirectories(logPathSetting);
                 log.info("RocksDB<{}> log path: {}", subPath, logPathSetting);
-                options.setDbLogDir(logPathSetting.toString());
+                dbOptions.setDbLogDir(logPathSetting.toString());
             }
             String path = FileSystems.getDefault().getPath(basePath, subPath).toFile().toString();
 
-            // Configure log level
-            String logLevel = conf.getString(ROCKSDB_LOG_LEVEL, "info");
-            switch (logLevel) {
-            case "debug":
-                options.setInfoLogLevel(InfoLogLevel.DEBUG_LEVEL);
-                break;
-            case "info":
-                options.setInfoLogLevel(InfoLogLevel.INFO_LEVEL);
-                break;
-            case "warn":
-                options.setInfoLogLevel(InfoLogLevel.WARN_LEVEL);
-                break;
-            case "error":
-                options.setInfoLogLevel(InfoLogLevel.ERROR_LEVEL);
-                break;
-            default:
-                log.warn("Unrecognized RockDB log level: {}", logLevel);
-            }
-
-            // Keep log files for 1month
-            options.setKeepLogFileNum(30);
-            options.setLogFileTimeToRoll(TimeUnit.DAYS.toSeconds(1));
-
-            try {
-                if (readOnly) {
-                    db = RocksDB.openReadOnly(options, path);
-                } else {
-                    db = RocksDB.open(options, path);
-                }
-            } catch (RocksDBException e) {
-                throw new IOException("Error open RocksDB database", e);
+            if (readOnly) {
+                db = RocksDB.openReadOnly(dbOptions, path, cfDescs, cfHandles);
+            } else {
+                db = RocksDB.open(dbOptions, path, cfDescs, cfHandles);
             }
+        } catch (RocksDBException e) {
+            throw new IOException("Error open RocksDB database", e);
         }
 
         optionSync.setSync(true);
@@ -215,9 +129,6 @@ public class KeyValueStorageRocksDB implements KeyValueStorage {
     @Override
     public void close() throws IOException {
         db.close();
-        if (cache != null) {
-            cache.close();
-        }
         optionSync.close();
         optionDontSync.close();
         optionCache.close();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
index df79c1b..72e337c 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgerMetadataIndex.java
@@ -70,7 +70,7 @@ public class LedgerMetadataIndex implements Closeable {
 
     public LedgerMetadataIndex(ServerConfiguration conf, KeyValueStorageFactory storageFactory, String basePath,
             StatsLogger stats) throws IOException {
-        ledgersDb = storageFactory.newKeyValueStorage(basePath, "ledgers", DbConfigType.Small, conf);
+        ledgersDb = storageFactory.newKeyValueStorage(basePath, "ledgers", DbConfigType.LedgerMetadata, conf);
 
         ledgers = ConcurrentLongHashMap.<LedgerData>newBuilder().build();
         ledgersCount = new AtomicInteger();
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java
index a48de34..9a2a5f5 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexCheckOp.java
@@ -59,7 +59,7 @@ public class LedgersIndexCheckOp {
 
         try {
             KeyValueStorage index = new KeyValueStorageRocksDB(basePath, LedgersSubPath,
-                    DbConfigType.Small, conf, true);
+                    DbConfigType.Default, conf, true);
             // Read all ledgers from db
             KeyValueStorage.CloseableIterator<Map.Entry<byte[], byte[]>> iterator = index.iterator();
             int ctr = 0;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
index 15410ef..7e10de8 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LedgersIndexRebuildOp.java
@@ -91,7 +91,7 @@ public class LedgersIndexRebuildOp {
                     + "Starting to build a new ledgers index", ledgers.size());
 
             try (KeyValueStorage newIndex = KeyValueStorageRocksDB.factory.newKeyValueStorage(
-                    basePath, tempLedgersSubPath, DbConfigType.Small, conf)) {
+                    basePath, tempLedgersSubPath, DbConfigType.Default, conf)) {
                 LOG.info("Created ledgers index at temp location {}", tempPath);
 
                 for (Long ledgerId : ledgers) {
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
index 7eae6da..55c5c90 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/LocationsIndexRebuildOp.java
@@ -77,7 +77,7 @@ public class LocationsIndexRebuildOp {
         LOG.info("Found {} active ledgers in ledger manager", activeLedgers.size());
 
         KeyValueStorage newIndex = KeyValueStorageRocksDB.factory.newKeyValueStorage(basePath, "locations",
-                DbConfigType.Huge, conf);
+                DbConfigType.Default, conf);
 
         int totalEntryLogs = entryLogs.size();
         int completedEntryLogs = 0;
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/PersistentEntryLogMetadataMap.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/PersistentEntryLogMetadataMap.java
index 99f6069..ec289fd 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/PersistentEntryLogMetadataMap.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/bookie/storage/ldb/PersistentEntryLogMetadataMap.java
@@ -85,7 +85,7 @@ public class PersistentEntryLogMetadataMap implements EntryLogMetadataMap {
             throw new IOException(err);
         }
         metadataMapDB = KeyValueStorageRocksDB.factory.newKeyValueStorage(metadataPath, METADATA_CACHE,
-                DbConfigType.Small, conf);
+                DbConfigType.Default, conf);
     }
 
     @Override
diff --git a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
index 73901a3..9ce0556 100644
--- a/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
+++ b/bookkeeper-server/src/main/java/org/apache/bookkeeper/conf/ServerConfiguration.java
@@ -26,6 +26,7 @@ import com.google.common.collect.Lists;
 import io.netty.util.internal.PlatformDependent;
 // CHECKSTYLE.ON: IllegalImport
 import java.io.File;
+import java.net.URL;
 import java.util.concurrent.TimeUnit;
 import org.apache.bookkeeper.bookie.FileChannelProvider;
 import org.apache.bookkeeper.bookie.InterleavedLedgerStorage;
@@ -323,6 +324,15 @@ public class ServerConfiguration extends AbstractConfiguration<ServerConfigurati
     protected static final String DATA_INTEGRITY_CHECKING_ENABLED = "dataIntegrityChecking";
     protected static final String DATA_INTEGRITY_COOKIE_STAMPING_ENABLED = "dataIntegrityStampMissingCookies";
 
+    // Used for default,command until or test case
+    protected static final String DEFAULT_ROCKSDB_CONF = "defaultRocksdbConf";
+
+    // Used for ledgers db, doesn't need particular configuration
+    protected static final String ENTRY_LOCATION_ROCKSDB_CONF = "entryLocationRocksdbConf";
+
+    // Used for location index, lots of writes and much bigger dataset
+    protected static final String LEDGER_METADATA_ROCKSDB_CONF = "ledgerMetadataRocksdbConf";
+
     /**
      * Construct a default configuration object.
      */
@@ -3884,4 +3894,76 @@ public class ServerConfiguration extends AbstractConfiguration<ServerConfigurati
     public boolean isDataIntegrityStampMissingCookiesEnabled() {
         return this.getBoolean(DATA_INTEGRITY_COOKIE_STAMPING_ENABLED, false);
     }
+
+    /**
+     * Get default rocksdb conf.
+     *
+     * @return String configured default rocksdb conf.
+     */
+    public String getDefaultRocksDBConf() {
+        String defaultPath = "conf/default_rocksdb.conf";
+        URL defURL = getClass().getClassLoader().getResource(defaultPath);
+        if (defURL != null) {
+            defaultPath = defURL.getPath();
+        }
+        return getString(DEFAULT_ROCKSDB_CONF, defaultPath);
+    }
+
+    /**
+     * Set default rocksdb conf.
+     *
+     * @return Configuration Object with default rocksdb conf
+     */
+    public ServerConfiguration setDefaultRocksDBConf(String defaultRocksdbConf) {
+        this.setProperty(DEFAULT_ROCKSDB_CONF, defaultRocksdbConf);
+        return this;
+    }
+
+    /**
+     * Get entry Location rocksdb conf.
+     *
+     * @return String configured entry Location rocksdb conf.
+     */
+    public String getEntryLocationRocksdbConf() {
+        String defaultPath = "conf/entry_location_rocksdb.conf";
+        URL defURL = getClass().getClassLoader().getResource(defaultPath);
+        if (defURL != null) {
+            defaultPath = defURL.getPath();
+        }
+        return getString(ENTRY_LOCATION_ROCKSDB_CONF, defaultPath);
+    }
+
+    /**
+     * Set entry Location rocksdb conf.
+     *
+     * @return Configuration Object with entry Location rocksdb conf
+     */
+    public ServerConfiguration setEntryLocationRocksdbConf(String entryLocationRocksdbConf) {
+        this.setProperty(ENTRY_LOCATION_ROCKSDB_CONF, entryLocationRocksdbConf);
+        return this;
+    }
+
+    /**
+     * Get ledger metadata rocksdb conf.
+     *
+     * @return String configured ledger metadata rocksdb conf.
+     */
+    public String getLedgerMetadataRocksdbConf() {
+        String defaultPath = "conf/ledger_metadata_rocksdb.conf";
+        URL defURL = getClass().getClassLoader().getResource(defaultPath);
+        if (defURL != null) {
+            defaultPath = defURL.getPath();
+        }
+        return getString(LEDGER_METADATA_ROCKSDB_CONF, defaultPath);
+    }
+
+    /**
+     * Set ledger metadata rocksdb conf.
+     *
+     * @return Configuration Object with ledger metadata rocksdb conf
+     */
+    public ServerConfiguration setLedgerMetadataRocksdbConf(String ledgerMetadataRocksdbConf) {
+        this.setProperty(LEDGER_METADATA_ROCKSDB_CONF, ledgerMetadataRocksdbConf);
+        return this;
+    }
 }
diff --git a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageTest.java b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageTest.java
index 1c8ae85..ed1e308 100644
--- a/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageTest.java
+++ b/bookkeeper-server/src/test/java/org/apache/bookkeeper/bookie/storage/ldb/KeyValueStorageTest.java
@@ -77,7 +77,7 @@ public class KeyValueStorageTest {
         File tmpDir = Files.createTempDirectory("junitTemporaryFolder").toFile();
         Files.createDirectory(Paths.get(tmpDir.toString(), "subDir"));
 
-        KeyValueStorage db = storageFactory.newKeyValueStorage(tmpDir.toString(), "subDir", DbConfigType.Small,
+        KeyValueStorage db = storageFactory.newKeyValueStorage(tmpDir.toString(), "subDir", DbConfigType.Default,
                 configuration);
 
         assertEquals(null, db.getFloor(toArray(3)));
diff --git a/bookkeeper-server/src/test/resources/conf/default_rocksdb.conf b/bookkeeper-server/src/test/resources/conf/default_rocksdb.conf
new file mode 100644
index 0000000..0f3a087
--- /dev/null
+++ b/bookkeeper-server/src/test/resources/conf/default_rocksdb.conf
@@ -0,0 +1,29 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+[DBOptions]
+ # set by jni: options.setCreateIfMissing
+ create_if_missing=true
+ # set by jni: options.setInfoLogLevel
+ info_log_level=INFO_LEVEL
+ # set by jni: options.setKeepLogFileNum
+ keep_log_file_num=30
+
+[CFOptions "default"]
+ # set by jni: options.setLogFileTimeToRoll
+ log_file_time_to_roll=86400
\ No newline at end of file
diff --git a/bookkeeper-server/src/test/resources/conf/entry_location_rocksdb.conf b/bookkeeper-server/src/test/resources/conf/entry_location_rocksdb.conf
new file mode 100644
index 0000000..df3ac9f
--- /dev/null
+++ b/bookkeeper-server/src/test/resources/conf/entry_location_rocksdb.conf
@@ -0,0 +1,69 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+[DBOptions]
+ # set by jni: options.setCreateIfMissing
+ create_if_missing=true
+ # set by jni: options.setInfoLogLevel
+ info_log_level=INFO_LEVEL
+ # set by jni: options.setKeepLogFileNum
+ keep_log_file_num=30
+ # set by jni: options.setLogFileTimeToRoll
+ log_file_time_to_roll=86400
+ # set by jni: options.setMaxBackgroundJobs or options.setIncreaseParallelism
+ max_background_jobs=2
+ # set by jni: options.setMaxSubcompactions
+ max_subcompactions=1
+ # set by jni: options.setMaxTotalWalSize
+ max_total_wal_size=536870912
+ # set by jni: options.setMaxOpenFiles
+ max_open_files=-1
+ # set by jni: options.setDeleteObsoleteFilesPeriodMicros
+ delete_obsolete_files_period_micros=3600000000
+
+[CFOptions "default"]
+ # set by jni: options.setCompressionType
+ compression=kLZ4Compression
+ # set by jni: options.setWriteBufferSize
+ write_buffer_size=67108864
+ # set by jni: options.setMaxWriteBufferNumber
+ max_write_buffer_number=4
+ # set by jni: options.setNumLevels
+ num_levels=7
+ # set by jni: options.setLevelZeroFileNumCompactionTrigger
+ level0_file_num_compaction_trigger=4
+ # set by jni: options.setMaxBytesForLevelBase
+ max_bytes_for_level_base=268435456
+ # set by jni: options.setTargetFileSizeBase
+ target_file_size_base=67108864
+
+[TableOptions/BlockBasedTable "default"]
+ # set by jni: tableOptions.setBlockSize
+ block_size=65536
+ # set by jni: tableOptions.setBlockCache
+ block_cache=206150041
+ # set by jni: tableOptions.setFormatVersion
+ format_version=2
+ # set by jni: tableOptions.setChecksumType
+ checksum=kxxHash
+ # set by jni: tableOptions.setFilterPolicy, bloomfilter:[bits_per_key]:[use_block_based_builder]
+ filter_policy=rocksdb.BloomFilter:10:false
+ # set by jni: tableOptions.setCacheIndexAndFilterBlocks
+ cache_index_and_filter_blocks=true
+ # set by jni: options.setLevelCompactionDynamicLevelBytes
+ level_compaction_dynamic_level_bytes=true
\ No newline at end of file
diff --git a/bookkeeper-server/src/test/resources/conf/ledger_metadata_rocksdb.conf b/bookkeeper-server/src/test/resources/conf/ledger_metadata_rocksdb.conf
new file mode 100644
index 0000000..0f3a087
--- /dev/null
+++ b/bookkeeper-server/src/test/resources/conf/ledger_metadata_rocksdb.conf
@@ -0,0 +1,29 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+[DBOptions]
+ # set by jni: options.setCreateIfMissing
+ create_if_missing=true
+ # set by jni: options.setInfoLogLevel
+ info_log_level=INFO_LEVEL
+ # set by jni: options.setKeepLogFileNum
+ keep_log_file_num=30
+
+[CFOptions "default"]
+ # set by jni: options.setLogFileTimeToRoll
+ log_file_time_to_roll=86400
\ No newline at end of file
diff --git a/conf/default_rocksdb.conf b/conf/default_rocksdb.conf
new file mode 100644
index 0000000..0f3a087
--- /dev/null
+++ b/conf/default_rocksdb.conf
@@ -0,0 +1,29 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+[DBOptions]
+ # set by jni: options.setCreateIfMissing
+ create_if_missing=true
+ # set by jni: options.setInfoLogLevel
+ info_log_level=INFO_LEVEL
+ # set by jni: options.setKeepLogFileNum
+ keep_log_file_num=30
+
+[CFOptions "default"]
+ # set by jni: options.setLogFileTimeToRoll
+ log_file_time_to_roll=86400
\ No newline at end of file
diff --git a/conf/entry_location_rocksdb.conf b/conf/entry_location_rocksdb.conf
new file mode 100644
index 0000000..df3ac9f
--- /dev/null
+++ b/conf/entry_location_rocksdb.conf
@@ -0,0 +1,69 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+[DBOptions]
+ # set by jni: options.setCreateIfMissing
+ create_if_missing=true
+ # set by jni: options.setInfoLogLevel
+ info_log_level=INFO_LEVEL
+ # set by jni: options.setKeepLogFileNum
+ keep_log_file_num=30
+ # set by jni: options.setLogFileTimeToRoll
+ log_file_time_to_roll=86400
+ # set by jni: options.setMaxBackgroundJobs or options.setIncreaseParallelism
+ max_background_jobs=2
+ # set by jni: options.setMaxSubcompactions
+ max_subcompactions=1
+ # set by jni: options.setMaxTotalWalSize
+ max_total_wal_size=536870912
+ # set by jni: options.setMaxOpenFiles
+ max_open_files=-1
+ # set by jni: options.setDeleteObsoleteFilesPeriodMicros
+ delete_obsolete_files_period_micros=3600000000
+
+[CFOptions "default"]
+ # set by jni: options.setCompressionType
+ compression=kLZ4Compression
+ # set by jni: options.setWriteBufferSize
+ write_buffer_size=67108864
+ # set by jni: options.setMaxWriteBufferNumber
+ max_write_buffer_number=4
+ # set by jni: options.setNumLevels
+ num_levels=7
+ # set by jni: options.setLevelZeroFileNumCompactionTrigger
+ level0_file_num_compaction_trigger=4
+ # set by jni: options.setMaxBytesForLevelBase
+ max_bytes_for_level_base=268435456
+ # set by jni: options.setTargetFileSizeBase
+ target_file_size_base=67108864
+
+[TableOptions/BlockBasedTable "default"]
+ # set by jni: tableOptions.setBlockSize
+ block_size=65536
+ # set by jni: tableOptions.setBlockCache
+ block_cache=206150041
+ # set by jni: tableOptions.setFormatVersion
+ format_version=2
+ # set by jni: tableOptions.setChecksumType
+ checksum=kxxHash
+ # set by jni: tableOptions.setFilterPolicy, bloomfilter:[bits_per_key]:[use_block_based_builder]
+ filter_policy=rocksdb.BloomFilter:10:false
+ # set by jni: tableOptions.setCacheIndexAndFilterBlocks
+ cache_index_and_filter_blocks=true
+ # set by jni: options.setLevelCompactionDynamicLevelBytes
+ level_compaction_dynamic_level_bytes=true
\ No newline at end of file
diff --git a/conf/ledger_metadata_rocksdb.conf b/conf/ledger_metadata_rocksdb.conf
new file mode 100644
index 0000000..0f3a087
--- /dev/null
+++ b/conf/ledger_metadata_rocksdb.conf
@@ -0,0 +1,29 @@
+#/**
+# * Licensed to the Apache Software Foundation (ASF) under one
+# * or more contributor license agreements.  See the NOTICE file
+# * distributed with this work for additional information
+# * regarding copyright ownership.  The ASF licenses this file
+# * to you under the Apache License, Version 2.0 (the
+# * "License"); you may not use this file except in compliance
+# * with the License.  You may obtain a copy of the License at
+# *
+# *     http://www.apache.org/licenses/LICENSE-2.0
+# *
+# * Unless required by applicable law or agreed to in writing, software
+# * distributed under the License is distributed on an "AS IS" BASIS,
+# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# * See the License for the specific language governing permissions and
+# * limitations under the License.
+# */
+
+[DBOptions]
+ # set by jni: options.setCreateIfMissing
+ create_if_missing=true
+ # set by jni: options.setInfoLogLevel
+ info_log_level=INFO_LEVEL
+ # set by jni: options.setKeepLogFileNum
+ keep_log_file_num=30
+
+[CFOptions "default"]
+ # set by jni: options.setLogFileTimeToRoll
+ log_file_time_to_roll=86400
\ No newline at end of file