ozone.container.cache.size 1024 PERFORMANCE, CONTAINER, STORAGE The open container is cached on the data node side. We maintain an LRU cache for caching the recently used containers. This setting controls the size of that cache. dfs.container.ipc 9859 OZONE, CONTAINER, MANAGEMENT The ipc port number of container. dfs.container.ipc.random.port false OZONE, DEBUG, CONTAINER Allocates a random free port for ozone container. This is used only while running unit tests. dfs.container.ratis.datanode.storage.dir OZONE, CONTAINER, STORAGE, MANAGEMENT, RATIS This directory is used for storing Ratis metadata like logs. If this is not set then default metadata dirs is used. A warning will be logged if this not set. Ideally, this should be mapped to a fast disk like an SSD. dfs.container.ratis.enabled false OZONE, MANAGEMENT, PIPELINE, RATIS Ozone supports different kinds of replication pipelines. Ratis is one of the replication pipeline supported by ozone. dfs.container.grpc.enabled false OZONE, MANAGEMENT, PIPELINE, RATIS Ozone supports different kinds of replication pipelines protocols. grpc is one of the replication pipeline protocol supported by ozone. dfs.container.ratis.ipc 9858 OZONE, CONTAINER, PIPELINE, RATIS, MANAGEMENT The ipc port number of container. dfs.container.ratis.ipc.random.port false OZONE,DEBUG Allocates a random free port for ozone ratis port for the container. This is used only while running unit tests. dfs.container.ratis.rpc.type GRPC OZONE, RATIS, MANAGEMENT Ratis supports different kinds of transports like netty, GRPC, Hadoop RPC etc. This picks one of those for this cluster. dfs.container.ratis.num.write.chunk.threads 60 OZONE, RATIS, PERFORMANCE Maximum number of threads in the thread pool that Ratis will use for writing chunks (60 by default). dfs.container.ratis.segment.size 1073741824 OZONE, RATIS, PERFORMANCE The size of the raft segment used by Apache Ratis on datanodes. (1 GB by default) dfs.container.ratis.segment.preallocated.size 134217728 OZONE, RATIS, PERFORMANCE The size of the buffer which is preallocated for raft segment used by Apache Ratis on datanodes.(128 MB by default) dfs.ratis.client.request.timeout.duration 3s OZONE, RATIS, MANAGEMENT The timeout duration for ratis client request. dfs.ratis.server.request.timeout.duration 3s OZONE, RATIS, MANAGEMENT The timeout duration for ratis server request. ozone.container.report.interval 60000ms OZONE, CONTAINER, MANAGEMENT Time interval of the datanode to send container report. Each datanode periodically send container report upon receive sendContainerReport from SCM. Unit could be defined with postfix (ns,ms,s,m,h,d) ozone.administrators OZONE, SECURITY Ozone administrator users delimited by the comma. If not set, only the user who launches an ozone service will be the admin user. This property must be set if ozone services are started by different users. Otherwise, the RPC layer will reject calls from other servers which are started by users not in the list. ozone.block.deleting.container.limit.per.interval 10 OZONE, PERFORMANCE, SCM A maximum number of containers to be scanned by block deleting service per time interval. The block deleting service spawns a thread to handle block deletions in a container. This property is used to throttle the number of threads spawned for block deletions. ozone.block.deleting.limit.per.task 1000 OZONE, PERFORMANCE, SCM A maximum number of blocks to be deleted by block deleting service per time interval. This property is used to throttle the actual number of block deletions on a data node per container. ozone.block.deleting.service.interval 1m OZONE, PERFORMANCE, SCM Time interval of the block deleting service. The block deleting service runs on each datanode periodically and deletes blocks queued for deletion. Unit could be defined with postfix (ns,ms,s,m,h,d) ozone.block.deleting.service.timeout 300000ms OZONE, PERFORMANCE, SCM A timeout value of block deletion service. If this is set greater than 0, the service will stop waiting for the block deleting completion after this time. If timeout happens to a large proportion of block deletion, this needs to be increased with ozone.block.deleting.limit.per.task. This setting supports multiple time unit suffixes as described in dfs.heartbeat.interval. If no suffix is specified, then milliseconds is assumed. ozone.client.connection.timeout 5000ms OZONE, PERFORMANCE, CLIENT Connection timeout for Ozone client in milliseconds. ozone.client.protocol org.apache.hadoop.ozone.client.rpc.RpcClient OZONE, CLIENT, MANAGEMENT Protocol class to be used by the client to connect to ozone cluster. The build-in implementation includes: org.apache.hadoop.ozone.client.rpc.RpcClient for RPC org.apache.hadoop.ozone.client.rest.RestClient for REST The default is the RpClient. Please do not change this unless you have a very good understanding of what you are doing. ozone.client.socket.timeout 5000ms OZONE, CLIENT Socket timeout for Ozone client. Unit could be defined with postfix (ns,ms,s,m,h,d) ozone.enabled false OZONE, REQUIRED Status of the Ozone Object Storage service is enabled. Set to true to enable Ozone. Set to false to disable Ozone. Unless this value is set to true, Ozone services will not be started in the cluster. Please note: By default ozone is disabled on a hadoop cluster. ozone.handler.type distributed OZONE, REST Tells ozone which storage handler to use. The possible values are: distributed - The Ozone distributed storage handler, which speaks to OM/SCM on the backend and provides REST services to clients. local - Local Storage handler strictly for testing - To be removed. ozone.key.deleting.limit.per.task 1000 OM, PERFORMANCE A maximum number of keys to be scanned by key deleting service per time interval in OM. Those keys are sent to delete metadata and generate transactions in SCM for next async deletion between SCM and DataNode. ozone.om.address OM, REQUIRED The address of the Ozone OM service. This allows clients to discover the address of the OM. ozone.om.group.rights READ_WRITE OM, SECURITY Default group permissions in Ozone OM. ozone.om.handler.count.key 20 OM, PERFORMANCE The number of RPC handler threads for OM service endpoints. ozone.om.http-address 0.0.0.0:9874 OM, MANAGEMENT The address and the base port where the OM web UI will listen on. If the port is 0, then the server will start on a free port. However, it is best to specify a well-known port, so it is easy to connect and see the OM management UI. ozone.om.http-bind-host 0.0.0.0 OM, MANAGEMENT The actual address the OM web server will bind to. If this optional the address is set, it overrides only the hostname portion of ozone.om.http-address. ozone.om.http.enabled true OM, MANAGEMENT Property to enable or disable OM web user interface. ozone.om.https-address 0.0.0.0:9875 OM, MANAGEMENT, SECURITY The address and the base port where the OM web UI will listen on using HTTPS. If the port is 0 then the server will start on a free port. ozone.om.https-bind-host 0.0.0.0 OM, MANAGEMENT, SECURITY The actual address the OM web server will bind to using HTTPS. If this optional address is set, it overrides only the hostname portion of ozone.om.http-address. ozone.om.keytab.file OM, SECURITY The keytab file for Kerberos authentication in OM. ozone.om.db.cache.size.mb 128 OM, PERFORMANCE The size of OM DB cache in MB that used for caching files. This value is set to an abnormally low value in the default configuration. That is to make unit testing easy. Generally, this value should be set to something like 16GB or more, if you intend to use Ozone at scale. A large value for this key allows a proportionally larger amount of OM metadata to be cached in memory. This makes OM operations faster. ozone.om.user.max.volume 1024 OM, MANAGEMENT The maximum number of volumes a user can have on a cluster.Increasing or decreasing this number has no real impact on ozone cluster. This is defined only for operational purposes. Only an administrator can create a volume, once a volume is created there are no restrictions on the number of buckets or keys inside each bucket a user can create. ozone.om.user.rights READ_WRITE OM, SECURITY Default user permissions used in OM. ozone.localstorage.root ${hadoop.tmp.dir}/ozone OZONE, DEBUG This is used only for testing purposes. This value is used by the local storage handler to simulate a REST backend. This is useful only when debugging the REST front end independent of OM and SCM. To be removed. ozone.metadata.dirs OZONE, OM, SCM, CONTAINER, REQUIRED, STORAGE Ozone metadata is shared among OM, which acts as the namespace manager for ozone, SCM which acts as the block manager and data nodes which maintain the name of the key(Key Name and BlockIDs). This replicated and distributed metadata store is maintained under the directory pointed by this key. Since metadata can be I/O intensive, at least on OM and SCM we recommend having SSDs. If you have the luxury of mapping this path to SSDs on all machines in the cluster, that will be excellent. If Ratis metadata directories are not specified, Ratis server will emit a warning and use this path for storing its metadata too. ozone.metastore.impl RocksDB OZONE, OM, SCM, CONTAINER, STORAGE Ozone metadata store implementation. Ozone metadata are well distributed to multiple services such as ozoneManager, scm. They are stored in some local key-value databases. This property determines which database library to use. Supported value is either LevelDB or RocksDB. ozone.metastore.rocksdb.statistics ALL OZONE, OM, SCM, STORAGE, PERFORMANCE The statistics level of the rocksdb store. If you use any value from org.rocksdb.StatsLevel (eg. ALL or EXCEPT_DETAILED_TIMERS), the rocksdb statistics will be exposed over JMX bean with the choosed setting. Set it to OFF to not initialize rocksdb statistics at all. Please note that collection of statistics could have 5-10% performance penalty. Check the rocksdb documentation for more details. ozone.scm.block.client.address OZONE, SCM The address of the Ozone SCM block client service. If not defined value of ozone.scm.client.address is used. ozone.scm.block.client.bind.host 0.0.0.0 OZONE, SCM The hostname or IP address used by the SCM block client endpoint to bind. ozone.scm.block.client.port 9863 OZONE, SCM The port number of the Ozone SCM block client service. ozone.scm.block.deletion.max.retry 4096 OZONE, SCM SCM wraps up many blocks in a deletion transaction and sends that to data node for physical deletion periodically. This property determines how many times SCM is going to retry sending a deletion operation to the data node. ozone.scm.block.size.in.mb 256 OZONE, SCM The default size of a scm block in bytes. This is maps to the default Ozone block size. ozone.scm.chunk.size 16777216 OZONE, SCM, CONTAINER, PERFORMANCE The chunk size for reading/writing chunk operations in bytes. The chunk size defaults to 8MB. If the value configured is more than the maximum size (16MB), it will be reset to the maximum size. This maps to the network packet sizes and file write operations in the client to datanode protocol. ozone.scm.client.address OZONE, SCM, REQUIRED The address of the Ozone SCM client service. This is a required setting. It is a string in the host:port format. The port number is optional and defaults to 9860. ozone.scm.client.bind.host 0.0.0.0 OZONE, SCM, MANAGEMENT The hostname or IP address used by the SCM client endpoint to bind. This setting is used by the SCM only and never used by clients. The setting can be useful in multi-homed setups to restrict the availability of the SCM client service to a specific interface. The default is appropriate for most clusters. ozone.scm.client.port 9860 OZONE, SCM, MANAGEMENT The port number of the Ozone SCM client service. ozone.scm.container.deletion-choosing.policy org.apache.hadoop.ozone.container.common.impl.TopNOrderedContainerDeletionChoosingPolicy OZONE, MANAGEMENT The policy used for choosing desire containers for block deletion. Datanode selects some containers to process block deletion in a certain interval defined by ozone.block.deleting.service.interval. The number of containers to process in each interval is defined by ozone.block.deleting.container.limit.per.interval. This property is used to configure the policy applied while selecting containers. There are two policies supporting now: RandomContainerDeletionChoosingPolicy and TopNOrderedContainerDeletionChoosingPolicy. org.apache.hadoop.ozone.container.common.impl.RandomContainerDeletionChoosingPolicy implements a simply random policy that to return a random list of containers. org.apache.hadoop.ozone.container.common.impl.TopNOrderedContainerDeletionChoosingPolicy implements a policy that choosing top count number of containers in a pending-deletion-blocks's num based descending order. ozone.scm.container.placement.impl org.apache.hadoop.hdds.scm.container.placement.algorithms.SCMContainerPlacementRandom OZONE, MANAGEMENT Placement policy class for containers. Defaults to SCMContainerPlacementRandom.class ozone.scm.container.provision_batch_size 20 OZONE, PERFORMANCE Pre-provision specified number of containers for block allocation. ozone.scm.container.size.gb 5 OZONE, PERFORMANCE, MANAGEMENT Default container size used by Ozone. This value is specified in GB. There are two considerations while picking this number. The speed at which a container can be replicated, determined by the network speed and the metadata that each container generates. So selecting a large number creates less SCM metadata, but recovery time will be more. 5GB is a number that maps to quick replication times in gigabit networks, but still balances the amount of metadata. ozone.scm.datanode.address OZONE, MANAGEMENT The address of the Ozone SCM service used for internal communication between the DataNodes and the SCM. It is a string in the host:port format. The port number is optional and defaults to 9861. This setting is optional. If unspecified then the hostname portion is picked from the ozone.scm.client.address setting and the default service port of 9861 is chosen. ozone.scm.datanode.bind.host OZONE, MANAGEMENT The hostname or IP address used by the SCM service endpoint to bind. ozone.scm.datanode.id OZONE, MANAGEMENT The path that datanodes will use to store the datanode ID. If this value is not set, then datanode ID is created under the metadata directory. ozone.scm.datanode.port 9861 OZONE, MANAGEMENT The port number of the Ozone SCM service. ozone.scm.db.cache.size.mb 128 OZONE, PERFORMANCE SCM keeps track of the Containers in the cluster. This DB holds the container metadata. This value is set to a small value to make the unit testing runs smooth. In production, we recommend a value of 16GB or higher. This allows SCM to avoid disk I/O's while looking up the container location. ozone.scm.dead.node.interval 10m OZONE, MANAGEMENT The interval between heartbeats before a node is tagged as dead. ozone.scm.handler.count.key 10 OZONE, MANAGEMENT, PERFORMANCE The number of RPC handler threads for each SCM service endpoint. The default is appropriate for small clusters (tens of nodes). Set a value that is appropriate for the cluster size. Generally, HDFS recommends RPC handler count is set to 20 * log2(Cluster Size) with an upper limit of 200. However, SCM will not have the same amount of traffic as Namenode, so a value much smaller than that will work well too. ozone.scm.heartbeat.interval 30s OZONE, MANAGEMENT The heartbeat interval from a data node to SCM. Yes, it is not three but 30, since most data nodes will heart beating via Ratis heartbeats. If a client is not able to talk to a data node, it will notify OM/SCM eventually. So a 30 second HB seems to work. This assumes that replication strategy used is Ratis if not, this value should be set to something smaller like 3 seconds. ozone.scm.heartbeat.log.warn.interval.count 10 OZONE, MANAGEMENT Defines how frequently we will log the missing of a heartbeat to SCM. For example in the default case, we will write a warning message for each ten consecutive heartbeats that we miss to SCM. This helps in reducing clutter in a data node log, but trade off is that logs will have less of this statement. ozone.scm.heartbeat.rpc-timeout 1000 OZONE, MANAGEMENT Timeout value for the RPC from Datanode to SCM in milliseconds. ozone.scm.heartbeat.thread.interval 3s OZONE, MANAGEMENT When a heartbeat from the data node arrives on SCM, It is queued for processing with the time stamp of when the heartbeat arrived. There is a heartbeat processing thread inside SCM that runs at a specified interval. This value controls how frequently this thread is run. There are some assumptions build into SCM such as this value should allow the heartbeat processing thread to run at least three times more frequently than heartbeats and at least five times more than stale node detection time. If you specify a wrong value, SCM will gracefully refuse to run. For more info look at the node manager tests in SCM. In short, you don't need to change this. ozone.scm.http-address 0.0.0.0:9876 OZONE, MANAGEMENT The address and the base port where the SCM web ui will listen on. If the port is 0 then the server will start on a free port. ozone.scm.http-bind-host 0.0.0.0 OZONE, MANAGEMENT The actual address the SCM web server will bind to. If this optional address is set, it overrides only the hostname portion of ozone.scm.http-address. ozone.scm.http.enabled true OZONE, MANAGEMENT Property to enable or disable SCM web ui. ozone.scm.https-address 0.0.0.0:9877 OZONE, MANAGEMENT The address and the base port where the SCM web UI will listen on using HTTPS. If the port is 0 then the server will start on a free port. ozone.scm.https-bind-host 0.0.0.0 OZONE, MANAGEMENT The actual address the SCM web server will bind to using HTTPS. If this optional address is set, it overrides only the hostname portion of ozone.scm.http-address. ozone.scm.keytab.file OZONE, SECURITY The keytab file for Kerberos authentication in SCM. ozone.scm.names OZONE The value of this property is a set of DNS | DNS:PORT | IP Address | IP:PORT. Written as a comma separated string. e.g. scm1, scm2:8020, 7.7.7.7:7777. This property allows datanodes to discover where SCM is, so that datanodes can send heartbeat to SCM. ozone.scm.stale.node.interval 90s OZONE, MANAGEMENT The interval for stale node flagging. Please see ozone.scm.heartbeat.thread.interval before changing this value. ozone.trace.enabled false OZONE, DEBUG Setting this flag to true dumps the HTTP request/ response in the logs. Very useful when debugging REST protocol. ozone.web.authentication.kerberos.principal OZONE, SECURITY The server principal used by the SCM and OM for web UI SPNEGO authentication when Kerberos security is enabled. This is typically set to HTTP/_HOST@REALM.TLD The SPNEGO server principal begins with the prefix HTTP/ by convention. If the value is '*', the web server will attempt to login with every principal specified in the keytab file. scm.container.client.idle.threshold 10s OZONE, PERFORMANCE In the standalone pipelines, the SCM clients use netty to communicate with the container. It also uses connection pooling to reduce client side overheads. This allows a connection to stay idle for a while before the connection is closed. scm.container.client.max.size 256 OZONE, PERFORMANCE Controls the maximum number of connections that we cached via clientconnection pooling. If the number of connection exceed this count then the oldest idle connection is evicted. scm.container.client.max.outstanding.requests 100 OZONE, PERFORMANCE Controls the maximum number of outstanding async requests that can be handled by the Standalone as well as Ratis client. ozone.scm.container.creation.lease.timeout 60s OZONE, SCM Container creation timeout in milliseconds to be used by SCM. When BEGIN_CREATE event happens the container is moved from ALLOCATED to CREATING state, SCM will now wait for the configured amount of time to get COMPLETE_CREATE event if it doesn't receive it will move the container to DELETING. ozone.key.preallocation.maxsize 134217728 OZONE, OM, PERFORMANCE When a new key write request is sent to OM, if a size is requested, at most 128MB of size is allocated at request time. If client needs more space for the write, separate block allocation requests will be made. ozone.client.list.cache 1000 OZONE, PERFORMANCE Configuration property to configure the cache size of client list calls. ozone.replication 3 OZONE, CLIENT Default replication value. The actual number of replications can be specified when writing the key. The default is used if replication is not specified. Supported values: 1 and 3. ozone.replication.type RATIS OZONE, CLIENT Default replication type to be used while writing key into ozone. The value can be specified when writing the key, default is used when nothing is specified. Supported values: RATIS, STAND_ALONE and CHAINED. ozone.scm.container.close.threshold 0.9f OZONE, SCM This determines the threshold to be used for closing a container. When the container used percentage reaches this threshold, the container will be closed. Value should be a positive, non-zero percentage in float notation (X.Yf), with 1.0f meaning 100%. ozone.rest.client.http.connection.max 100 OZONE, CLIENT This defines the overall connection limit for the connection pool used in RestClient. ozone.rest.client.http.connection.per-route.max 20 OZONE, CLIENT This defines the connection limit per one HTTP route/host. Total max connection is limited by ozone.rest.client.http.connection.max property. ozone.open.key.cleanup.service.interval.seconds 86400 OZONE, OM, PERFORMANCE A background job periodically checks open key entries and delete the expired ones. This entry controls the interval of this cleanup check. ozone.open.key.expire.threshold 86400 OZONE, OM, PERFORMANCE Controls how long an open key operation is considered active. Specifically, if a key has been open longer than the value of this config entry, that open key is considered as expired (e.g. due to client crash). Default to 24 hours. hadoop.tags.custom OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,OM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE ozone.tags.system OZONE,MANAGEMENT,SECURITY,PERFORMANCE,DEBUG,CLIENT,SERVER,OM,SCM,CRITICAL,RATIS,CONTAINER,REQUIRED,REST,STORAGE,PIPELINE,STANDALONE hdds.rest.rest-csrf.enabled false If true, then enables Object Store REST server protection against cross-site request forgery (CSRF). hdds.rest.http-address 0.0.0.0:9880 The http address of Object Store REST server inside the datanode. hdds.rest.netty.high.watermark 65535 High watermark configuration to Netty for Object Store REST server. hdds.rest.netty.low.watermark 32768 Low watermark configuration to Netty for Object Store REST server. hdds.datanode.plugins org.apache.hadoop.ozone.web.OzoneHddsDatanodeService Comma-separated list of HDDS datanode plug-ins to be activated when HDDS service starts as part of datanode. hdds.datanode.storage.utilization.warning.threshold 0.75 OZONE, SCM, MANAGEMENT If a datanode overall storage utilization exceeds more than this value, a warning will be logged while processing the nodeReport in SCM. hdds.datanode.storage.utilization.critical.threshold 0.95 OZONE, SCM, MANAGEMENT If a datanode overall storage utilization exceeds more than this value, the datanode will be marked out of space.