|
@@ -19,15 +19,8 @@ package org.apache.hadoop.hdfs.server.datanode;
|
|
|
|
|
|
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_ADMIN;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INITIAL_DELAY_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_HTTPS_NEED_AUTH_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_CLIENT_WRITE_PACKET_SIZE_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_ADDRESS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY;
|
|
@@ -51,17 +44,10 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOUR
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SCAN_PERIOD_HOURS_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SIMULATEDDATASTORAGE_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STARTUP_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_STORAGEID_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_SYNCONCLOSE_KEY;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_TRANSFERTO_ALLOWED_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_DATANODE_USER_NAME_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_FEDERATION_NAMESERVICES;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT;
|
|
|
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ENABLED_KEY;
|
|
|
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ENABLED_DEFAULT;
|
|
|
import static org.apache.hadoop.hdfs.server.common.Util.now;
|
|
@@ -104,7 +90,6 @@ import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.LocalFileSystem;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
-import org.apache.hadoop.hdfs.DFSConfigKeys;
|
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
import org.apache.hadoop.hdfs.HDFSPolicyProvider;
|
|
|
import org.apache.hadoop.hdfs.HdfsConfiguration;
|
|
@@ -396,9 +381,7 @@ public class DataNode extends Configured
|
|
|
AtomicInteger xmitsInProgress = new AtomicInteger();
|
|
|
Daemon dataXceiverServer = null;
|
|
|
ThreadGroup threadGroup = null;
|
|
|
- long blockReportInterval;
|
|
|
- long initialBlockReportDelay = DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT * 1000L;
|
|
|
- long heartBeatInterval;
|
|
|
+ private DNConf dnConf;
|
|
|
private boolean heartbeatsDisabledForTests = false;
|
|
|
private DataStorage storage = null;
|
|
|
private HttpServer infoServer = null;
|
|
@@ -408,18 +391,9 @@ public class DataNode extends Configured
|
|
|
private volatile String hostName; // Host name of this datanode
|
|
|
|
|
|
private static String dnThreadName;
|
|
|
- int socketTimeout;
|
|
|
- int socketWriteTimeout = 0;
|
|
|
- boolean transferToAllowed = true;
|
|
|
- private boolean dropCacheBehindWrites = false;
|
|
|
- private boolean syncBehindWrites = false;
|
|
|
- private boolean dropCacheBehindReads = false;
|
|
|
- private long readaheadLength = 0;
|
|
|
-
|
|
|
- int writePacketSize = 0;
|
|
|
+
|
|
|
boolean isBlockTokenEnabled;
|
|
|
BlockPoolTokenSecretManager blockPoolTokenSecretManager;
|
|
|
- boolean syncOnClose;
|
|
|
|
|
|
public DataBlockScanner blockScanner = null;
|
|
|
private DirectoryScanner directoryScanner = null;
|
|
@@ -487,49 +461,6 @@ public class DataNode extends Configured
|
|
|
return name;
|
|
|
}
|
|
|
|
|
|
- private void initConfig(Configuration conf) {
|
|
|
- this.socketTimeout = conf.getInt(DFS_CLIENT_SOCKET_TIMEOUT_KEY,
|
|
|
- HdfsServerConstants.READ_TIMEOUT);
|
|
|
- this.socketWriteTimeout = conf.getInt(DFS_DATANODE_SOCKET_WRITE_TIMEOUT_KEY,
|
|
|
- HdfsServerConstants.WRITE_TIMEOUT);
|
|
|
- /* Based on results on different platforms, we might need set the default
|
|
|
- * to false on some of them. */
|
|
|
- this.transferToAllowed = conf.getBoolean(
|
|
|
- DFS_DATANODE_TRANSFERTO_ALLOWED_KEY,
|
|
|
- DFS_DATANODE_TRANSFERTO_ALLOWED_DEFAULT);
|
|
|
- this.writePacketSize = conf.getInt(DFS_CLIENT_WRITE_PACKET_SIZE_KEY,
|
|
|
- DFS_CLIENT_WRITE_PACKET_SIZE_DEFAULT);
|
|
|
-
|
|
|
- this.readaheadLength = conf.getLong(
|
|
|
- DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_READAHEAD_BYTES_DEFAULT);
|
|
|
- this.dropCacheBehindWrites = conf.getBoolean(
|
|
|
- DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_WRITES_DEFAULT);
|
|
|
- this.syncBehindWrites = conf.getBoolean(
|
|
|
- DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_SYNC_BEHIND_WRITES_DEFAULT);
|
|
|
- this.dropCacheBehindReads = conf.getBoolean(
|
|
|
- DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_KEY,
|
|
|
- DFSConfigKeys.DFS_DATANODE_DROP_CACHE_BEHIND_READS_DEFAULT);
|
|
|
-
|
|
|
- this.blockReportInterval = conf.getLong(DFS_BLOCKREPORT_INTERVAL_MSEC_KEY,
|
|
|
- DFS_BLOCKREPORT_INTERVAL_MSEC_DEFAULT);
|
|
|
- this.initialBlockReportDelay = conf.getLong(
|
|
|
- DFS_BLOCKREPORT_INITIAL_DELAY_KEY,
|
|
|
- DFS_BLOCKREPORT_INITIAL_DELAY_DEFAULT) * 1000L;
|
|
|
- if (this.initialBlockReportDelay >= blockReportInterval) {
|
|
|
- this.initialBlockReportDelay = 0;
|
|
|
- LOG.info("dfs.blockreport.initialDelay is greater than " +
|
|
|
- "dfs.blockreport.intervalMsec." + " Setting initial delay to 0 msec:");
|
|
|
- }
|
|
|
- this.heartBeatInterval = conf.getLong(DFS_HEARTBEAT_INTERVAL_KEY,
|
|
|
- DFS_HEARTBEAT_INTERVAL_DEFAULT) * 1000L;
|
|
|
-
|
|
|
- // do we need to sync block file contents to disk when blockfile is closed?
|
|
|
- this.syncOnClose = conf.getBoolean(DFS_DATANODE_SYNCONCLOSE_KEY,
|
|
|
- DFS_DATANODE_SYNCONCLOSE_DEFAULT);
|
|
|
- }
|
|
|
|
|
|
private void startInfoServer(Configuration conf) throws IOException {
|
|
|
// create a servlet to serve full-file content
|
|
@@ -688,7 +619,7 @@ public class DataNode extends Configured
|
|
|
// find free port or use privileged port provided
|
|
|
ServerSocket ss;
|
|
|
if(secureResources == null) {
|
|
|
- ss = (socketWriteTimeout > 0) ?
|
|
|
+ ss = (dnConf.socketWriteTimeout > 0) ?
|
|
|
ServerSocketChannel.open().socket() : new ServerSocket();
|
|
|
Server.bind(ss, socAddr, 0);
|
|
|
} else {
|
|
@@ -760,11 +691,13 @@ public class DataNode extends Configured
|
|
|
private volatile boolean shouldServiceRun = true;
|
|
|
UpgradeManagerDatanode upgradeManager = null;
|
|
|
private final DataNode dn;
|
|
|
+ private final DNConf dnConf;
|
|
|
|
|
|
BPOfferService(InetSocketAddress nnAddr, DataNode dn) {
|
|
|
this.dn = dn;
|
|
|
this.bpRegistration = dn.createRegistration();
|
|
|
this.nnAddr = nnAddr;
|
|
|
+ this.dnConf = dn.getDnConf();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -866,9 +799,9 @@ public class DataNode extends Configured
|
|
|
void scheduleBlockReport(long delay) {
|
|
|
if (delay > 0) { // send BR after random delay
|
|
|
lastBlockReport = System.currentTimeMillis()
|
|
|
- - ( dn.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
|
|
|
+ - ( dnConf.blockReportInterval - DFSUtil.getRandom().nextInt((int)(delay)));
|
|
|
} else { // send at next heartbeat
|
|
|
- lastBlockReport = lastHeartbeat - dn.blockReportInterval;
|
|
|
+ lastBlockReport = lastHeartbeat - dnConf.blockReportInterval;
|
|
|
}
|
|
|
resetBlockReportTime = true; // reset future BRs for randomness
|
|
|
}
|
|
@@ -965,7 +898,7 @@ public class DataNode extends Configured
|
|
|
// send block report if timer has expired.
|
|
|
DatanodeCommand cmd = null;
|
|
|
long startTime = now();
|
|
|
- if (startTime - lastBlockReport > dn.blockReportInterval) {
|
|
|
+ if (startTime - lastBlockReport > dnConf.blockReportInterval) {
|
|
|
|
|
|
// Create block report
|
|
|
long brCreateStartTime = now();
|
|
@@ -987,7 +920,7 @@ public class DataNode extends Configured
|
|
|
// If we have sent the first block report, then wait a random
|
|
|
// time before we start the periodic block reports.
|
|
|
if (resetBlockReportTime) {
|
|
|
- lastBlockReport = startTime - DFSUtil.getRandom().nextInt((int)(dn.blockReportInterval));
|
|
|
+ lastBlockReport = startTime - DFSUtil.getRandom().nextInt((int)(dnConf.blockReportInterval));
|
|
|
resetBlockReportTime = false;
|
|
|
} else {
|
|
|
/* say the last block report was at 8:20:14. The current report
|
|
@@ -997,7 +930,7 @@ public class DataNode extends Configured
|
|
|
* 2) unexpected like 11:35:43, next report should be at 12:20:14
|
|
|
*/
|
|
|
lastBlockReport += (now() - lastBlockReport) /
|
|
|
- dn.blockReportInterval * dn.blockReportInterval;
|
|
|
+ dnConf.blockReportInterval * dnConf.blockReportInterval;
|
|
|
}
|
|
|
LOG.info("sent block report, processed command:" + cmd);
|
|
|
}
|
|
@@ -1059,9 +992,9 @@ public class DataNode extends Configured
|
|
|
*/
|
|
|
private void offerService() throws Exception {
|
|
|
LOG.info("For namenode " + nnAddr + " using BLOCKREPORT_INTERVAL of "
|
|
|
- + dn.blockReportInterval + "msec" + " Initial delay: "
|
|
|
- + dn.initialBlockReportDelay + "msec" + "; heartBeatInterval="
|
|
|
- + dn.heartBeatInterval);
|
|
|
+ + dnConf.blockReportInterval + "msec" + " Initial delay: "
|
|
|
+ + dnConf.initialBlockReportDelay + "msec" + "; heartBeatInterval="
|
|
|
+ + dnConf.heartBeatInterval);
|
|
|
|
|
|
//
|
|
|
// Now loop for a long time....
|
|
@@ -1073,7 +1006,7 @@ public class DataNode extends Configured
|
|
|
//
|
|
|
// Every so often, send heartbeat or block-report
|
|
|
//
|
|
|
- if (startTime - lastHeartbeat > dn.heartBeatInterval) {
|
|
|
+ if (startTime - lastHeartbeat > dnConf.heartBeatInterval) {
|
|
|
//
|
|
|
// All heartbeat messages include following info:
|
|
|
// -- Datanode name
|
|
@@ -1111,7 +1044,7 @@ public class DataNode extends Configured
|
|
|
// There is no work to do; sleep until hearbeat timer elapses,
|
|
|
// or work arrives, and then iterate again.
|
|
|
//
|
|
|
- long waitTime = dn.heartBeatInterval -
|
|
|
+ long waitTime = dnConf.heartBeatInterval -
|
|
|
(System.currentTimeMillis() - lastHeartbeat);
|
|
|
synchronized(receivedBlockList) {
|
|
|
if (waitTime > 0 && receivedBlockList.size() == 0) {
|
|
@@ -1134,7 +1067,7 @@ public class DataNode extends Configured
|
|
|
}
|
|
|
LOG.warn("RemoteException in offerService", re);
|
|
|
try {
|
|
|
- long sleepTime = Math.min(1000, dn.heartBeatInterval);
|
|
|
+ long sleepTime = Math.min(1000, dnConf.heartBeatInterval);
|
|
|
Thread.sleep(sleepTime);
|
|
|
} catch (InterruptedException ie) {
|
|
|
Thread.currentThread().interrupt();
|
|
@@ -1202,7 +1135,7 @@ public class DataNode extends Configured
|
|
|
LOG.info("in register:" + ";bpDNR="+bpRegistration.storageInfo);
|
|
|
|
|
|
// random short delay - helps scatter the BR from all DNs
|
|
|
- scheduleBlockReport(dn.initialBlockReportDelay);
|
|
|
+ scheduleBlockReport(dnConf.initialBlockReportDelay);
|
|
|
}
|
|
|
|
|
|
|
|
@@ -1412,11 +1345,11 @@ public class DataNode extends Configured
|
|
|
this.secureResources = resources;
|
|
|
this.dataDirs = dataDirs;
|
|
|
this.conf = conf;
|
|
|
+ this.dnConf = new DNConf(conf);
|
|
|
|
|
|
storage = new DataStorage();
|
|
|
|
|
|
// global DN settings
|
|
|
- initConfig(conf);
|
|
|
registerMXBean();
|
|
|
initDataXceiver(conf);
|
|
|
startInfoServer(conf);
|
|
@@ -1664,7 +1597,7 @@ public class DataNode extends Configured
|
|
|
* Creates either NIO or regular depending on socketWriteTimeout.
|
|
|
*/
|
|
|
protected Socket newSocket() throws IOException {
|
|
|
- return (socketWriteTimeout > 0) ?
|
|
|
+ return (dnConf.socketWriteTimeout > 0) ?
|
|
|
SocketChannel.open().socket() : new Socket();
|
|
|
}
|
|
|
|
|
@@ -2091,10 +2024,10 @@ public class DataNode extends Configured
|
|
|
InetSocketAddress curTarget =
|
|
|
NetUtils.createSocketAddr(targets[0].getName());
|
|
|
sock = newSocket();
|
|
|
- NetUtils.connect(sock, curTarget, socketTimeout);
|
|
|
- sock.setSoTimeout(targets.length * socketTimeout);
|
|
|
+ NetUtils.connect(sock, curTarget, dnConf.socketTimeout);
|
|
|
+ sock.setSoTimeout(targets.length * dnConf.socketTimeout);
|
|
|
|
|
|
- long writeTimeout = socketWriteTimeout +
|
|
|
+ long writeTimeout = dnConf.socketWriteTimeout +
|
|
|
HdfsServerConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1);
|
|
|
OutputStream baseStream = NetUtils.getOutputStream(sock, writeTimeout);
|
|
|
out = new DataOutputStream(new BufferedOutputStream(baseStream,
|
|
@@ -2537,7 +2470,7 @@ public class DataNode extends Configured
|
|
|
DatanodeRegistration bpReg = bpos.bpRegistration;
|
|
|
InterDatanodeProtocol datanode = bpReg.equals(id)?
|
|
|
this: DataNode.createInterDataNodeProtocolProxy(id, getConf(),
|
|
|
- socketTimeout);
|
|
|
+ dnConf.socketTimeout);
|
|
|
ReplicaRecoveryInfo info = callInitReplicaRecovery(datanode, rBlock);
|
|
|
if (info != null &&
|
|
|
info.getGenerationStamp() >= block.getGenerationStamp() &&
|
|
@@ -2926,20 +2859,8 @@ public class DataNode extends Configured
|
|
|
(DataXceiverServer) this.dataXceiverServer.getRunnable();
|
|
|
return dxcs.balanceThrottler.getBandwidth();
|
|
|
}
|
|
|
-
|
|
|
- long getReadaheadLength() {
|
|
|
- return readaheadLength;
|
|
|
- }
|
|
|
-
|
|
|
- boolean shouldDropCacheBehindWrites() {
|
|
|
- return dropCacheBehindWrites;
|
|
|
- }
|
|
|
-
|
|
|
- boolean shouldDropCacheBehindReads() {
|
|
|
- return dropCacheBehindReads;
|
|
|
- }
|
|
|
|
|
|
- boolean shouldSyncBehindWrites() {
|
|
|
- return syncBehindWrites;
|
|
|
+ DNConf getDnConf() {
|
|
|
+ return dnConf;
|
|
|
}
|
|
|
}
|