diff --git a/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md b/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
index f8bc95b9a54..62e17919ad3 100644
--- a/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
+++ b/hadoop-common-project/hadoop-common/src/site/markdown/DeprecatedProperties.md
@@ -51,6 +51,7 @@ The following table lists the configuration property names that are deprecated i
| dfs.secondary.http.address | dfs.namenode.secondary.http-address |
| dfs.socket.timeout | dfs.client.socket-timeout |
| dfs.umaskmode | fs.permissions.umask-mode |
+| dfs.web.ugi | hadoop.http.staticuser.user |
| dfs.write.packet.size | dfs.client-write-packet-size |
| fs.checkpoint.dir | dfs.namenode.checkpoint.dir |
| fs.checkpoint.edits.dir | dfs.namenode.checkpoint.edits.dir |
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index 35dce0e2825..897ec63db3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -264,7 +264,7 @@
dfs.datanode.dns.interface.
-
+
dfs.datanode.dns.nameserver
default
@@ -276,7 +276,7 @@
dfs.datanode.dns.nameserver.
-
+
dfs.namenode.backup.address
0.0.0.0:50100
@@ -285,7 +285,7 @@
If the port is 0 then the server will start on a free port.
-
+
dfs.namenode.backup.http-address
0.0.0.0:50105
@@ -1441,6 +1441,13 @@
The prefix for a given nameservice, contains a comma-separated
list of namenodes for a given nameservice (eg EXAMPLENAMESERVICE).
+
+ Unique identifiers for each NameNode in the nameservice, delimited by
+ commas. This will be used by DataNodes to determine all the NameNodes
+ in the cluster. For example, if you used “mycluster” as the nameservice
+ ID previously, and you wanted to use “nn1” and “nn2” as the individual
+ IDs of the NameNodes, you would configure a property
+ dfs.ha.namenodes.mycluster, and its value "nn1,nn2".
@@ -3036,4 +3043,1024 @@
refreshes the configuration files used by the class.
+
+
+ datanode.https.port
+ 50475
+
+ HTTPS port for DataNode.
+
+
+
+
+ dfs.balancer.dispatcherThreads
+ 200
+
+ Size of the thread pool for the HDFS balancer block mover.
+ dispatchExecutor
+
+
+
+
+ dfs.balancer.movedWinWidth
+ 5400000
+
+ Window of time in ms for the HDFS balancer tracking blocks and its
+ locations.
+
+
+
+
+ dfs.balancer.moverThreads
+ 1000
+
+ Thread pool size for executing block moves.
+ moverThreadAllocator
+
+
+
+
+ dfs.balancer.max-size-to-move
+ 10737418240
+
+ Maximum number of bytes that can be moved by the balancer in a single
+ thread.
+
+
+
+
+ dfs.balancer.getBlocks.min-block-size
+ 10485760
+
+ Minimum block threshold size in bytes to ignore when fetching a source's
+ block list.
+
+
+
+
+ dfs.balancer.getBlocks.size
+ 2147483648
+
+ Total size in bytes of Datanode blocks to get when fetching a source's
+ block list.
+
+
+
+
+ dfs.block.invalidate.limit
+ 1000
+
+ Limit on the list of invalidated block list kept by the Namenode.
+
+
+
+
+ dfs.block.misreplication.processing.limit
+ 10000
+
+ Maximum number of blocks to process for initializing replication queues.
+
+
+
+
+ dfs.block.placement.ec.classname
+ org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant
+
+ Placement policy class for striped files.
+ Defaults to BlockPlacementPolicyRackFaultTolerant.class
+
+
+
+
+ dfs.block.replicator.classname
+ org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault
+
+ Class representing block placement policy for non-striped files.
+
+
+
+
+ dfs.blockreport.incremental.intervalMsec
+ 0
+
+ If set to a positive integer, the value in ms to wait between sending
+ incremental block reports from the Datanode to the Namenode.
+
+
+
+
+ dfs.checksum.type
+ CRC32C
+
+ Checksum type
+
+
+
+
+ dfs.client.block.write.locateFollowingBlock.retries
+ 5
+
+ Number of retries to use when finding the next block during HDFS writes.
+
+
+
+
+ dfs.client.failover.proxy.provider
+
+
+ The prefix (plus a required nameservice ID) for the class name of the
+ configured Failover proxy provider for the host. For more detailed
+ information, please consult the "Configuration Details" section of
+ the HDFS High Availability documentation.
+
+
+
+
+ dfs.client.key.provider.cache.expiry
+ 864000000
+
+ DFS client security key cache expiration in milliseconds.
+
+
+
+
+ dfs.client.max.block.acquire.failures
+ 3
+
+ Maximum failures allowed when trying to get block information from a specific datanode.
+
+
+
+
+ dfs.client.read.prefetch.size
+
+
+ The number of bytes for the DFSClient will fetch from the Namenode
+ during a read operation. Defaults to 10 * ${dfs.blocksize}.
+
+
+
+
+ dfs.client.read.short.circuit.replica.stale.threshold.ms
+ 1800000
+
+ Threshold in milliseconds for read entries during short-circuit local reads.
+
+
+
+
+ dfs.client.read.shortcircuit.buffer.size
+ 1048576
+
+ Buffer size in bytes for short-circuit local reads.
+
+
+
+
+ dfs.client.replica.accessor.builder.classes
+
+
+ Comma-separated classes for building ReplicaAccessor. If the classes
+ are specified, client will use external BlockReader that uses the
+ ReplicaAccessor built by the builder.
+
+
+
+
+ dfs.client.retry.interval-ms.get-last-block-length
+ 4000
+
+ Retry interval in milliseconds to wait between retries in getting
+ block lengths from the datanodes.
+
+
+
+
+ dfs.client.retry.max.attempts
+ 10
+
+ Max retry attempts for DFSClient talking to namenodes.
+
+
+
+
+ dfs.client.retry.policy.enabled
+ false
+
+ If true, turns on DFSClient retry policy.
+
+
+
+
+ dfs.client.retry.policy.spec
+ 10000,6,60000,10
+
+ Set to pairs of timeouts and retries for DFSClient.
+
+
+
+
+ dfs.client.retry.times.get-last-block-length
+ 3
+
+ Number of retries for calls to fetchLocatedBlocksAndGetLastBlockLength().
+
+
+
+
+ dfs.client.retry.window.base
+ 3000
+
+ Base time window in ms for DFSClient retries. For each retry attempt,
+ this value is extended linearly (e.g. 3000 ms for first attempt and
+ first retry, 6000 ms for second retry, 9000 ms for third retry, etc.).
+
+
+
+
+ dfs.client.socket-timeout
+ 60000
+
+ Default timeout value in milliseconds for all sockets.
+
+
+
+
+ dfs.client.socketcache.capacity
+ 16
+
+ Socket cache capacity (in entries) for short-circuit reads.
+
+
+
+
+ dfs.client.socketcache.expiryMsec
+ 3000
+
+ Socket cache expiration for short-circuit reads in msec.
+
+
+
+
+ dfs.client.test.drop.namenode.response.number
+ 0
+
+ The number of Namenode responses dropped by DFSClient for each RPC call. Used
+ for testing the NN retry cache.
+
+
+
+
+ dfs.client.hedged.read.threadpool.size
+ 0
+
+ Support 'hedged' reads in DFSClient. To enable this feature, set the parameter
+ to a positive number. The threadpool size is how many threads to dedicate
+ to the running of these 'hedged', concurrent reads in your client.
+
+
+
+
+ dfs.client.hedged.read.threshold.millis
+ 500
+
+ Configure 'hedged' reads in DFSClient. This is the number of milliseconds
+ to wait before starting up a 'hedged' read.
+
+
+
+
+ dfs.client.use.legacy.blockreader
+ false
+
+ If true, use the RemoteBlockReader class for local read short circuit. If false, use
+ the newer RemoteBlockReader2 class.
+
+
+
+
+ dfs.client.write.byte-array-manager.count-limit
+ 2048
+
+ The maximum number of arrays allowed for each array length.
+
+
+
+
+ dfs.client.write.byte-array-manager.count-reset-time-period-ms
+ 10000
+
+ The time period in milliseconds that the allocation count for each array length is
+ reset to zero if there is no increment.
+
+
+
+
+ dfs.client.write.byte-array-manager.count-threshold
+ 128
+
+ The count threshold for each array length so that a manager is created only after the
+ allocation count exceeds the threshold. In other words, the particular array length
+ is not managed until the allocation count exceeds the threshold.
+
+
+
+
+ dfs.client.write.byte-array-manager.enabled
+ false
+
+ If true, enables byte array manager used by DFSOutputStream.
+
+
+
+
+ dfs.client.write.max-packets-in-flight
+ 80
+
+ The maximum number of DFSPackets allowed in flight.
+
+
+
+
+ dfs.content-summary.limit
+ 5000
+
+ The maximum content summary counts allowed in one locking period. 0 or a negative number
+ means no limit (i.e. no yielding).
+
+
+
+
+ dfs.content-summary.sleep-microsec
+ 500
+
+ The length of time in microseconds to put the thread to sleep, between reaquiring the locks
+ in content summary computation.
+
+
+
+
+ dfs.data.transfer.client.tcpnodelay
+ true
+
+ If true, set TCP_NODELAY to sockets for transferring data from DFS client.
+
+
+
+
+ dfs.datanode.balance.max.concurrent.moves
+ 5
+
+ Maximum number of threads for Datanode balancer pending moves. This
+ value is reconfigurable via the "dfsadmin -reconfig" command.
+
+
+
+
+ dfs.datanode.fsdataset.factory
+
+
+ The class name for the underlying storage that stores replicas for a
+ Datanode. Defaults to
+ org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.FsDatasetFactory.
+
+
+
+
+ dfs.datanode.fsdataset.volume.choosing.policy
+
+
+ The class name of the policy for choosing volumes in the list of
+ directories. Defaults to
+ org.apache.hadoop.hdfs.server.datanode.fsdataset.RoundRobinVolumeChoosingPolicy.
+ If you would like to take into account available disk space, set the
+ value to
+ "org.apache.hadoop.hdfs.server.datanode.fsdataset.AvailableSpaceVolumeChoosingPolicy".
+
+
+
+
+ dfs.datanode.hostname
+
+
+ Optional. The hostname for the Datanode containing this
+ configuration file. Will be different for each machine.
+ Defaults to current hostname.
+
+
+
+
+ dfs.datanode.lazywriter.interval.sec
+ 60
+
+ Interval in seconds for Datanodes for lazy persist writes.
+
+
+
+
+ dfs.datanode.network.counts.cache.max.size
+ 2147483647
+
+ The maximum number of entries the datanode per-host network error
+ count cache may contain.
+
+
+
+
+ dfs.datanode.oob.timeout-ms
+ 1500,0,0,0
+
+ Timeout value when sending OOB response for each OOB type, which are
+ OOB_RESTART, OOB_RESERVED1, OOB_RESERVED2, and OOB_RESERVED3,
+ respectively. Currently, only OOB_RESTART is used.
+
+
+
+
+ dfs.datanode.parallel.volumes.load.threads.num
+
+
+ Maximum number of threads to use for upgrading data directories.
+ The default value is the number of storage directories in the
+ DataNode.
+
+
+
+
+ dfs.datanode.ram.disk.replica.tracker
+
+
+ Name of the class implementing the RamDiskReplicaTracker interface.
+ Defaults to
+ org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker.
+
+
+
+
+ dfs.datanode.restart.replica.expiration
+ 50
+
+ During shutdown for restart, the amount of time in seconds budgeted for
+ datanode restart.
+
+
+
+
+ dfs.datanode.socket.reuse.keepalive
+ 4000
+
+ The window of time in ms before the DataXceiver closes a socket for a
+ single request. If a second request occurs within that window, the
+ socket can be reused.
+
+
+
+
+ dfs.datanode.socket.write.timeout
+ 480000
+
+ Timeout in ms for clients socket writes to DataNodes.
+
+
+
+
+ dfs.datanode.sync.behind.writes.in.background
+ false
+
+ If set to true, then sync_file_range() system call will occur
+ asynchronously. This property is only valid when the property
+ dfs.datanode.sync.behind.writes is true.
+
+
+
+
+ dfs.datanode.transferTo.allowed
+ true
+
+ If false, break block tranfers on 32-bit machines greater than
+ or equal to 2GB into smaller chunks.
+
+
+
+
+ dfs.ha.fencing.methods
+
+
+ A list of scripts or Java classes which will be used to fence
+ the Active NameNode during a failover. See the HDFS High
+ Availability documentation for details on automatic HA
+ configuration.
+
+
+
+
+ dfs.ha.standby.checkpoints
+ true
+
+ If true, a NameNode in Standby state periodically takes a checkpoint
+ of the namespace, saves it to its local storage and then upload to
+ the remote NameNode.
+
+
+
+
+ dfs.ha.zkfc.port
+ 8019
+
+ The port number that the zookeeper failover controller RPC
+ server binds to.
+
+
+
+
+ dfs.journalnode.edits.dir
+ /tmp/hadoop/dfs/journalnode/
+
+ The directory where the journal edit files are stored.
+
+
+
+
+ dfs.journalnode.kerberos.internal.spnego.principal
+
+
+ Kerberos SPNEGO principal name used by the journal node.
+
+
+
+
+ dfs.journalnode.kerberos.principal
+
+
+ Kerberos principal name for the journal node.
+
+
+
+
+ dfs.journalnode.keytab.file
+
+
+ Kerberos keytab file for the journal node.
+
+
+
+
+ dfs.ls.limit
+ 1000
+
+ Limit the number of files printed by ls. If less or equal to
+ zero, at most DFS_LIST_LIMIT_DEFAULT (= 1000) will be printed.
+
+
+
+
+ dfs.mover.movedWinWidth
+ 5400000
+
+ The minimum time interval, in milliseconds, that a block can be
+ moved to another location again.
+
+
+
+
+ dfs.mover.moverThreads
+ 1000
+
+ Configure the balancer's mover thread pool size.
+
+
+
+
+ dfs.mover.retry.max.attempts
+ 10
+
+ The maximum number of retries before the mover consider the
+ move failed.
+
+
+
+
+ dfs.namenode.audit.log.async
+ false
+
+ If true, enables asynchronous audit log.
+
+
+
+
+ dfs.namenode.audit.log.token.tracking.id
+ false
+
+ If true, adds a tracking ID for all audit log events.
+
+
+
+
+ dfs.namenode.available-space-block-placement-policy.balanced-space-preference-fraction
+ 0.6
+
+ Only used when the dfs.block.replicator.classname is set to
+ org.apache.hadoop.hdfs.server.blockmanagement.AvailableSpaceBlockPlacementPolicy.
+ Special value between 0 and 1, noninclusive. Increases chance of
+ placing blocks on Datanodes with less disk space used.
+
+
+
+
+ dfs.namenode.backup.dnrpc-address
+
+
+ Service RPC address for the backup Namenode.
+
+
+
+
+ dfs.namenode.delegation.token.always-use
+ false
+
+ For testing. Setting to true always allows the DT secret manager
+ to be used, even if security is disabled.
+
+
+
+
+ dfs.namenode.edits.asynclogging
+ false
+
+ If set to true, enables asynchronous edit logs in the Namenode. If set
+ to false, the Namenode uses the traditional synchronous edit logs.
+
+
+
+
+ dfs.namenode.edits.dir.minimum
+ 1
+
+ dfs.namenode.edits.dir includes both required directories
+ (specified by dfs.namenode.edits.dir.required) and optional directories.
+
+ The number of usable optional directories must be greater than or equal
+ to this property. If the number of usable optional directories falls
+ below dfs.namenode.edits.dir.minimum, HDFS will issue an error.
+
+ This property defaults to 1.
+
+
+
+
+ dfs.namenode.edits.journal-plugin
+
+
+ When FSEditLog is creating JournalManagers from dfs.namenode.edits.dir,
+ and it encounters a URI with a schema different to "file" it loads the
+ name of the implementing class from
+ "dfs.namenode.edits.journal-plugin.[schema]". This class must implement
+ JournalManager and have a constructor which takes (Configuration, URI).
+
+
+
+
+ dfs.namenode.file.close.num-committed-allowed
+ 0
+
+ Normally a file can only be closed with all its blocks are committed.
+ When this value is set to a positive integer N, a file can be closed
+ when N blocks are committed and the rest complete.
+
+
+
+
+ dfs.namenode.inode.attributes.provider.class
+
+
+ Name of class to use for delegating HDFS authorization.
+
+
+
+
+ dfs.namenode.max-num-blocks-to-log
+ 1000
+
+ Puts a limit on the number of blocks printed to the log by the Namenode
+ after a block report.
+
+
+
+
+ dfs.namenode.max.op.size
+ 52428800
+
+ Maximum opcode size in bytes.
+
+
+
+
+ dfs.namenode.missing.checkpoint.periods.before.shutdown
+ 3
+
+ The number of checkpoint period windows (as defined by the property
+ dfs.namenode.checkpoint.period) allowed by the Namenode to perform
+ saving the namespace before shutdown.
+
+
+
+
+ dfs.namenode.name.cache.threshold
+ 10
+
+ Frequently accessed files that are accessed more times than this
+ threshold are cached in the FSDirectory nameCache.
+
+
+
+
+ dfs.namenode.replication.max-streams
+ 2
+
+ Hard limit for the number of highest-priority replication streams.
+
+
+
+
+ dfs.namenode.replication.max-streams-hard-limit
+ 4
+
+ Hard limit for all replication streams.
+
+
+
+
+ dfs.namenode.replication.pending.timeout-sec
+ -1
+
+ Timeout in seconds for block replication. If this value is 0 or less,
+ then it will default to 5 minutes.
+
+
+
+
+ dfs.namenode.stale.datanode.minimum.interval
+ 3
+
+ Minimum number of missed heartbeats intervals for a datanode to
+ be marked stale by the Namenode. The actual interval is calculated as
+ (dfs.namenode.stale.datanode.minimum.interval * dfs.heartbeat.interval)
+ in seconds. If this value is greater than the property
+ dfs.namenode.stale.datanode.interval, then the calculated value above
+ is used.
+
+
+
+
+ dfs.namenode.storageinfo.defragment.timeout.ms
+ 4
+
+ Timeout value in ms for the StorageInfo compaction run.
+
+
+
+
+ dfs.namenode.storageinfo.defragment.interval.ms
+ 600000
+
+ The thread for checking the StorageInfo for defragmentation will
+ run periodically. The time between runs is determined by this
+ property.
+
+
+
+
+ dfs.namenode.storageinfo.defragment.ratio
+ 0.75
+
+ The defragmentation threshold for the StorageInfo.
+
+
+
+
+ dfs.pipeline.ecn
+ false
+
+ If true, allows ECN (explicit congestion notification) from the
+ Datanode.
+
+
+
+
+ dfs.qjournal.accept-recovery.timeout.ms
+ 120000
+
+ Quorum timeout in milliseconds during accept phase of
+ recovery/synchronization for a specific segment.
+
+
+
+
+ dfs.qjournal.finalize-segment.timeout.ms
+ 120000
+
+ Quorum timeout in milliseconds during finalizing for a specific
+ segment.
+
+
+
+
+ dfs.qjournal.get-journal-state.timeout.ms
+ 120000
+
+ Timeout in milliseconds when calling getJournalState().
+ JournalNodes.
+
+
+
+
+ dfs.qjournal.new-epoch.timeout.ms
+ 120000
+
+ Timeout in milliseconds when getting an epoch number for write
+ access to JournalNodes.
+
+
+
+
+ dfs.qjournal.prepare-recovery.timeout.ms
+ 120000
+
+ Quorum timeout in milliseconds during preparation phase of
+ recovery/synchronization for a specific segment.
+
+
+
+
+ dfs.qjournal.queued-edits.limit.mb
+ 10
+
+ Queue size in MB for quorum journal edits.
+
+
+
+
+ dfs.qjournal.select-input-streams.timeout.ms
+ 20000
+
+ Timeout in milliseconds for accepting streams from JournalManagers.
+
+
+
+
+ dfs.qjournal.start-segment.timeout.ms
+ 20000
+
+ Quorum timeout in milliseconds for starting a log segment.
+
+
+
+
+ dfs.qjournal.write-txns.timeout.ms
+ 20000
+
+ Write timeout in milliseconds when writing to a quorum of remote
+ journals.
+
+
+
+
+ dfs.quota.by.storage.type.enabled
+ true
+
+ If true, enables quotas based on storage type.
+
+
+
+
+ dfs.secondary.namenode.kerberos.principal
+
+
+ Kerberos principal name for the Secondary NameNode.
+
+
+
+
+ dfs.secondary.namenode.keytab.file
+
+
+ Kerberos keytab file for the Secondary NameNode.
+
+
+
+
+ dfs.web.authentication.filter
+ org.apache.hadoop.hdfs.web.AuthFilter
+
+ Authentication filter class used for WebHDFS.
+
+
+
+
+ dfs.web.authentication.simple.anonymous.allowed
+
+
+ If true, allow anonymous user to access WebHDFS. Set to
+ false to disable anonymous authentication.
+
+
+
+
+ dfs.web.ugi
+
+
+ dfs.web.ugi is deprecated. Use hadoop.http.staticuser.user instead.
+
+
+
+
+ dfs.webhdfs.netty.high.watermark
+ 65535
+
+ High watermark configuration to Netty for Datanode WebHdfs.
+
+
+
+
+ dfs.webhdfs.netty.low.watermark
+ 32768
+
+ Low watermark configuration to Netty for Datanode WebHdfs.
+
+
+
+
+ dfs.webhdfs.oauth2.access.token.provider
+
+
+ Access token provider class for WebHDFS using OAuth2.
+ Defaults to org.apache.hadoop.hdfs.web.oauth2.ConfCredentialBasedAccessTokenProvider.
+
+
+
+
+ dfs.webhdfs.oauth2.client.id
+
+
+ Client id used to obtain access token with either credential or
+ refresh token.
+
+
+
+
+ dfs.webhdfs.oauth2.enabled
+ false
+
+ If true, enables OAuth2 in WebHDFS
+
+
+
+
+ dfs.webhdfs.oauth2.refresh.url
+
+
+ URL against which to post for obtaining bearer token with
+ either credential or refresh token.
+
+
+
+
+ ssl.server.keystore.keypassword
+
+
+ Keystore key password for HTTPS SSL configuration
+
+
+
+
+ ssl.server.keystore.location
+
+
+ Keystore location for HTTPS SSL configuration
+
+
+
+
+ ssl.server.keystore.password
+
+
+ Keystore password for HTTPS SSL configuration
+
+
+
+
+ ssl.server.truststore.location
+
+
+ Truststore location for HTTPS SSL configuration
+
+
+
+
+ ssl.server.truststore.password
+
+
+ Truststore password for HTTPS SSL configuration
+
+
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
index 9637f59ad96..46420f101f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/tools/TestHdfsConfigFields.java
@@ -45,7 +45,67 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
// Set error modes
errorIfMissingConfigProps = true;
- errorIfMissingXmlProps = false;
+ errorIfMissingXmlProps = true;
+
+ // Initialize used variables
+ configurationPropsToSkipCompare = new HashSet();
+
+ // Ignore testing based parameter
+ configurationPropsToSkipCompare.add("ignore.secure.ports.for.testing");
+
+ // Remove deprecated properties listed in Configuration#DeprecationDelta
+ configurationPropsToSkipCompare.add(DFSConfigKeys.DFS_DF_INTERVAL_KEY);
+
+ // Remove default properties
+ configurationPropsToSkipCompare
+ .add(DFSConfigKeys.DFS_IMAGE_COMPRESSION_CODEC_DEFAULT);
+ configurationPropsToSkipCompare
+ .add(DFSConfigKeys.DFS_WEBHDFS_AUTHENTICATION_FILTER_DEFAULT);
+
+ // Remove support property
+ configurationPropsToSkipCompare
+ .add(DFSConfigKeys.DFS_NAMENODE_MIN_SUPPORTED_DATANODE_VERSION_KEY);
+ configurationPropsToSkipCompare
+ .add(DFSConfigKeys.DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY);
+
+ // Purposely hidden, based on comments in DFSConfigKeys
+ configurationPropsToSkipCompare
+ .add(DFSConfigKeys.DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY);
+
+ // Fully deprecated properties?
+ configurationPropsToSkipCompare
+ .add("dfs.corruptfilesreturned.max");
+ configurationPropsToSkipCompare
+ .add("dfs.datanode.hdfs-blocks-metadata.enabled");
+ configurationPropsToSkipCompare
+ .add("dfs.metrics.session-id");
+ configurationPropsToSkipCompare
+ .add("dfs.datanode.synconclose");
+ configurationPropsToSkipCompare
+ .add("dfs.datanode.non.local.lazy.persist");
+ configurationPropsToSkipCompare
+ .add("dfs.namenode.tolerate.heartbeat.multiplier");
+ configurationPropsToSkipCompare
+ .add("dfs.namenode.stripe.min");
+ configurationPropsToSkipCompare
+ .add("dfs.namenode.replqueue.threshold-pct");
+
+ // Removed by HDFS-6440
+ configurationPropsToSkipCompare
+ .add("dfs.ha.log-roll.rpc.timeout");
+
+ // Example (not real) property in hdfs-default.xml
+ configurationPropsToSkipCompare.add("dfs.ha.namenodes");
+
+ // Property used for internal testing only
+ configurationPropsToSkipCompare
+ .add(DFSConfigKeys.DFS_DATANODE_DUPLICATE_REPLICA_DELETION);
+
+ // Property not intended for users
+ configurationPropsToSkipCompare
+ .add(DFSConfigKeys.DFS_DATANODE_STARTUP_KEY);
+ configurationPropsToSkipCompare
+ .add(DFSConfigKeys.DFS_NAMENODE_STARTUP_KEY);
// Allocate
xmlPropsToSkipCompare = new HashSet();
@@ -58,21 +118,12 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
// Used dynamically as part of DFSConfigKeys.DFS_NAMENODE_EDITS_PLUGIN_PREFIX
xmlPropsToSkipCompare.add("dfs.namenode.edits.journal-plugin.qjournal");
- // Example (not real) property in hdfs-default.xml
- xmlPropsToSkipCompare.add("dfs.ha.namenodes.EXAMPLENAMESERVICE");
-
// Defined in org.apache.hadoop.fs.CommonConfigurationKeys
xmlPropsToSkipCompare.add("hadoop.user.group.metrics.percentiles.intervals");
// Used oddly by DataNode to create new config String
xmlPropsToSkipCompare.add("hadoop.hdfs.configuration.version");
- // Kept in the NfsConfiguration class in the hadoop-hdfs-nfs module
- xmlPrefixToSkipCompare.add("nfs");
-
- // Not a hardcoded property. Used by SaslRpcClient
- xmlPrefixToSkipCompare.add("dfs.namenode.kerberos.principal.pattern");
-
// Skip comparing in branch-2. Removed in trunk with HDFS-7985.
xmlPropsToSkipCompare.add("dfs.webhdfs.enabled");
@@ -82,5 +133,21 @@ public class TestHdfsConfigFields extends TestConfigurationFieldsBase {
// Ignore HTrace properties
xmlPropsToSkipCompare.add("fs.client.htrace");
xmlPropsToSkipCompare.add("hadoop.htrace");
+
+ // Ignore SpanReceiveHost properties
+ xmlPropsToSkipCompare.add("dfs.htrace.spanreceiver.classes");
+ xmlPropsToSkipCompare.add("dfs.client.htrace.spanreceiver.classes");
+
+ // Remove deprecated properties listed in Configuration#DeprecationDelta
+ xmlPropsToSkipCompare.add(DFSConfigKeys.DFS_DF_INTERVAL_KEY);
+
+ // Kept in the NfsConfiguration class in the hadoop-hdfs-nfs module
+ xmlPrefixToSkipCompare.add("nfs");
+
+ // Not a hardcoded property. Used by SaslRpcClient
+ xmlPrefixToSkipCompare.add("dfs.namenode.kerberos.principal.pattern");
+
+ // Skip over example property
+ xmlPrefixToSkipCompare.add("dfs.ha.namenodes");
}
}