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.