Skip to content

Commit

Permalink
HDFS-9895. Remove unnecessary conf cache from DataNode. Contributed b…
Browse files Browse the repository at this point in the history
…y Xiaobing Zhou.
  • Loading branch information
arp7 committed Sep 15, 2016
1 parent fcbac00 commit ec3ea18
Show file tree
Hide file tree
Showing 6 changed files with 130 additions and 117 deletions.
Expand Up @@ -165,6 +165,10 @@ private static long getConfiguredScanPeriodMs(Configuration conf) {
} }
} }


public BlockScanner(DataNode datanode) {
this(datanode, datanode.getConf());
}

public BlockScanner(DataNode datanode, Configuration conf) { public BlockScanner(DataNode datanode, Configuration conf) {
this.datanode = datanode; this.datanode = datanode;
this.conf = new Conf(conf); this.conf = new Conf(conf);
Expand Down
Expand Up @@ -56,6 +56,7 @@
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BP_READY_TIMEOUT_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BP_READY_TIMEOUT_KEY;
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT;


import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
Expand All @@ -72,7 +73,6 @@
*/ */
@InterfaceAudience.Private @InterfaceAudience.Private
public class DNConf { public class DNConf {
final Configuration conf;
final int socketTimeout; final int socketTimeout;
final int socketWriteTimeout; final int socketWriteTimeout;
final int socketKeepaliveTimeout; final int socketKeepaliveTimeout;
Expand Down Expand Up @@ -120,73 +120,77 @@ public class DNConf {
private final int volFailuresTolerated; private final int volFailuresTolerated;
private final int volsConfigured; private final int volsConfigured;
private final int maxDataLength; private final int maxDataLength;
private Configurable dn;


public DNConf(Configuration conf) { public DNConf(final Configurable dn) {
this.conf = conf; this.dn = dn;
socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY, socketTimeout = getConf().getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
HdfsConstants.READ_TIMEOUT); HdfsConstants.READ_TIMEOUT);
socketWriteTimeout = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY, socketWriteTimeout = getConf().getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
HdfsConstants.WRITE_TIMEOUT); HdfsConstants.WRITE_TIMEOUT);
socketKeepaliveTimeout = conf.getInt( socketKeepaliveTimeout = getConf().getInt(
DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY, DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY,
DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT); DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_DEFAULT);
this.transferSocketSendBufferSize = conf.getInt( this.transferSocketSendBufferSize = getConf().getInt(
DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_SEND_BUFFER_SIZE_KEY, DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_SEND_BUFFER_SIZE_KEY,
DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_SEND_BUFFER_SIZE_DEFAULT); DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_SEND_BUFFER_SIZE_DEFAULT);
this.transferSocketRecvBufferSize = conf.getInt( this.transferSocketRecvBufferSize = getConf().getInt(
DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_RECV_BUFFER_SIZE_KEY, DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_RECV_BUFFER_SIZE_KEY,
DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_RECV_BUFFER_SIZE_DEFAULT); DFSConfigKeys.DFS_DATANODE_TRANSFER_SOCKET_RECV_BUFFER_SIZE_DEFAULT);
this.tcpNoDelay = conf.getBoolean( this.tcpNoDelay = getConf().getBoolean(
DFSConfigKeys.DFS_DATA_TRANSFER_SERVER_TCPNODELAY, DFSConfigKeys.DFS_DATA_TRANSFER_SERVER_TCPNODELAY,
DFSConfigKeys.DFS_DATA_TRANSFER_SERVER_TCPNODELAY_DEFAULT); DFSConfigKeys.DFS_DATA_TRANSFER_SERVER_TCPNODELAY_DEFAULT);


/* Based on results on different platforms, we might need set the default /* Based on results on different platforms, we might need set the default
* to false on some of them. */ * to false on some of them. */
transferToAllowed = conf.getBoolean( transferToAllowed = getConf().getBoolean(
DFS_DATANODE_TRANSFERTO_ALLOWED_KEY, DFS_DATANODE_TRANSFERTO_ALLOWED_KEY,
DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT); DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT);


writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY, writePacketSize = getConf().getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT); DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);


readaheadLength = conf.getLong( readaheadLength = getConf().getLong(
HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY, HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY,
HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT); HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
maxDataLength = conf.getInt(DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH, maxDataLength = getConf().getInt(DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH,
DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT); DFSConfigKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
dropCacheBehindWrites = conf.getBoolean( dropCacheBehindWrites = getConf().getBoolean(
DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_KEY, DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_KEY,
DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_DEFAULT); DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_DEFAULT);
syncBehindWrites = conf.getBoolean( syncBehindWrites = getConf().getBoolean(
DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_KEY, DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_KEY,
DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_DEFAULT); DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_DEFAULT);
syncBehindWritesInBackground = conf.getBoolean( syncBehindWritesInBackground = getConf().getBoolean(
DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_IN_BACKGROUND_KEY, DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_IN_BACKGROUND_KEY,
DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_IN_BACKGROUND_DEFAULT); DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_IN_BACKGROUND_DEFAULT);
dropCacheBehindReads = conf.getBoolean( dropCacheBehindReads = getConf().getBoolean(
DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY, DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY,
DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT); DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT);
connectToDnViaHostname = conf.getBoolean( connectToDnViaHostname = getConf().getBoolean(
DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME, DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME,
DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT); DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME_DEFAULT);
this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, this.blockReportInterval = getConf().getLong(
DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT); DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
this.ibrInterval = conf.getLong( this.ibrInterval = getConf().getLong(
DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_KEY, DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_KEY,
DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_DEFAULT); DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_DEFAULT);
this.blockReportSplitThreshold = conf.getLong(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY, this.blockReportSplitThreshold = getConf().getLong(
DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT); DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY,
this.cacheReportInterval = conf.getLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY, DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT);
this.cacheReportInterval = getConf().getLong(
DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT); DFS_CACHEREPORT_INTERVAL_MSEC_DEFAULT);


this.dfsclientSlowIoWarningThresholdMs = conf.getLong( this.dfsclientSlowIoWarningThresholdMs = getConf().getLong(
HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY, HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_KEY,
HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT); HdfsClientConfigKeys.DFS_CLIENT_SLOW_IO_WARNING_THRESHOLD_DEFAULT);
this.datanodeSlowIoWarningThresholdMs = conf.getLong( this.datanodeSlowIoWarningThresholdMs = getConf().getLong(
DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY, DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY,
DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT); DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT);


long initBRDelay = conf.getTimeDuration( long initBRDelay = getConf().getTimeDuration(
DFS_BLOCKREPORT_INITIAL_DELAY_KEY, DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT, TimeUnit.SECONDS) * 1000L; DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT, TimeUnit.SECONDS) * 1000L;
if (initBRDelay >= blockReportInterval) { if (initBRDelay >= blockReportInterval) {
Expand All @@ -197,11 +201,11 @@ public DNConf(Configuration conf) {
} }
initialBlockReportDelayMs = initBRDelay; initialBlockReportDelayMs = initBRDelay;


heartBeatInterval = conf.getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY, heartBeatInterval = getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS) * 1000L; DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS) * 1000L;
long confLifelineIntervalMs = long confLifelineIntervalMs =
conf.getLong(DFS_DATANODE_LIFELINE_INTERVAL_SECONDS_KEY, getConf().getLong(DFS_DATANODE_LIFELINE_INTERVAL_SECONDS_KEY,
3 * conf.getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY, 3 * getConf().getTimeDuration(DFS_HEARTBEAT_INTERVAL_KEY,
DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS)) * 1000L; DFS_HEARTBEAT_INTERVAL_DEFAULT, TimeUnit.SECONDS)) * 1000L;
if (confLifelineIntervalMs <= heartBeatInterval) { if (confLifelineIntervalMs <= heartBeatInterval) {
confLifelineIntervalMs = 3 * heartBeatInterval; confLifelineIntervalMs = 3 * heartBeatInterval;
Expand All @@ -215,47 +219,50 @@ public DNConf(Configuration conf) {
lifelineIntervalMs = confLifelineIntervalMs; lifelineIntervalMs = confLifelineIntervalMs;


// do we need to sync block file contents to disk when blockfile is closed? // do we need to sync block file contents to disk when blockfile is closed?
this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY, this.syncOnClose = getConf().getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY,
DFS_DATANODE_SYNCONCLOSE_DEFAULT); DFS_DATANODE_SYNCONCLOSE_DEFAULT);


this.minimumNameNodeVersion = conf.get(DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY, this.minimumNameNodeVersion = getConf().get(
DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_KEY,
DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT); DFS_DATANODE_MIN_SUPPORTED_NAMENODE_VERSION_DEFAULT);


this.encryptDataTransfer = conf.getBoolean(DFS_ENCRYPT_DATA_TRANSFER_KEY, this.encryptDataTransfer = getConf().getBoolean(
DFS_ENCRYPT_DATA_TRANSFER_KEY,
DFS_ENCRYPT_DATA_TRANSFER_DEFAULT); DFS_ENCRYPT_DATA_TRANSFER_DEFAULT);
this.encryptionAlgorithm = conf.get(DFS_DATA_ENCRYPTION_ALGORITHM_KEY); this.encryptionAlgorithm = getConf().get(DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
this.trustedChannelResolver = TrustedChannelResolver.getInstance(conf); this.trustedChannelResolver = TrustedChannelResolver.getInstance(getConf());
this.saslPropsResolver = DataTransferSaslUtil.getSaslPropertiesResolver( this.saslPropsResolver = DataTransferSaslUtil.getSaslPropertiesResolver(
conf); getConf());
this.ignoreSecurePortsForTesting = conf.getBoolean( this.ignoreSecurePortsForTesting = getConf().getBoolean(
IGNORE_SECURE_PORTS_FOR_TESTING_KEY, IGNORE_SECURE_PORTS_FOR_TESTING_KEY,
IGNORE_SECURE_PORTS_FOR_TESTING_DEFAULT); IGNORE_SECURE_PORTS_FOR_TESTING_DEFAULT);


this.xceiverStopTimeout = conf.getLong( this.xceiverStopTimeout = getConf().getLong(
DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY, DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_KEY,
DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT); DFS_DATANODE_XCEIVER_STOP_TIMEOUT_MILLIS_DEFAULT);


this.maxLockedMemory = conf.getLong( this.maxLockedMemory = getConf().getLong(
DFS_DATANODE_MAX_LOCKED_MEMORY_KEY, DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT); DFS_DATANODE_MAX_LOCKED_MEMORY_DEFAULT);


this.restartReplicaExpiry = conf.getLong( this.restartReplicaExpiry = getConf().getLong(
DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY, DFS_DATANODE_RESTART_REPLICA_EXPIRY_KEY,
DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT) * 1000L; DFS_DATANODE_RESTART_REPLICA_EXPIRY_DEFAULT) * 1000L;


this.allowNonLocalLazyPersist = conf.getBoolean( this.allowNonLocalLazyPersist = getConf().getBoolean(
DFS_DATANODE_NON_LOCAL_LAZY_PERSIST, DFS_DATANODE_NON_LOCAL_LAZY_PERSIST,
DFS_DATANODE_NON_LOCAL_LAZY_PERSIST_DEFAULT); DFS_DATANODE_NON_LOCAL_LAZY_PERSIST_DEFAULT);


this.bpReadyTimeout = conf.getTimeDuration( this.bpReadyTimeout = getConf().getTimeDuration(
DFS_DATANODE_BP_READY_TIMEOUT_KEY, DFS_DATANODE_BP_READY_TIMEOUT_KEY,
DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT, TimeUnit.SECONDS); DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT, TimeUnit.SECONDS);


this.volFailuresTolerated = this.volFailuresTolerated =
conf.getInt(DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY, getConf().getInt(
DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_KEY,
DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT); DFSConfigKeys.DFS_DATANODE_FAILED_VOLUMES_TOLERATED_DEFAULT);
String[] dataDirs = String[] dataDirs =
conf.getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY); getConf().getTrimmedStrings(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY);
this.volsConfigured = (dataDirs == null) ? 0 : dataDirs.length; this.volsConfigured = (dataDirs == null) ? 0 : dataDirs.length;
} }


Expand All @@ -270,7 +277,7 @@ String getMinimumNameNodeVersion() {
* @return Configuration the configuration * @return Configuration the configuration
*/ */
public Configuration getConf() { public Configuration getConf() {
return conf; return this.dn.getConf();
} }


/** /**
Expand Down

0 comments on commit ec3ea18

Please sign in to comment.