Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
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) {
this.datanode = datanode;
this.conf = new Conf(conf);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -70,7 +70,6 @@
*/
@InterfaceAudience.Private
public class DNConf {
final Configuration conf;
final int socketTimeout;
final int socketWriteTimeout;
final int socketKeepaliveTimeout;
Expand Down Expand Up @@ -113,71 +112,71 @@ public class DNConf {

// Allow LAZY_PERSIST writes from non-local clients?
private final boolean allowNonLocalLazyPersist;

private final DataNode dn;
private final int volFailuresTolerated;
private final int volsConfigured;

public DNConf(Configuration conf) {
this.conf = conf;
socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
public DNConf(final DataNode dn) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The dn.getConf() object is not referenced outside the constructor so you can just pass a reference to that object. Also DNConf need not keep a reference to the dn. I think you can just revert all changes to this file.

this.dn = dn;
socketTimeout = getConf().getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
HdfsConstants.READ_TIMEOUT);
socketWriteTimeout = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
socketWriteTimeout = getConf().getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
HdfsConstants.WRITE_TIMEOUT);
socketKeepaliveTimeout = conf.getInt(
socketKeepaliveTimeout = getConf().getInt(
DFSConfigKeys.DFS_DATANODE_SOCKET_REUSE_KEEPALIVE_KEY,
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_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_DEFAULT);

/* Based on results on different platforms, we might need set the default
* to false on some of them. */
transferToAllowed = conf.getBoolean(
transferToAllowed = getConf().getBoolean(
DFS_DATANODE_TRANSFERTO_ALLOWED_KEY,
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);

readaheadLength = conf.getLong(
readaheadLength = getConf().getLong(
HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY,
HdfsClientConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
dropCacheBehindWrites = conf.getBoolean(
dropCacheBehindWrites = getConf().getBoolean(
DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_KEY,
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_DEFAULT);
syncBehindWritesInBackground = conf.getBoolean(
syncBehindWritesInBackground = getConf().getBoolean(
DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_IN_BACKGROUND_KEY,
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_DEFAULT);
connectToDnViaHostname = conf.getBoolean(
connectToDnViaHostname = getConf().getBoolean(
DFSConfigKeys.DFS_DATANODE_USE_DN_HOSTNAME,
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);
this.ibrInterval = conf.getLong(
this.ibrInterval = getConf().getLong(
DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_KEY,
DFSConfigKeys.DFS_BLOCKREPORT_INCREMENTAL_INTERVAL_MSEC_DEFAULT);
this.blockReportSplitThreshold = conf.getLong(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY,
this.blockReportSplitThreshold = getConf().getLong(DFS_BLOCKREPORT_SPLIT_THRESHOLD_KEY,
DFS_BLOCKREPORT_SPLIT_THRESHOLD_DEFAULT);
this.cacheReportInterval = conf.getLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
this.cacheReportInterval = getConf().getLong(DFS_CACHEREPORT_INTERVAL_MSEC_KEY,
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_DEFAULT);
this.datanodeSlowIoWarningThresholdMs = conf.getLong(
this.datanodeSlowIoWarningThresholdMs = getConf().getLong(
DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY,
DFSConfigKeys.DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_DEFAULT);

long initBRDelay = conf.getLong(
long initBRDelay = getConf().getLong(
DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT) * 1000L;
if (initBRDelay >= blockReportInterval) {
Expand All @@ -188,11 +187,11 @@ public DNConf(Configuration conf) {
}
initialBlockReportDelayMs = initBRDelay;

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

// 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);

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);

this.encryptDataTransfer = conf.getBoolean(DFS_ENCRYPT_DATA_TRANSFER_KEY,
this.encryptDataTransfer = getConf().getBoolean(DFS_ENCRYPT_DATA_TRANSFER_KEY,
DFS_ENCRYPT_DATA_TRANSFER_DEFAULT);
this.encryptionAlgorithm = conf.get(DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
this.trustedChannelResolver = TrustedChannelResolver.getInstance(conf);
this.encryptionAlgorithm = getConf().get(DFS_DATA_ENCRYPTION_ALGORITHM_KEY);
this.trustedChannelResolver = TrustedChannelResolver.getInstance(getConf());
this.saslPropsResolver = DataTransferSaslUtil.getSaslPropertiesResolver(
conf);
this.ignoreSecurePortsForTesting = conf.getBoolean(
getConf());
this.ignoreSecurePortsForTesting = getConf().getBoolean(
IGNORE_SECURE_PORTS_FOR_TESTING_KEY,
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_DEFAULT);

this.maxLockedMemory = conf.getLong(
this.maxLockedMemory = getConf().getLong(
DFS_DATANODE_MAX_LOCKED_MEMORY_KEY,
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_DEFAULT) * 1000L;

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

this.bpReadyTimeout = conf.getLong(
this.bpReadyTimeout = getConf().getLong(
DFS_DATANODE_BP_READY_TIMEOUT_KEY,
DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT);

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);
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;
}

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

/**
Expand Down
Loading