You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@bookkeeper.apache.org by GitBox <gi...@apache.org> on 2018/04/03 20:43:59 UTC

[GitHub] sijie closed pull request #1294: BK configuration file updates

sijie closed pull request #1294: BK configuration file updates
URL: https://github.com/apache/bookkeeper/pull/1294
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

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 aa87622b0..2aaea61f5 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
@@ -465,7 +465,7 @@ public ServerConfiguration setPageSize(int pageSize) {
      * @return minimum size of initial file info cache.
      */
     public int getFileInfoCacheInitialCapacity() {
-        return getInt(FILEINFO_CACHE_INITIAL_CAPACITY, 64);
+        return getInt(FILEINFO_CACHE_INITIAL_CAPACITY, Math.max(getOpenFileLimit() / 4, 64));
     }
 
     /**
@@ -2062,7 +2062,7 @@ public ServerConfiguration setCompactionRateByBytes(int rate) {
      */
     @Beta
     public boolean getJournalRemovePagesFromCache() {
-        return getBoolean(JOURNAL_REMOVE_FROM_PAGE_CACHE, false);
+        return getBoolean(JOURNAL_REMOVE_FROM_PAGE_CACHE, true);
     }
 
     /**
diff --git a/conf/bk_server.conf b/conf/bk_server.conf
index 49f7b23f2..8c294b9a9 100755
--- a/conf/bk_server.conf
+++ b/conf/bk_server.conf
@@ -1,5 +1,3 @@
-#!/bin/sh
-#
 #/**
 # * Licensed to the Apache Software Foundation (ASF) under one
 # * or more contributor license agreements.  See the NOTICE file
@@ -18,38 +16,30 @@
 # * limitations under the License.
 # */
 
-## Bookie settings
+# The bookie server configuration are organized in sections
+#
+# - `Bookie Server`     : bookie server generic settings, including network related settings.
+# - `Bookie Storage`    : bookie storage related settings, such as journal, entrylogger, gc and ledger storages.
+# - `Security`          : security related settings
+# - `Metadata Services` : metadata service related settings 
+# - `Stats Providers`   : stats providers related settings
+# - `Auto Recovery`     : auto recovery related settings
+# 
+
+############################################## Bookie Server ##############################################
+
 
 #############################################################################
 ## Server parameters
 #############################################################################
 
-# Port that bookie server listen on
+# The port that the bookie server listens on.
 bookiePort=3181
 
-# Directories BookKeeper outputs its write ahead log.
-# Could define multi directories to store write head logs, separated by ','.
-# For example:
-#   journalDirectories=/tmp/bk-journal1,/tmp/bk-journal2
-# If journalDirectories is set, bookies will skip journalDirectory and use
-# this setting directory.
-# journalDirectories=/tmp/bk-journal
-
-# Directory Bookkeeper outputs its write ahead log
-# @deprecated since 4.5.0. journalDirectories is preferred over journalDirectory.
-journalDirectory=/tmp/bk-txn
-
 # Configure the bookie to allow/disallow multiple ledger/index/journal directories
 # in the same filesystem disk partition
 # allowMultipleDirsUnderSameDiskPartition=false
 
-# Directories to store index files. If not specified, will use ledgerDirectories to store.
-# indexDirectories=/tmp/bk-data
-
-# Minimum safe Usable size to be available in index directory for bookie to create
-# Index File while replaying journal at the time of bookie Start in Readonly Mode (in bytes)
-# minUsableSizeForIndexFileCreation=1073741824
-
 # Set the network interface that the bookie should listen on.
 # If not set, the bookie will listen on all interfaces.
 # listeningInterface=eth0
@@ -72,34 +62,16 @@ journalDirectory=/tmp/bk-txn
 # set the listening interface.
 # allowLoopback=false
 
-# Interval to watch whether bookie is dead or not, in milliseconds
-# bookieDeathWatchInterval=1000
-
-# When entryLogPerLedgerEnabled is enabled, checkpoint doesn't happens
-# when a new active entrylog is created / previous one is rolled over.
-# Instead SyncThread checkpoints periodically with 'flushInterval' delay
-# (in milliseconds) in between executions. Checkpoint flushes both ledger 
-# entryLogs and ledger index pages to disk. 
-# Flushing entrylog and index files will introduce much random disk I/O.
-# If separating journal dir and ledger dirs each on different devices,
-# flushing would not affect performance. But if putting journal dir
-# and ledger dirs on same device, performance degrade significantly
-# on too frequent flushing. You can consider increment flush interval
-# to get better performance, but you need to pay more time on bookie
-# server restart after failure.
-# This config is used only when entryLogPerLedgerEnabled is enabled.
-# flushInterval=10000
-
-# Allow the expansion of bookie storage capacity. Newly added ledger
-# and index dirs must be empty.
-# allowStorageExpansion=false
-
 # Whether the bookie should use its hostname to register with the
 # co-ordination service(eg: Zookeeper service).
 # When false, bookie will use its ipaddress for the registration.
 # Defaults to false.
 # useHostNameAsBookieID=false
 
+# If bookie is using hostname for registration and in ledger metadata then
+# whether to use short hostname or FQDN hostname. Defaults to false.
+# useShortHostName=false
+
 # Whether the bookie is allowed to use an ephemeral port (port 0) as its
 # server port. By default, an ephemeral port is not allowed.
 # Using an ephemeral port as the service port usually indicates a configuration
@@ -114,34 +86,18 @@ journalDirectory=/tmp/bk-txn
 # this bookie will be available only to BookKeeper clients executed on the local JVM.
 # disableServerSocketBind=false
 
-# The number of bytes we should use as chunk allocation for
-# org.apache.bookkeeper.bookie.SkipListArena
-# skipListArenaChunkSize=4194304
-
-# The max size we should allocate from the skiplist arena. Allocations
-# larger than this should be allocated directly by the VM to avoid fragmentation.
-# skipListArenaMaxAllocSize=131072
-
-# The bookie authentication provider factory class name.
-# If this is null, no authentication will take place.
-# bookieAuthProviderFactoryClass=null
+# Interval to watch whether bookie is dead or not, in milliseconds
+# bookieDeathWatchInterval=1000
 
+# Configure a list of server components to enable and load on a bookie server.
+# This provides the plugin run extra services along with a bookie server.
+#
+# extraServerComponents=
 
 #############################################################################
-## Garbage collection settings
+## Thread settings
 #############################################################################
 
-# How long the interval to trigger next garbage collection, in milliseconds
-# Since garbage collection is running in background, too frequent gc
-# will heart performance. It is better to give a higher number of gc
-# interval if there is enough disk capacity.
-# gcWaitTime=1000
-
-# How long the interval to trigger next garbage collection of overreplicated
-# ledgers, in milliseconds [Default: 1 day]. This should not be run very frequently
-# since we read the metadata for all the ledgers on the bookie from zk
-# gcOverreplicatedLedgerWaitTime=86400000
-
 # Number of threads that should handle write requests. if zero, the writes would
 # be handled by netty threads directly.
 # numAddWorkerThreads=1
@@ -150,6 +106,13 @@ journalDirectory=/tmp/bk-txn
 # be handled by netty threads directly.
 # numReadWorkerThreads=8
 
+# The number of threads that should handle long poll requests.
+# numLongPollWorkerThreads=10
+
+# The number of threads used for handling journal callback. If a zero or negative number is provided,
+# the callbacks are executed directly at force write threads.
+# numJournalCallbackThreads=1
+
 # Number of threads that should be used for high priority requests
 # (i.e. recovery reads and adds, and fencing).
 # numHighPriorityWorkerThreads=8
@@ -162,53 +125,10 @@ journalDirectory=/tmp/bk-txn
 # avoid the executor queue to grow indefinitely
 # maxPendingAddRequestsPerThread=10000
 
-# Whether force compaction is allowed when the disk is full or almost full.
-# Forcing GC may get some space back, but may also fill up disk space more quickly.
-# This is because new log files are created before GC, while old garbage
-# log files are deleted after GC.
-# isForceGCAllowWhenNoSpace=false
-
-# True if the bookie should double check readMetadata prior to gc
-# verifyMetadataOnGC=false
-
-#############################################################################
-## TLS settings
-#############################################################################
-
-# TLS Provider (JDK or OpenSSL).
-# tlsProvider=OpenSSL
-
-# The path to the class that provides security.
-# tlsProviderFactoryClass=org.apache.bookkeeper.security.SSLContextFactory
-
-# Type of security used by server.
-# tlsClientAuthentication=true
-
-# Bookie Keystore type.
-# tlsKeyStoreType=JKS
-
-# Bookie Keystore location (path).
-# tlsKeyStore=null
-
-# Bookie Keystore password path, if the keystore is protected by a password.
-# tlsKeyStorePasswordPath=null
-
-# Bookie Truststore type.
-# tlsTrustStoreType=null
-
-# Bookie Truststore location (path).
-# tlsTrustStore=null
-
-# Bookie Truststore password path, if the trust store is protected by a password.
-# tlsTrustStorePasswordPath=null
-
 #############################################################################
 ## Long poll request parameter settings
 #############################################################################
 
-# The number of threads that should handle long poll requests.
-# numLongPollWorkerThreads=10
-
 # The tick duration in milliseconds for long poll requests.
 # requestTimerTickDurationMs=10
 
@@ -216,31 +136,21 @@ journalDirectory=/tmp/bk-txn
 # requestTimerNumTicks=1024
 
 #############################################################################
-## AutoRecovery settings
+## Read-only mode support
 #############################################################################
 
-# The interval between auditor bookie checks.
-# The auditor bookie check, checks ledger metadata to see which bookies should
-# contain entries for each ledger. If a bookie which should contain entries is
-# unavailable, then the ledger containing that entry is marked for recovery.
-# Setting this to 0 disabled the periodic check. Bookie checks will still
-# run when a bookie fails.
-# The interval is specified in seconds.
-# auditorPeriodicBookieCheckInterval=86400
-
-# The number of entries that a replication will rereplicate in parallel.
-# rereplicationEntryBatchSize=10
-
-# Auto-replication
-# The grace period, in seconds, that the replication worker waits before fencing and
-# replicating a ledger fragment that's still being written to upon bookie failure.
-# openLedgerRereplicationGracePeriod=30
+# If all ledger directories configured are full, then support only read requests for clients.
+# If "readOnlyModeEnabled=true" then on all ledger disks full, bookie will be converted
+# to read-only mode and serve only read requests. Otherwise the bookie will be shutdown.
+# By default this will be disabled.
+# readOnlyModeEnabled=true
 
-# Whether the bookie itself can start auto-recovery service also or not
-# autoRecoveryDaemonEnabled=false
+# Whether the bookie is force started in read only mode or not
+# forceReadOnlyBookie=false
 
-# How long to wait, in seconds, before starting auto recovery of a lost bookie
-# lostBookieRecoveryDelay=0
+# Persiste the bookie status locally on the disks. So the bookies can keep their status upon restarts
+# @Since 4.6
+# persistBookieStatusEnabled=false
 
 #############################################################################
 ## Netty server settings
@@ -273,10 +183,83 @@ journalDirectory=/tmp/bk-txn
 # The Recv ByteBuf allocator max buf size.
 # byteBufAllocatorSizeMax=1048576
 
+# The maximum netty frame size in bytes. Any message received larger than this will be rejeted. Default value is 5MB.
+# nettyMaxFrameSizeBytes=5242880
+
+#############################################################################
+##  server settings
+#############################################################################
+
+# The flag enables/disables starting the admin http server. Default value is 'false'.
+# httpServerEnabled=false
+
+# The http server port to listen on. Default value is 8080.
+# httpServerPort=8080
+
+############################################## Security ##############################################
+
+# The bookie authentication provider factory class name.
+# If this is null, no authentication will take place.
+# bookieAuthProviderFactoryClass=null
+
+# The list of users are permitted to run the bookie process. any users can run the bookie process if it is not set.
+#
+# Example settings:
+#   permittedStartupUsers=user1,user2,user3
+#
+# permittedStartupUsers=
+
+#############################################################################
+## TLS settings
+#############################################################################
+
+# TLS Provider (JDK or OpenSSL).
+# tlsProvider=OpenSSL
+
+# The path to the class that provides security.
+# tlsProviderFactoryClass=org.apache.bookkeeper.security.SSLContextFactory
+
+# Type of security used by server.
+# tlsClientAuthentication=true
+
+# Bookie Keystore type.
+# tlsKeyStoreType=JKS
+
+# Bookie Keystore location (path).
+# tlsKeyStore=null
+
+# Bookie Keystore password path, if the keystore is protected by a password.
+# tlsKeyStorePasswordPath=null
+
+# Bookie Truststore type.
+# tlsTrustStoreType=null
+
+# Bookie Truststore location (path).
+# tlsTrustStore=null
+
+# Bookie Truststore password path, if the trust store is protected by a password.
+# tlsTrustStorePasswordPath=null
+
+
+############################################## Bookie Storage ##############################################
+
+
 #############################################################################
 ## Journal settings
 #############################################################################
 
+# Directories BookKeeper outputs its write ahead log.
+# Could define multi directories to store write head logs, separated by ','.
+# For example:
+#   journalDirectories=/tmp/bk-journal1,/tmp/bk-journal2
+# If journalDirectories is set, bookies will skip journalDirectory and use
+# this setting directory.
+journalDirectories=/tmp/bk-txn
+
+# Directory Bookkeeper outputs its write ahead log
+# @deprecated since 4.5.0. journalDirectories is preferred over journalDirectory.
+# journalDirectory=/tmp/bk-txn
+
 # The journal format version to write.
 # Available formats are 1-5:
 # 1: no header
@@ -304,7 +287,7 @@ journalDirectory=/tmp/bk-txn
 # journalWriteBufferSizeKB=64
 
 # Should we remove pages from page cache after force write
-# journalRemoveFromPageCache=false
+# journalRemoveFromPageCache=true
 
 # Should the data be fsynced on journal before acknowledgment.
 # By default, data sync is enabled to guarantee durability of writes.
@@ -319,14 +302,11 @@ journalDirectory=/tmp/bk-txn
 # journalAdaptiveGroupWrites=true
 
 # Maximum latency to impose on a journal write to achieve grouping
-# journalMaxGroupWaitMSec=200
+# journalMaxGroupWaitMSec=2
 
 # Maximum writes to buffer to achieve grouping
 # journalBufferedWritesThreshold=524288
 
-# The number of threads that should handle journal callbacks
-# numJournalCallbackThreads=1
-
 # All the journal writes and commits should be aligned to given size.
 # If not, zeros will be padded to align to given size.
 # It only takes effects when journalFormatVersionToWrite is set to 5
@@ -343,8 +323,20 @@ journalDirectory=/tmp/bk-txn
 #############################################################################
 
 # Ledger storage implementation class
+#
+# Options:
+#   - org.apache.bookkeeper.bookie.InterleavedLedgerStorage
+#   - org.apache.bookkeeper.bookie.SortedLedgerStorage
+#   - org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage
+# Default value:
+#   org.apache.bookkeeper.bookie.SortedLedgerStorage
+#
 # ledgerStorageClass=org.apache.bookkeeper.bookie.SortedLedgerStorage
 
+# @Deprecated - `sortedLedgerStorageEnabled` is deprecated in favor of using `ledgerStorageClass`
+# Whether sorted-ledger storage enabled (default true)
+# sortedLedgerStorageEnabled=ture
+
 # Directory Bookkeeper outputs ledger snapshots
 # could define multi directories to store snapshots, separated by ','
 # For example:
@@ -354,70 +346,34 @@ journalDirectory=/tmp/bk-txn
 # which reduce the contention between random i/o and sequential write.
 # It is possible to run with a single disk, but performance will be significantly lower.
 ledgerDirectories=/tmp/bk-data
+# Directories to store index files. If not specified, will use ledgerDirectories to store.
+# indexDirectories=/tmp/bk-data
 
-# Interval at which the auditor will do a check of all ledgers in the cluster.
-# By default this runs once a week. The interval is set in seconds.
-# To disable the periodic check completely, set this to 0.
-# Note that periodic checking will put extra load on the cluster, so it should
-# not be run more frequently than once a day.
-# auditorPeriodicCheckInterval=604800
+# Minimum safe Usable size to be available in index directory for bookie to create
+# Index File while replaying journal at the time of bookie Start in Readonly Mode (in bytes)
+# minUsableSizeForIndexFileCreation=1073741824
 
-# Whether sorted-ledger storage enabled (default true)
-# sortedLedgerStorageEnabled=ture
+# When entryLogPerLedgerEnabled is enabled, checkpoint doesn't happens
+# when a new active entrylog is created / previous one is rolled over.
+# Instead SyncThread checkpoints periodically with 'flushInterval' delay
+# (in milliseconds) in between executions. Checkpoint flushes both ledger 
+# entryLogs and ledger index pages to disk. 
+# Flushing entrylog and index files will introduce much random disk I/O.
+# If separating journal dir and ledger dirs each on different devices,
+# flushing would not affect performance. But if putting journal dir
+# and ledger dirs on same device, performance degrade significantly
+# on too frequent flushing. You can consider increment flush interval
+# to get better performance, but you need to pay more time on bookie
+# server restart after failure.
+# This config is used only when entryLogPerLedgerEnabled is enabled.
+# flushInterval=10000
 
-# The skip list data size limitation (default 64MB) in EntryMemTable
-# skipListSizeLimit=67108864L
+# Allow the expansion of bookie storage capacity. Newly added ledger
+# and index dirs must be empty.
+# allowStorageExpansion=false
 
 #############################################################################
-## Ledger cache settings
-#############################################################################
-
-# Max number of ledger index files could be opened in bookie server
-# If number of ledger index files reaches this limitation, bookie
-# server started to swap some ledgers from memory to disk.
-# Too frequent swap will affect performance. You can tune this number
-# to gain performance according your requirements.
-# openFileLimit=900
-
-# Size of a index page in ledger cache, in bytes
-# A larger index page can improve performance writing page to disk,
-# which is efficent when you have small number of ledgers and these
-# ledgers have similar number of entries.
-# If you have large number of ledgers and each ledger has fewer entries,
-# smaller index page would improve memory usage.
-# pageSize=8192
-
-# How many index pages provided in ledger cache
-# If number of index pages reaches this limitation, bookie server
-# starts to swap some ledgers from memory to disk. You can increment
-# this value when you found swap became more frequent. But make sure
-# pageLimit*pageSize should not more than JVM max memory limitation,
-# otherwise you would got OutOfMemoryException.
-# In general, incrementing pageLimit, using smaller index page would
-# gain bettern performance in lager number of ledgers with fewer entries case
-# If pageLimit is -1, bookie server will use 1/3 of JVM memory to compute
-# the limitation of number of index pages.
-# pageLimit=-1
-
-#############################################################################
-## Ledger manager settings
-#############################################################################
-
-# Ledger Manager Class
-# What kind of ledger manager is used to manage how ledgers are stored, managed
-# and garbage collected. Try to read 'BookKeeper Internals' for detail info.
-ledgerManagerFactoryClass=org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory
-
-# @Drepcated - `ledgerManagerType` is deprecated in favor of using `ledgerManagerFactoryClass`.
-# ledgerManagerType=hierarchical
-
-# Root Zookeeper path to store ledger metadata
-# This parameter is used by zookeeper-based ledger manager as a root znode to
-# store all ledgers.
-# zkLedgersRootPath=/ledgers
-
-#############################################################################
-## Entry log settings
+## Entry log settings
 #############################################################################
 
 # Max file size of entry logger, in bytes
@@ -453,10 +409,6 @@ ledgerManagerFactoryClass=org.apache.bookkeeper.meta.HierarchicalLedgerManagerFa
 # Set the rate at which compaction will readd entries. The unit is adds per second.
 # compactionRate=1000
 
-# If bookie is using hostname for registration and in ledger metadata then
-# whether to use short hostname or FQDN hostname. Defaults to false.
-# useShortHostName=false
-
 # Threshold of minor compaction
 # For those entry log files whose remaining size percentage reaches below
 # this threshold will be compacted in a minor compaction.
@@ -498,32 +450,34 @@ ledgerManagerFactoryClass=org.apache.bookkeeper.meta.HierarchicalLedgerManagerFa
 # Set the rate at which compaction will readd entries. The unit is bytes added per second.
 # compactionRateByBytes=1000000
 
-#############################################################################
-## Statistics
-#############################################################################
-
-# Whether statistics are enabled
-# enableStatistics=true
-
-# Stats Provider Class (if statistics are enabled)
-# statsProviderClass=org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
+# Flag to enable/disable transactional compaction. If it is set to true, it will use transactional compaction,
+# which it will use new entry log files to store compacted entries during compaction; if it is set to false,
+# it will use normal compaction, which it shares same entry log file with normal add operations.
+# useTransactionalCompaction=false
 
 #############################################################################
-## Read-only mode support
+## Garbage collection settings
 #############################################################################
 
-# If all ledger directories configured are full, then support only read requests for clients.
-# If "readOnlyModeEnabled=true" then on all ledger disks full, bookie will be converted
-# to read-only mode and serve only read requests. Otherwise the bookie will be shutdown.
-# By default this will be disabled.
-# readOnlyModeEnabled=false
+# How long the interval to trigger next garbage collection, in milliseconds
+# Since garbage collection is running in background, too frequent gc
+# will heart performance. It is better to give a higher number of gc
+# interval if there is enough disk capacity.
+# gcWaitTime=1000
 
-# Whether the bookie is force started in read only mode or not
-# forceReadOnlyBookie=false
+# How long the interval to trigger next garbage collection of overreplicated
+# ledgers, in milliseconds [Default: 1 day]. This should not be run very frequently
+# since we read the metadata for all the ledgers on the bookie from zk
+# gcOverreplicatedLedgerWaitTime=86400000
 
-# Persiste the bookie status locally on the disks. So the bookies can keep their status upon restarts
-# @Since 4.6
-# persistBookieStatusEnabled=false
+# Whether force compaction is allowed when the disk is full or almost full.
+# Forcing GC may get some space back, but may also fill up disk space more quickly.
+# This is because new log files are created before GC, while old garbage
+# log files are deleted after GC.
+# isForceGCAllowWhenNoSpace=false
+
+# True if the bookie should double check readMetadata prior to gc
+# verifyMetadataOnGC=false
 
 #############################################################################
 ## Disk utilization
@@ -558,43 +512,62 @@ ledgerManagerFactoryClass=org.apache.bookkeeper.meta.HierarchicalLedgerManagerFa
 # diskCheckInterval=10000
 
 #############################################################################
-## ZooKeeper parameters
+## Sorted Ledger storage configuration
 #############################################################################
 
-# A list of one of more servers on which Zookeeper is running.
-# The server list can be comma separated values, for example:
-# zkServers=zk1:2181,zk2:2181,zk3:2181
-zkServers=localhost:2181
+# These configs are used when the selected 'ledgerStorageClass' is
+# org.apache.bookkeeper.bookie.SortedLedgerStorage
+# ledgerStorageClass=org.apache.bookkeeper.bookie.SortedLedgerStorage
 
-# ZooKeeper client session timeout in milliseconds
-# Bookie server will exit if it received SESSION_EXPIRED because it
-# was partitioned off from ZooKeeper for more than the session timeout
-# JVM garbage collection, disk I/O will cause SESSION_EXPIRED.
-# Increment this value could help avoiding this issue
-zkTimeout=10000
+# The skip list data size limitation (default 64MB) in EntryMemTable
+# skipListSizeLimit=67108864
 
-# The Zookeeper client backoff retry start time in millis.
-# zkRetryBackoffStartMs=1000
+# The number of bytes we should use as chunk allocation for
+# org.apache.bookkeeper.bookie.SkipListArena
+# skipListArenaChunkSize=4194304
 
-# The Zookeeper client backoff retry max time in millis.
-# zkRetryBackoffMaxMs=10000
+# The max size we should allocate from the skiplist arena. Allocations
+# larger than this should be allocated directly by the VM to avoid fragmentation.
+# skipListArenaMaxAllocSize=131072
 
-# Set ACLs on every node written on ZooKeeper, this way only allowed users
-# will be able to read and write BookKeeper metadata stored on ZooKeeper.
-# In order to make ACLs work you need to setup ZooKeeper JAAS authentication
-# all the bookies and Client need to share the same user, and this is usually
-# done using Kerberos authentication. See ZooKeeper documentation
-zkEnableSecurity=false
+# Max number of ledger index files could be opened in bookie server
+# If number of ledger index files reaches this limitation, bookie
+# server started to swap some ledgers from memory to disk.
+# Too frequent swap will affect performance. You can tune this number
+# to gain performance according your requirements.
+# openFileLimit=20000
 
-#############################################################################
-## Server parameters
-#############################################################################
+# The minimum total size of the internal file info cache table. Providing a large enough
+# estimate at construction time avoids the need for expensive resizing operations later,
+# but setting this value unnecessarily high wastes memory. The default value is `1/4` of
+# `openFileLimit` if openFileLimit is positive, otherwise it is 64.
+# fileInfoCacheInitialCapacity=
 
-# Configure a list of server components to enable and load on a bookie server.
-# This provides the plugin run extra services along with a bookie server.
-#
-# extraServerComponents=
+# The max idle time allowed for an open file info existed in the file info cache.
+# If the file info is idle for a long time, exceed the given time period. The file info
+# will be evicted and closed. If the value is zero or negative, the file info is evicted
+# only when opened files reached openFileLimit. The default value is 0.
+# fileInfoMaxIdleTime=0
 
+# Size of a index page in ledger cache, in bytes
+# A larger index page can improve performance writing page to disk,
+# which is efficent when you have small number of ledgers and these
+# ledgers have similar number of entries.
+# If you have large number of ledgers and each ledger has fewer entries,
+# smaller index page would improve memory usage.
+# pageSize=8192
+
+# How many index pages provided in ledger cache
+# If number of index pages reaches this limitation, bookie server
+# starts to swap some ledgers from memory to disk. You can increment
+# this value when you found swap became more frequent. But make sure
+# pageLimit*pageSize should not more than JVM max memory limitation,
+# otherwise you would got OutOfMemoryException.
+# In general, incrementing pageLimit, using smaller index page would
+# gain bettern performance in lager number of ledgers with fewer entries case
+# If pageLimit is -1, bookie server will use 1/3 of JVM memory to compute
+# the limitation of number of index pages.
+# pageLimit=-1
 
 #############################################################################
 ## DB Ledger storage configuration
@@ -602,6 +575,7 @@ zkEnableSecurity=false
 
 # These configs are used when the selected 'ledgerStorageClass' is
 # org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage
+# ledgerStorageClass=org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage
 
 # Size of Write Cache. Memory is allocated from JVM direct memory.
 # Write cache is used to buffer entries before flushing into the entry log
@@ -633,3 +607,253 @@ zkEnableSecurity=false
 # dbStorage_rocksDB_numLevels=-1
 # dbStorage_rocksDB_numFilesInLevel0=4
 # dbStorage_rocksDB_maxSizeInLevel1MB=256
+
+
+############################################## Metadata Services ##############################################
+
+
+#############################################################################
+## Metadata Service settings
+#############################################################################
+
+# metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location
+metadataServiceUri="zk+hierarchical://localhost:2181/ledgers"
+
+# @Deprecated - `ledgerManagerFactoryClass` is deprecated in favor of using `metadataServiceUri`
+# Ledger Manager Class
+# What kind of ledger manager is used to manage how ledgers are stored, managed
+# and garbage collected. Try to read 'BookKeeper Internals' for detail info.
+# ledgerManagerFactoryClass=org.apache.bookkeeper.meta.HierarchicalLedgerManagerFactory
+
+# @Deprecated - `ledgerManagerType` is deprecated in favor of using `ledgerManagerFactoryClass`.
+# ledgerManagerType=hierarchical
+
+# sometimes the bookkeeper server classes are shaded. the ledger manager factory classes might be relocated to be under other packages.
+# this would fail the clients using shaded factory classes since the factory classes are not matched. Users can enable this flag to
+# allow using shaded ledger manager class to connect to a bookkeeper cluster.
+# allowShadedLedgerManagerFactoryClass=false
+
+# the shaded ledger manager factory prefix. this is used when `allowShadedLedgerManagerFactoryClass` is set to true.
+# shadedLedgerManagerFactoryClassPrefix="dlshade."
+
+#############################################################################
+## ZooKeeper Metadata Service settings
+#############################################################################
+
+# @Deprecated - `zkLedgersRootPath` is deprecated in favor of using `metadataServiceUri`
+# Root Zookeeper path to store ledger metadata
+# This parameter is used by zookeeper-based ledger manager as a root znode to
+# store all ledgers.
+# zkLedgersRootPath=/ledgers
+
+# @Deprecated - `zkLedgersRootPath` is deprecated in favor of using `metadataServiceUri`
+# A list of one of more servers on which Zookeeper is running.
+# The server list can be comma separated values, for example:
+# zkServers=zk1:2181,zk2:2181,zk3:2181
+zkServers=localhost:2181
+
+# ZooKeeper client session timeout in milliseconds
+# Bookie server will exit if it received SESSION_EXPIRED because it
+# was partitioned off from ZooKeeper for more than the session timeout
+# JVM garbage collection, disk I/O will cause SESSION_EXPIRED.
+# Increment this value could help avoiding this issue
+zkTimeout=10000
+
+# The Zookeeper client backoff retry start time in millis.
+# zkRetryBackoffStartMs=1000
+
+# The Zookeeper client backoff retry max time in millis.
+# zkRetryBackoffMaxMs=10000
+
+# The Zookeeper request limit. It is only enabled when setting a positive value. Default value is 0.
+# zkRequestRateLimit=0
+
+# Set ACLs on every node written on ZooKeeper, this way only allowed users
+# will be able to read and write BookKeeper metadata stored on ZooKeeper.
+# In order to make ACLs work you need to setup ZooKeeper JAAS authentication
+# all the bookies and Client need to share the same user, and this is usually
+# done using Kerberos authentication. See ZooKeeper documentation
+zkEnableSecurity=false
+
+
+############################################## Statistics ##############################################
+
+
+#############################################################################
+## Stats Providers
+#############################################################################
+
+# Whether statistics are enabled
+# enableStatistics=true
+
+# The flag to enable recording task execution stats.
+# enableTaskExecutionStats=false
+
+# Stats Provider Class (if `enableStatistics` are enabled)
+# Options:
+#   - Prometheus        : org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
+#   - Codahale          : org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider
+#   - Twitter Finagle   : org.apache.bookkeeper.stats.twitter.finagle.FinagleStatsProvider
+#   - Twitter Ostrich   : org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider
+#   - Twitter Science   : org.apache.bookkeeper.stats.twitter.science.TwitterStatsProvider
+# Default value is:
+#   org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
+# 
+# For configuring corresponding stats provider, see details at each section below.
+#
+# statsProviderClass=org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
+
+#############################################################################
+## Prometheus Metrics Provider
+#############################################################################
+
+# These configs are used when using `PrometheusMetricsProvider`.
+# statsProviderClass=org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
+
+# Default port for Prometheus metrics exporter
+# prometheusStatsHttpPort=8000
+
+# latency stats rollover interval, in seconds
+# prometheusStatsLatencyRolloverSeconds=60
+
+#############################################################################
+## Codahale Metrics Provider
+#############################################################################
+
+# These configs are used when using `CodahaleMetricsProvider`.
+# statsProviderClass=org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider
+
+# metric name prefix, default is ""
+# codahaleStatsPrefix=
+
+# the frequency that stats reporters report stats, in seconds
+# codahaleStatsOutputFrequencySeconds=60
+
+# the graphite endpoint for reporting stats. see {@link http://metrics.dropwizard.io/3.1.0/manual/graphite/} for more details.
+# codahaleStatsGraphiteEndpoint=
+
+# the directory for reporting stats in csv format. see {@link https://metrics.dropwizard.io/3.1.0/manual/core/#csv} for more details.
+# codahaleStatsCSVEndpoint=
+
+# the slf4j endpoint for reporting stats. see {@link https://metrics.dropwizard.io/3.1.0/manual/core/#slf4j} for more details.
+# codahaleStatsSlf4jEndpoint=
+
+# the jmx endpoint for reporting stats. see {@link https://metrics.dropwizard.io/3.1.0/manual/core/#jmx} for more details.
+# codahaleStatsJmxEndpoint=
+
+#############################################################################
+## Twitter Finagle Metrics Provider
+#############################################################################
+
+# These configs are used when using `FinagleStatsProvider`.
+# statsProviderClass=org.apache.bookkeeper.stats.twitter.finagle.FinagleStatsProvider
+
+#############################################################################
+## Twitter Ostrich Metrics Provider
+#############################################################################
+
+# These configs are used when using `OstrichProvider`.
+# statsProviderClass=org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider
+
+# Flag to control whether to expose ostrich metrics via a http endpoint configured by `statsHttpPort`.
+# statsExport=false
+
+# The http port of exposing ostrich stats if `statsExport` is set to true
+# statsHttpPort=9002
+
+#############################################################################
+## Twitter Science Metrics Provider
+#############################################################################
+
+# These configs are used when using `TwitterStatsProvider`.
+# statsProviderClass=org.apache.bookkeeper.stats.twitter.science.TwitterStatsProvider
+
+# Flag to control whether to expose metrics throught a http endpoint configured by `statsHttpPort`.
+# statsExport=false
+
+# The http port of exposing stats if `statsExport` is set to true
+# statsHttpPort=9002
+
+
+############################################## Auto Recovery ##############################################
+
+# Whether the bookie itself can start auto-recovery service also or not
+# autoRecoveryDaemonEnabled=false
+
+# The default digest type used for opening ledgers.
+# digestType=CRC32
+
+# The default password for opening ledgers. Default value is "".
+# passwd=
+
+# The flag to enable/disable digest type auto-detection. If it is enabled, the bookkeeper client will ignore
+# the provided digest type provided at `digestType` and the provided passwd provided at `passwd`.
+# enableDigestTypeAutodetection=true
+
+#############################################################################
+## Placement settings
+#############################################################################
+
+# the following settings take effects when `autoRecoveryDaemonEnabled` is true.
+
+# The ensemble placement policy used for re-replicating entries.
+# 
+# Options:
+#   - org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
+#   - org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy
+#
+# Default value:
+#   org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
+#
+# ensemblePlacementPolicy=org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
+
+# The dns resolver class used for resolving network locations for bookies. This setting is used
+# when using either RackawareEnsemblePlacementPolicy and RegionAwareEnsemblePlacementPolicy
+# reppDnsResolverClass=org.apache.bookkeeper.net.ScriptBasedMapping
+
+# The bash script used by `ScriptBasedMapping` dns resolver for resolving bookies' network locations.
+# networkTopologyScriptFileName=
+
+# The max number of args used in the script provided at `networkTopologyScriptFileName`
+# networkTopologyScriptNumberArgs=100
+
+#############################################################################
+## Auditor settings
+#############################################################################
+
+# The interval between auditor bookie checks.
+# The auditor bookie check, checks ledger metadata to see which bookies should
+# contain entries for each ledger. If a bookie which should contain entries is
+# unavailable, then the ledger containing that entry is marked for recovery.
+# Setting this to 0 disabled the periodic check. Bookie checks will still
+# run when a bookie fails.
+# The interval is specified in seconds.
+# auditorPeriodicBookieCheckInterval=86400
+
+# Interval at which the auditor will do a check of all ledgers in the cluster.
+# By default this runs once a week. The interval is set in seconds.
+# To disable the periodic check completely, set this to 0.
+# Note that periodic checking will put extra load on the cluster, so it should
+# not be run more frequently than once a day.
+# auditorPeriodicCheckInterval=604800
+
+# The percentage of a ledger (fragment)'s entries will be verified before claiming this fragment as missing fragment.
+# Default is 0, which only verify the first and last entries of a given fragment.
+# auditorLedgerVerificationPercentage=0
+
+# How long to wait, in seconds, before starting auto recovery of a lost bookie
+# lostBookieRecoveryDelay=0
+
+#############################################################################
+## Replication Worker settings
+#############################################################################
+
+# The number of entries that a replication will rereplicate in parallel.
+# rereplicationEntryBatchSize=10
+
+# The grace period, in seconds, that the replication worker waits before fencing and
+# replicating a ledger fragment that's still being written to upon bookie failure.
+# openLedgerRereplicationGracePeriod=30
+
+# The time to backoff when replication worker encounters exceptions on replicating a ledger, in milliseconds.
+# rwRereplicateBackoffMs=5000
diff --git a/site/_data/config/bk_server.yaml b/site/_data/config/bk_server.yaml
index 93ba270e5..6eeb32927 100644
--- a/site/_data/config/bk_server.yaml
+++ b/site/_data/config/bk_server.yaml
@@ -4,25 +4,9 @@ groups:
   - param: bookiePort
     description: The port that the bookie server listens on.
     default: 3181
-  - param: journalDirectories
-    description: |
-      The directories to which Bookkeeper outputs its write-ahead log (WAL).  Could define multi directories to store write head logs, separated by ','.
-      For example:
-        journalDirectories=/tmp/bk-journal1,/tmp/bk-journal2
-      If journalDirectories is set, bookies will skip journalDirectory and use this setting directory.
-    default: /tmp/bk-journal
-  - param: journalDirectory
-    description: The directory to which Bookkeeper outputs its write-ahead log (WAL).
-    default: /tmp/bk-txn
   - param: allowMultipleDirsUnderSameDiskPartition
     description: Configure the bookie to allow/disallow multiple ledger/index/journal directories in the same filesystem disk partition
     default: false
-  - param: indexDirectories
-    description: The directories in which index files are stored. If not specified, the value of [`ledgerDirectories`](#ledgerDirectories) will be used.
-    default: /tmp/bk-data
-  - param: minUsableSizeForIndexFileCreation
-    description: Minimum safe usable size to be available in index directory for bookie to create index file while replaying journal at the time of bookie start in readonly mode (in bytes)
-    default: 1073741824
   - param: listeningInterface
     description: The network interface that the bookie should listen on. If not set, the bookie will listen on all interfaces.
     default: eth0
@@ -39,18 +23,12 @@ groups:
 
       Using a loopback interface as the primary interface usually indicates a configuration error. It's fairly common in some VPS setups, for example, to not configure a hostname or to have the hostname resolve to 127.0.0.1. If this is the case, then all bookies in the cluster will establish their identities as 127.0.0.1:3181, and only one will be able to join the cluster. For VPSs configured like this, you should explicitly set the listening interface.
     default: 'false'
-  - param: bookieDeathWatchInterval
-    description: Interval to watch whether bookie is dead or not, in milliseconds.
-    default: 1000
-  - param: flushInterval
-    description: When entryLogPerLedgerEnabled is enabled, checkpoint doesn't happens when a new active entrylog is created / previous one is rolled over. Instead SyncThread checkpoints periodically with 'flushInterval' delay (in milliseconds) in between executions. Checkpoint flushes both ledger entryLogs and ledger index pages to disk.  Flushing entrylog and index files will introduce much random disk I/O. If separating journal dir and ledger dirs each on different devices, flushing would not affect performance. But if putting journal dir and ledger dirs on same device, performance degrade significantly on too frequent flushing. You can consider increment flush interval to get better performance, but you need to pay more time on bookie server restart after failure. This config is used only when entryLogPerLedgerEnabled is enabled.
-    default: 10000
-  - param: allowStorageExpansion
-    description: Allow the expansion of bookie storage capacity. Newly added ledger and index directories must be empty.
-    default: 'false'
   - param: useHostNameAsBookieID
     description: Whether the bookie should use its hostname to register with the ZooKeeper coordination service. When `false`, the bookie will use its IP address for the registration.
     default: 'false'
+  - param: useShortHostName
+    description: Whether the bookie should use short hostname or [FQDN](https://en.wikipedia.org/wiki/Fully_qualified_domain_name) hostname for registration and ledger metadata when `useHostNameAsBookieID` is enabled.
+    default: 'false'
   - param: allowEphemeralPorts
     description: Whether the bookie is allowed to use an ephemeral port (port 0) as its server port. By default, an ephemeral port is not allowed. Using an ephemeral port as the service port usually indicates a configuration error. However, in unit tests, using an ephemeral port will address port conflict problems and allow running tests in parallel.
     default: 'false'
@@ -60,39 +38,105 @@ groups:
   - param: disableServerSocketBind
     description: Whether allow the bookie to disable bind on network interfaces, this bookie will be available only to BookKeeper clients executed on the local JVM.
     default: 'false'
-  - param: skipListArenaChunkSize
-    description: The number of bytes we should use as chunk allocation for org.apache.bookkeeper.bookie.SkipListArena
-    default: 4194304
-  - param: skipListArenaMaxAllocSize
-    description: The max size we should allocate from the skiplist arena. Allocations larger than this should be allocated directly by the VM to avoid fragmentation.
-    default: 131072
-  - param: bookieAuthProviderFactoryClass
-    description: The bookie authentication provider factory class name. If this is null, no authentication will take place.
-    default: null
+  - param: bookieDeathWatchInterval
+    description: Interval to watch whether bookie is dead or not, in milliseconds.
+    default: 1000
+  - param: extraServerComponents
+    description: Configure a list of extra server components to enable and load on a bookie server. This provides a plugin mechanism to run extra server components along with a bookie server.
+    default: ''
 
 - name: Worker thread settings
   params:
   - param: numAddWorkerThreads
-    description: The number of threads that handle write requests. if zero, writes are handled by [Netty threads](http://netty.io/wiki/thread-model.html) directly.
+    description: The number of threads that handle write requests. if zero, writes are handled by [Netty threads](//netty.io/wiki/thread-model.html) directly.
     default: 1
   - param: numReadWorkerThreads
-    description: The number of threads that handle read requests. If zero, reads are handled by [Netty threads](http://netty.io/wiki/thread-model.html) directly.
+    description: The number of threads that handle read requests. If zero, reads are handled by [Netty threads](//netty.io/wiki/thread-model.html) directly.
     default: 8
+  - param: numLongPollWorkerThreads
+    description: The number of threads that handle long poll requests. If zero, long poll requests are handled by [Netty threads](//netty.io/wiki/thread-model.html) directly.
+    default: 10
+  - param: numJournalCallbackThreads
+    description: The number of threads that handle journal callbacks. If zero, journal callbacks are executed directly on force write threads.
+    default: 1
   - param: numHighPriorityWorkerThreads
-    description: The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by [Netty threads](http://netty.io/wiki/thread-model.html) directly.
+    description: The number of threads that should be used for high priority requests (i.e. recovery reads and adds, and fencing). If zero, reads are handled by [Netty threads](//netty.io/wiki/thread-model.html) directly.
     default: 8
+  - param: maxPendingAddRequestsPerThread
+    description: If read worker threads are enabled, limit the number of pending requests, to avoid the executor queue to grow indefinitely. If zero or negative, the number of pending requests is unlimited.
+    default: 10000
+  - param: maxPendingReadRequestsPerThread
+    description: If add worker threads are enabled, limit the number of pending requests, to avoid the executor queue to grow indefinitely. If zero or negative, the number of pending requests is unlimited.
+    default: 10000
 
-- name: Garbage collection settings
+- name: Long poll settings
   params:
-  - param: gcWaitTime
-    description: How long the interval to trigger next garbage collection, in milliseconds. Since garbage collection is running in background, too frequent gc will heart performance. It is better to give a higher number of gc interval if there is enough disk capacity.
-    default: 1000
-  - param: gcOverreplicatedLedgerWaitTime
-    description: How long the interval to trigger next garbage collection of overreplicated ledgers, in milliseconds. This should not be run very frequently since we read the metadata for all the ledgers on the bookie from zk.
-    default: 86400000
-  - param: isForceGCAllowWhenNoSpace
-    description: Whether force compaction is allowed when the disk is full or almost full. Forcing GC may get some space back, but may also fill up disk space more quickly. This is because new log files are created before GC, while old garbage log files are deleted after GC.
+  - param: requestTimerTickDurationMs
+    description: The tick duration for long poll request timer, in milliseconds. See [HashedWheelTimer](//netty.io/4.1/api/io/netty/util/HashedWheelTimer.html) for more details.
+    default: 10
+  - param: requestTimerNumTicks
+    description: The number of ticks per wheel for long poll request timer. See [HashedWheelTimer](//netty.io/4.1/api/io/netty/util/HashedWheelTimer.html) for more details.
+    default: 1024
+
+- name: Read-only mode support
+  params:
+  - param: readOnlyModeEnabled
+    description: If all ledger directories configured are full, then support only read requests for clients. If "readOnlyModeEnabled=true" then on all ledger disks full, bookie will be converted to read-only mode and serve only read requests. Otherwise the bookie will be shutdown. By default this will be disabled.
+    default: 'true'
+  - param: forceReadOnlyBookie
+    description: Whether the bookie is force started in read only mode or not.
+    default: 'false'
+  - param: persistBookieStatusEnabled
+    description: Persist the bookie status locally on the disks. So the bookies can keep their status upon restarts.
+    default: 'false'
+
+- name: Netty server settings
+  params:
+  - param: serverTcpNoDelay
+    description: |
+      This settings is used to enabled/disabled Nagle's algorithm, which is a means of improving the efficiency of TCP/IP networks by reducing the number of packets that need to be sent over the network.
+
+      If you are sending many small messages, such that more than one can fit in a single IP packet, setting server.tcpnodelay to false to enable Nagle algorithm can provide better performance.
+    default: 'true'
+  - param: serverSockKeepalive
+    description: This setting is used to send keep-alive messages on connection-oriented sockets.
+    default: 'true'
+  - param: serverTcpLinger
+    description: The socket linger timeout on close. When enabled, a close or shutdown will not return until all queued messages for the socket have been successfully sent or the linger timeout has been reached. Otherwise, the call returns immediately and the closing is done in the background.
+    default: 0
+  - param: byteBufAllocatorSizeInitial
+    description: The Recv ByteBuf allocator initial buf size.
+    default: 65536
+  - param: byteBufAllocatorSizeMin
+    description: The Recv ByteBuf allocator min buf size.
+    default: 65536
+  - param: byteBufAllocatorSizeMax
+    description: The Recv ByteBuf allocator max buf size.
+    default: 1048576
+  - param: nettyMaxFrameSizeBytes
+    description: The maximum netty frame size in bytes. Any message received larger than this will be rejected.
+    default: 5242880
+
+- name: Http server settings
+  params:
+  - param: httpServerEnabled
+    description: The flag enables/disables starting the admin http server.
     default: 'false'
+  - param: httpServerPort
+    description: The http server port to listen on if `httpServerEnabled` is set to true.
+    default: 8080
+
+- name: Security settings
+  params: 
+  - param: bookieAuthProviderFactoryClass
+    description: The bookie authentication provider factory class name. If this is null, no authentication will take place.
+    default: null
+  - param: permittedStartupUsers
+    description: |
+      The list of users are permitted to run the bookie process. Any users can run the bookie process if it is not set.
+
+      Example settings - "permittedStartupUsers=user1,user2,user3"
+    default: null
 
 - name: TLS settings
   params:
@@ -127,64 +171,21 @@ groups:
     description: Bookie Truststore password path, if the truststore is protected by a password.
     default: null
 
-
-- name: Long poll request parameter settings
-  params:
-  - param: numLongPollWorkerThreads
-    description: The number of threads that should handle long poll requests.
-    default: 10
-  - param: requestTimerTickDurationMs
-    description: The tick duration in milliseconds for long poll requests.
-    default: 10
-  - param: requestTimerNumTicks
-    description: The number of ticks per wheel for the long poll request timer.
-    default: 1024
-
-
-- name: AutoRecovery settings
-  params:
-  - param: auditorPeriodicBookieCheckInterval
-    description: The time interval between auditor bookie checks, in seconds. The auditor bookie check checks ledger metadata to see which bookies should contain entries for each ledger. If a bookie that should contain entries is unavailable, then the ledger containing that entry is marked for recovery. Setting this to 0 disables the periodic check. Bookie checks will still run when a bookie fails. The default is once per day.
-    default: 86400
-  - param: rereplicationEntryBatchSize
-    description: The number of entries that a replication will rereplicate in parallel.
-    default: 10
-  - param: openLedgerRereplicationGracePeriod
-    description: The grace period, in seconds, that the replication worker waits before fencing and replicating a ledger fragment that's still being written to upon bookie failure.
-    default: 30
-  - param: autoRecoveryDaemonEnabled
-    description: Whether the bookie itself can start auto-recovery service also or not.
-    default: false
-  - param: lostBookieRecoveryDelay
-    description: How long to wait, in seconds, before starting autorecovery of a lost bookie.
-    default: 0
-
-- name: Netty server settings
+- name: Journal settings
   params:
-  - param: serverTcpNoDelay
+  - param: journalDirectories
     description: |
-      This settings is used to enabled/disabled Nagle's algorithm, which is a means of improving the efficiency of TCP/IP networks by reducing the number of packets that need to be sent over the network.
-
-      If you are sending many small messages, such that more than one can fit in a single IP packet, setting server.tcpnodelay to false to enable Nagle algorithm can provide better performance.
-    default: 'true'
-  - param: serverSockKeepalive
-    description: This setting is used to send keep-alive messages on connection-oriented sockets.
-    default: 'true'
-  - param: serverTcpLinger
-    description: The socket linger timeout on close. When enabled, a close or shutdown will not return until all queued messages for the socket have been successfully sent or the linger timeout has been reached. Otherwise, the call returns immediately and the closing is done in the background.
-    default: 0
-  - param: byteBufAllocatorSizeInitial
-    description: The Recv ByteBuf allocator initial buf size.
-    default: 65536
-  - param: byteBufAllocatorSizeMin
-    description: The Recv ByteBuf allocator min buf size.
-    default: 65536
-  - param: byteBufAllocatorSizeMax
-    description: The Recv ByteBuf allocator max buf size.
-    default: 1048576
+      The directories to which Bookkeeper outputs its write-ahead log (WAL).  Could define multi directories to store write head logs, separated by ','.
+      For example:
+        journalDirectories=/tmp/bk-journal1,/tmp/bk-journal2
+      If journalDirectories is set, bookies will skip journalDirectory and use this setting directory.
+    default: /tmp/bk-journal
+  - param: journalDirectory
+    description: |
+      @Deprecated since 4.5.0, in favor of using `journalDirectories`.
 
-- name: Journal settings
-  params:
+      The directory to which Bookkeeper outputs its write-ahead log (WAL).
+    default: /tmp/bk-txn
   - param: journalFormatVersionToWrite
     description: |
       The journal format version to write.
@@ -212,22 +213,27 @@ groups:
     default: 64
   - param: journalRemoveFromPageCache
     description:  Should we remove pages from page cache after force write
-    default: 'false'
+    default: 'true'
+  - param: journalSyncData
+    description: |
+      Should the data be fsynced on journal before acknowledgment.
+      By default, data sync is enabled to guarantee durability of writes. Beware - when disabling data sync in the bookie journal
+      might improve the bookie write performance, it will also introduce the possibility of data loss. With no fsync, the journal
+      entries are written in the OS page cache but not flushed to disk. In case of power failure, the affected bookie might lose
+      the unflushed data. If the ledger is replicated to multiple bookies, the chances of data loss are reduced though still present.
+    default: 'true'
   - param: journalAdaptiveGroupWrites
     description: Should we group journal force writes, which optimize group commit for higher throughput.
     default: 'true'
   - param: journalMaxGroupWaitMSec
     description: Maximum latency to impose on a journal write to achieve grouping.
-    default: 200
+    default: 2
   - param: journalBufferedWritesThreshold
     description: Maximum writes to buffer to achieve grouping.
     default: 524288
   - param: journalFlushWhenQueueEmpty
     description: If we should flush the journal when journal queue is empty.
     default: 'false'
-  - param: numJournalCallbackThreads
-    description: The number of threads that should handle journal callbacks.
-    default: 1
   - param: journalAlignmentSize
     description: All the journal writes and commits should be aligned to given size. If not, zeros will be padded to align to given size.
     default: 512
@@ -241,47 +247,35 @@ groups:
 - name: Ledger storage settings
   params:
   - param: ledgerStorageClass
-    description: Ledger storage implementation class
-    default: org.apache.bookkeeper.bookie.SortedLedgerStorage
-  - param: ledgerDirectories
-    description: The directory to which Bookkeeper outputs ledger snapshots. You can define multiple directories to store snapshots separated by a comma, for example `/tmp/data-dir1,/tmp/data-dir2`.
-    default: /tmp/bk1-data,/tmp/bk2-data
-  - param: auditorPeriodicCheckInterval
     description: |
-      The time interval, in seconds, at which the auditor will check all ledgers in the cluster. By default this runs once a week.
+      Ledger storage implementation class
 
-      Set this to 0 to disable the periodic check completely. Note that periodic checking will put extra load on the cluster, so it should not be run more frequently than once a day.
-    default: 604800
+      Options:
+        - org.apache.bookkeeper.bookie.InterleavedLedgerStorage
+        - org.apache.bookkeeper.bookie.SortedLedgerStorage
+        - org.apache.bookkeeper.bookie.storage.ldb.DbLedgerStorage
+    default: org.apache.bookkeeper.bookie.SortedLedgerStorage
   - param: sortedLedgerStorageEnabled
-    description: Whether sorted-ledger storage enabled (default true)
-    default: 'true'
-  - param: skipListSizeLimit
-    description: The skip list data size limitation (default 64MB) in EntryMemTable
-    default: 67108864L
-
-- name: Ledger cache settings
-  params:
-  - param: openFileLimit
-    description: |
-      Max number of ledger index files could be opened in bookie server. If number of ledger index files reaches this limitation, bookie server started to swap some ledgers from memory to disk. Too frequent swap will affect performance. You can tune this number to gain performance according your requirements.
-    default: 900
-  - param: pageSize
-    description: |
-      Size of a index page in ledger cache, in bytes. A larger index page can improve performance writing page to disk, which is efficent when you have small number of ledgers and these ledgers have similar number of entries. If you have large number of ledgers and each ledger has fewer entries, smaller index page would improve memory usage.
-    default: 8192
-  - param: pageLimit
     description: |
-      How many index pages provided in ledger cache. If number of index pages reaches this limitation, bookie server starts to swap some ledgers from memory to disk. You can increment this value when you found swap became more frequent. But make sure pageLimit*pageSize should not more than JVM max memory limitation, otherwise you would got OutOfMemoryException. In general, incrementing pageLimit, using smaller index page would gain bettern performance in lager number of ledgers with fewer entries case. If pageLimit is -1, bookie server will use 1/3 of JVM memory to compute the limitation of number of index pages.
-    default: 1
+      @Deprecated in favor of using `ledgerStorageClass`
 
-- name: Ledger manager settings
-  params:
-  - param: ledgerManagerFactoryClass
-    description: The ledger manager factory class, which defines how ledgers are stored, managed, and garbage collected. See the [Ledger Manager](../../getting-started/concepts#ledger-manager) guide for more details.
-    default: flat
-  - param: zkLedgersRootPath
-    description: Root Zookeeper path to store ledger metadata. This parameter is used by zookeeper-based ledger manager as a root znode to store all ledgers.
-    default: /ledgers
+      Whether sorted-ledger storage enabled (default true)
+    default: 'true'
+  - param: ledgerDirectories
+    description: The directory to which Bookkeeper outputs ledger snapshots. You can define multiple directories to store snapshots separated by a comma, for example `/tmp/data-dir1,/tmp/data-dir2`.
+    default: /tmp/bk-data
+  - param: indexDirectories
+    description: The directories in which index files are stored. If not specified, the value of [`ledgerDirectories`](#ledgerDirectories) will be used.
+    default: /tmp/bk-data
+  - param: minUsableSizeForIndexFileCreation
+    description: Minimum safe usable size to be available in index directory for bookie to create index file while replaying journal at the time of bookie start in readonly mode (in bytes)
+    default: 1073741824
+  - param: flushInterval
+    description: When entryLogPerLedgerEnabled is enabled, checkpoint doesn't happens when a new active entrylog is created / previous one is rolled over. Instead SyncThread checkpoints periodically with 'flushInterval' delay (in milliseconds) in between executions. Checkpoint flushes both ledger entryLogs and ledger index pages to disk.  Flushing entrylog and index files will introduce much random disk I/O. If separating journal dir and ledger dirs each on different devices, flushing would not affect performance. But if putting journal dir and ledger dirs on same device, performance degrade significantly on too frequent flushing. You can consider increment flush interval to get better performance, but you need to pay more time on bookie server restart after failure. This config is used only when entryLogPerLedgerEnabled is enabled.
+    default: 10000
+  - param: allowStorageExpansion
+    description: Allow the expansion of bookie storage capacity. Newly added ledger and index directories must be empty.
+    default: 'false'
 
 - name: Entry log settings
   params:
@@ -314,7 +308,7 @@ groups:
     default: 0.2
   - param: minorCompactionInterval
     description: Interval to run minor compaction, in seconds. If it is set to less than zero, the minor compaction is disabled.
-    defauit: 3600
+    default: 3600
   - param: compactionMaxOutstandingRequests
     description: Set the maximum number of entries which can be compacted without flushing. When compacting, the entries are written to the entrylog and the new offsets are cached in memory. Once the entrylog is flushed the index is updated with the new offsets. This parameter controls the number of entries added to the entrylog before a flush is forced. A higher value for this parameter means  more memory will be used for offsets. Each offset consists of 3 longs. This parameter should *not* be modified unless you know what you're doing.
     default: 100000
@@ -333,26 +327,27 @@ groups:
   - param: compactionRateByBytes
     description: Set the rate at which compaction will read entries. The unit is bytes added per second.
     default: 1000000
+  - param: useTransactionalCompaction
+    description: |
+      Flag to enable/disable transactional compaction. If it is set to true, it will use transactional compaction, which uses
+      new entry log files to store entries after compaction; otherwise, it will use normal compaction, which shares same entry
+      log file with normal add operations.
+    default: 'false'
 
-- name: Statistics
-  params:
-  - param: enableStatistics
-    description: Whether statistics are enabled for the bookie.
-    default: true
-  - param: statsProviderClass
-    description: Stats provider class.
-    default: org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider
-
-- name: Read-only mode support
+- name: Garbage collection settings
   params:
-  - param: readOnlyModeEnabled
-    description: If all ledger directories configured are full, then support only read requests for clients. If "readOnlyModeEnabled=true" then on all ledger disks full, bookie will be converted to read-only mode and serve only read requests. Otherwise the bookie will be shutdown. By default this will be disabled.
+  - param: gcWaitTime
+    description: How long the interval to trigger next garbage collection, in milliseconds. Since garbage collection is running in background, too frequent gc will heart performance. It is better to give a higher number of gc interval if there is enough disk capacity.
+    default: 1000
+  - param: gcOverreplicatedLedgerWaitTime
+    description: How long the interval to trigger next garbage collection of overreplicated ledgers, in milliseconds. This should not be run very frequently since we read the metadata for all the ledgers on the bookie from zk.
+    default: 86400000
+  - param: isForceGCAllowWhenNoSpace
+    description: Whether force compaction is allowed when the disk is full or almost full. Forcing GC may get some space back, but may also fill up disk space more quickly. This is because new log files are created before GC, while old garbage log files are deleted after GC.
     default: 'false'
-  - param: forceReadOnlyBookie
-    description: Whether the bookie is force started in read only mode or not.
+  - param: verifyMetadataOnGC
+    description: Whether the bookie should double check if a ledger exists in metadata service prior to gc.
     default: 'false'
-  - param: persistBookieStatusEnabled
-    description: Persist the bookie status locally on the disks. So the bookies can keep their status upon restarts.
 
 - name: Disk utilization
   params:
@@ -360,32 +355,264 @@ groups:
     description: |
       For each ledger dir, maximum disk space which can be used. Default is 0.95f. i.e. 95% of disk can be used at most after which nothing will be written to that partition. If all ledger dir partions are full, then bookie will turn to readonly mode if 'readOnlyModeEnabled=true' is set, else it will shutdown. Valid values should be in between 0 and 1 (exclusive).
     default: 0.95
+  - param: diskUsageWarnThreshold
+    description: The disk free space low water mark threshold. Disk is considered full when usage threshold is exceeded. Disk returns back to non-full state when usage is below low water mark threshold. This prevents it from going back and forth between these states frequently when concurrent writes and compaction are happening. This also prevent bookie from switching frequently between read-only and read-writes states in the same cases.
+    default: 0.95
   - param: diskUsageLwmThreshold
     description: |
       Set the disk free space low water mark threshold. Disk is considered full when usage threshold is exceeded. Disk returns back to non-full state when usage is  below low water mark threshold. This prevents it from going back and forth between these states frequently when concurrent writes and compaction are happening. This also prevent bookie from switching frequently between read-only and read-writes states in the same cases.
     default: 0.90
-  - param: diskUsageWarnThreshold
-    description: The disk free space low water mark threshold. Disk is considered full when usage threshold is exceeded. Disk returns back to non-full state when usage is below low water mark threshold. This prevents it from going back and forth between these states frequently when concurrent writes and compaction are happening. This also prevent bookie from switching frequently between read-only and read-writes states in the same cases.
-    default: 0.95
   - param: diskCheckInterval
     description: Disk check interval in milliseconds. Interval to check the ledger dirs usage.
     default: 10000
 
-- name: ZooKeeper parameters
+- name: Sorted Ledger Storage Settings
+  params:
+  - param: skipListSizeLimit
+    description: The skip list data size limitation (default 64MB) in EntryMemTable
+    default: 67108864
+  - param: skipListArenaChunkSize
+    description: The number of bytes we should use as chunk allocation for org.apache.bookkeeper.bookie.SkipListArena
+    default: 4194304
+  - param: skipListArenaMaxAllocSize
+    description: The max size we should allocate from the skiplist arena. Allocations larger than this should be allocated directly by the VM to avoid fragmentation.
+    default: 131072
+  - param: openFileLimit
+    description: |
+      Max number of ledger index files could be opened in bookie server. If number of ledger index files reaches this limitation, bookie server started to swap some ledgers from memory to disk. Too frequent swap will affect performance. You can tune this number to gain performance according your requirements.
+    default: 20000
+  - param: fileInfoCacheInitialCapacity
+    description: |
+      The minimum total size of the internal file info cache table. Providing a large enough estimate at construction time avoids the need for expensive resizing operations later,
+      but setting this value unnecessarily high wastes memory. The default value is `1/4` of `openFileLimit` if openFileLimit is positive, otherwise it is 64.
+  - param: fileInfoMaxIdleTime
+    description: |
+      The max idle time allowed for an open file info existed in the file info cache. If the file info is idle for a long time, exceed the given time period. The file info will be
+      evicted and closed. If the value is zero or negative, the file info is evicted only when opened files reached `openFileLimit`.
+    default: 0
+  - param: pageSize
+    description: |
+      Size of a index page in ledger cache, in bytes. A larger index page can improve performance writing page to disk, which is efficent when you have small number of ledgers and these ledgers have similar number of entries. If you have large number of ledgers and each ledger has fewer entries, smaller index page would improve memory usage.
+    default: 8192
+  - param: pageLimit
+    description: |
+      How many index pages provided in ledger cache. If number of index pages reaches this limitation, bookie server starts to swap some ledgers from memory to disk. You can increment this value when you found swap became more frequent. But make sure pageLimit*pageSize should not more than JVM max memory limitation, otherwise you would got OutOfMemoryException. In general, incrementing pageLimit, using smaller index page would gain bettern performance in lager number of ledgers with fewer entries case. If pageLimit is -1, bookie server will use 1/3 of JVM memory to compute the limitation of number of index pages.
+    default: "-1"
+
+- name: DB Ledger Storage Settings
+  params:
+  - param: dbStorage_writeCacheMaxSizeMb
+    description: Size of write cache. Memory is allocated from JVM direct memory. Write cache is used for buffer entries before flushing into the entry log. For good performance, it should be big enough to hold a sub.
+    default: 512
+  - param: dbStorage_readAheadCacheMaxSizeMb
+    description: Size of read cache. Memory is allocated from JVM direct memory. The read cache is pre-filled doing read-ahead whenever a cache miss happens.
+    default: 256
+  - param: dbStorage_readAheadCacheBatchSize
+    description: How many entries to pre-fill in cache after a read cache miss
+    default: 1000
+  - param: dbStorage_rocksDB_blockSize
+    description: |
+      Size of RocksDB block-cache. RocksDB is used for storing ledger indexes.
+      For best performance, this cache should be big enough to hold a significant portion of the index database which can reach ~2GB in some cases.
+    default: 268435456
+  - param: dbStorage_rocksDB_writeBufferSizeMB
+    description: |
+      Size of RocksDB write buffer. RocksDB is used for storing ledger indexes.
+    default: 64
+  - param: dbStorage_rocksDB_sstSizeInMB
+    description: |
+      Size of RocksDB sst file size in MB. RocksDB is used for storing ledger indexes.
+    default: 64
+  - param: dbStorage_rocksDB_blockSize
+    default: 65536
+  - param: dbStorage_rocksDB_bloomFilterBitsPerKey
+    default: 10
+  - param: dbStorage_rocksDB_numLevels
+    default: "-1"
+  - param: dbStorage_rocksDB_numFilesInLevel0
+    default: 10
+  - param: dbStorage_rocksDB_maxSizeInLevel1MB
+    default: 256
+
+- name: Metadata Service Settings
   params:
+  - param: metadataServiceUri
+    description: metadata service uri that bookkeeper is used for loading corresponding metadata driver and resolving its metadata service location.
+    default: "zk+hierarchical://localhost:2181/ledgers"
+  - param: ledgerManagerFactoryClass
+    description: |
+      @Deprecated in favor of using `metadataServiceUri`
+
+      The ledger manager factory class, which defines how ledgers are stored, managed, and garbage collected. See the [Ledger Manager](../../getting-started/concepts#ledger-manager) guide for more details.
+    default: hierarchical
+  - param: allowShadedLedgerManagerFactoryClass
+    description: |
+      Sometimes the bookkeeper server classes are shaded. The ledger manager factory classes might be relocated to be under other packages.
+      This would fail the clients using shaded factory classes since the factory classes are stored in cookies and used for verification.
+      Users can enable this flag to allow using shaded ledger manager factory classes to connect to a bookkeeper cluster.
+    default: 'false'
+  - param: shadedLedgerManagerFactoryClassPrefix
+    description: The shaded ledger manager factory prefix. This is used when `allowShadedLedgerManagerFactoryClass` is set to true.
+    default: 'dlshade.'
+
+- name: ZooKeeper Metadata Service Settings
+  params:
+  - param: zkLedgersRootPath
+    description: |
+      @Deprecated in favor of using `metadataServiceUri`
+
+      Root Zookeeper path to store ledger metadata. This parameter is used by zookeeper-based ledger manager as a root znode to store all ledgers.
+    default: /ledgers
   - param: zkServers
     description: |
+      @Deprecated in favor of using `metadataServiceUri`
+
       A list of one of more servers on which Zookeeper is running. The server list can be comma separated values, for example `zkServers=zk1:2181,zk2:2181,zk3:2181`.
     default: "localhost:2181"
   - param: zkTimeout
     description: ZooKeeper client session timeout in milliseconds. Bookie server will exit if it received SESSION_EXPIRED because it was partitioned off from ZooKeeper for more than the session timeout  JVM garbage collection, disk I/O will cause SESSION_EXPIRED. Increment this value could help avoiding this issue.
-    default: 10
+    default: 10000
   - param: zkRetryBackoffStartMs
     description: The Zookeeper client backoff retry start time in millis.
     default: 1000
   - param: zkRetryBackoffMaxMs
     description: The Zookeeper client backoff retry max time in millis.
     default: 10000
+  - param: zkRequestRateLimit
+    description: The Zookeeper request limit. It is only enabled when setting a postivie value.
+    default: 0
   - param: zkEnableSecurity
     description: Set ACLs on every node written on ZooKeeper, this way only allowed users will be able to read and write BookKeeper metadata stored on ZooKeeper. In order to make ACLs work you need to setup ZooKeeper JAAS authentication all the bookies and Client need to share the same user, and this is usually done using Kerberos authentication. See ZooKeeper documentation
     default: 'false'
+
+- name: Statistics
+  params:
+  - param: enableStatistics
+    description: Whether statistics are enabled for the bookie.
+    default: true
+  - param: statsProviderClass
+    description: |
+      Stats provider class.
+      Options:
+        - Prometheus        : org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
+        - Codahale          : org.apache.bookkeeper.stats.codahale.CodahaleMetricsProvider
+        - Twitter Finagle   : org.apache.bookkeeper.stats.twitter.finagle.FinagleStatsProvider
+        - Twitter Ostrich   : org.apache.bookkeeper.stats.twitter.ostrich.OstrichProvider
+        - Twitter Science   : org.apache.bookkeeper.stats.twitter.science.TwitterStatsProvider
+    default: org.apache.bookkeeper.stats.prometheus.PrometheusMetricsProvider
+
+- name: Prometheus Metrics Provider Settings
+  params:
+  - param: prometheusStatsHttpPort
+    description: default port for prometheus metrics exporter
+    default: 8000
+  - param: prometheusStatsLatencyRolloverSeconds
+    description: latency stats rollover interval, in seconds
+    default: 60
+
+- name: Codahale Metrics Provider Settings
+  params:
+  - param: codahaleStatsPrefix
+    description: metric name prefix, default is empty.
+    default: ""
+  - param: codahaleStatsOutputFrequencySeconds
+    description: the frequency that stats reporters report stats, in seconds.
+    default: 60
+  - param: codahaleStatsGraphiteEndpoint
+    description: the graphite endpoint for reporting stats. see [graphite reporter](//metrics.dropwizard.io/3.1.0/manual/graphite/) for more details.
+    default: "null"
+  - param: codahaleStatsCSVEndpoint
+    description: the directory for reporting stats in csv format. see [csv reporter](//metrics.dropwizard.io/3.1.0/manual/core/#csv) for more details.
+    default: "null"
+  - param: codahaleStatsSlf4jEndpoint
+    description: the slf4j endpoint for reporting stats. see [slf4j reporter](//metrics.dropwizard.io/3.1.0/manual/core/#slf4j) for more details.
+    default: "null"
+  - param: codahaleStatsJmxEndpoint
+    description: the jmx endpoint for reporting stats. see [jmx reporter](//metrics.dropwizard.io/3.1.0/manual/core/#jmx) for more details.
+
+- name: Twitter Ostrich Metrics Provider
+  params:
+  - param: statsExport
+    description: Flag to control whether to expose ostrich metrics via a http endpoint configured by `statsHttpPort`.
+    default: "false"
+  - param: statsHttpPort
+    description: The http port of exposing ostrich stats if `statsExport` is set to true
+    default: 9002
+
+- name: Twitter Science Metrics Provider
+  params:
+  - param: statsExport
+    description: Flag to control whether to expose metrics via a http endpoint configured by `statsHttpPort`.
+    default: "false"
+  - param: statsHttpPort
+    description: The http port of exposing stats if `statsExport` is set to true
+    default: 9002
+
+- name: AutoRecovery general settings
+  params:
+  - param: autoRecoveryDaemonEnabled
+    description: Whether the bookie itself can start auto-recovery service also or not.
+    default: false
+  - param: digestType
+    description: The default digest type used for opening ledgers.
+    default: "CRC32"
+  - param: passwd
+    description: The default password used for opening ledgers. Default value is empty string.
+    default: ""
+  - param: enableDigestTypeAutodetection
+    description: The flag to enable/disable digest type auto-detection. If it is enabled, the bookkeeper client will ignore the provided digest type provided at `digestType` and the provided passwd provided at `passwd`.
+    default: true
+- name: AutoRecovery placement settings
+  params:
+  - param: ensemblePlacementPolicy
+    description: |
+      The ensemble placement policy used for finding bookie for re-replicating entries.
+    
+      Options:
+        - org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy
+        - org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy
+    default: "org.apache.bookkeeper.client.RackawareEnsemblePlacementPolicy"
+  - param: reppDnsResolverClass
+    description: |
+      The DNS resolver class used for resolving network locations for bookies. The setting is used
+      when using either RackawareEnsemblePlacementPolicy and RegionAwareEnsemblePlacementPolicy.
+    default: "org.apache.bookkeeper.net.ScriptBasedMapping"
+  - param: networkTopologyScriptFileName
+    description: |
+      The bash script used by `ScriptBasedMapping` DNS resolver for resolving bookies' network locations.
+  - param: networkTopologyScriptNumberArgs
+    description: |
+      The max number of args used in the script provided at `networkTopologyScriptFileName`.
+
+- name: AutoRecovery auditor settings
+  params:
+  - param: auditorPeriodicBookieCheckInterval
+    description: The time interval between auditor bookie checks, in seconds. The auditor bookie check checks ledger metadata to see which bookies should contain entries for each ledger. If a bookie that should contain entries is unavailable, then the ledger containing that entry is marked for recovery. Setting this to 0 disables the periodic check. Bookie checks will still run when a bookie fails. The default is once per day.
+    default: 86400
+  - param: auditorPeriodicCheckInterval
+    description: |
+      The time interval, in seconds, at which the auditor will check all ledgers in the cluster. By default this runs once a week.
+
+      Set this to 0 to disable the periodic check completely. Note that periodic checking will put extra load on the cluster, so it should not be run more frequently than once a day.
+    default: 604800
+  - param: auditorLedgerVerificationPercentage
+    description: |
+      The percentage of a ledger (fragment)'s entries will be verified before claiming a fragment as missing. If it is 0, it only verifies the first and last entries of a given fragment.
+    default: 0
+  - param: lostBookieRecoveryDelay
+    description: How long to wait, in seconds, before starting autorecovery of a lost bookie.
+    default: 0
+
+- name: AutoRecovery replication worker settings
+  params:
+  - param: rereplicationEntryBatchSize
+    description: The number of entries that a replication will rereplicate in parallel.
+    default: 10
+  - param: openLedgerRereplicationGracePeriod
+    description: The grace period, in seconds, that the replication worker waits before fencing and replicating a ledger fragment that's still being written to upon bookie failure.
+    default: 30
+  - param: rwRereplicateBackoffMs
+    description: The time to backoff when replication worker encounters exceptions on replicating a ledger, in milliseconds.
+    default: 5000
+
+
diff --git a/tests/docker-images/all-released-versions-image/scripts/update-conf-and-boot.sh b/tests/docker-images/all-released-versions-image/scripts/update-conf-and-boot.sh
index 6e4fbca51..ba082d616 100755
--- a/tests/docker-images/all-released-versions-image/scripts/update-conf-and-boot.sh
+++ b/tests/docker-images/all-released-versions-image/scripts/update-conf-and-boot.sh
@@ -24,6 +24,11 @@ sed -i "s|journalDirectory=.*|journalDirectory=$BK_JOURNALDIR|" /opt/bookkeeper/
 sed -i "s|ledgerDirectories=.*|ledgerDirectories=$BK_LEDGERDIR|" /opt/bookkeeper/*/conf/{bk_server,bookkeeper}.conf
 sed -i "s|zkServers=.*|zkServers=$BK_ZKCONNECTSTRING|" /opt/bookkeeper/*/conf/{bk_server,bookkeeper}.conf
 
+# 4.7.0 prefers metadataServiceUri instead of `zkServers`
+sed -i "s|metadataServiceUri=.*|metadataServiceUri=zk+hierarchical://$BK_ZKCONNECTSTRING/ledgers|" /opt/bookkeeper/*/conf/{bk_server,bookkeeper}.conf
+# 4.7.0 includes journalDirectories instead of `journalDirectory`
+sed -i "s|journalDirectories=.*|journalDirectories=$BK_JOURNALDIR|" /opt/bookkeeper/*/conf/{bk_server,bookkeeper}.conf
+
 # 4.3.1 & 4.3.2 shipped with a broken confs
 sed -i "s|\(# \)\?logSizeLimit=.*|logSizeLimit=1073741824|" /opt/bookkeeper/4.3.1/conf/bk_server.conf
 sed -i "s|\(# \)\?logSizeLimit=.*|logSizeLimit=1073741824|" /opt/bookkeeper/4.3.2/conf/bk_server.conf


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services