|
@@ -31,7 +31,6 @@ import java.io.PrintWriter;
|
|
|
import java.lang.management.ManagementFactory;
|
|
|
import java.net.InetAddress;
|
|
|
import java.net.URI;
|
|
|
-import java.net.UnknownHostException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
@@ -102,6 +101,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
|
|
|
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
|
|
|
import org.apache.hadoop.hdfs.server.blockmanagement.UnderReplicatedBlocks;
|
|
|
import org.apache.hadoop.hdfs.server.common.GenerationStamp;
|
|
|
import org.apache.hadoop.hdfs.server.common.HdfsConstants.BlockUCState;
|
|
@@ -133,11 +133,8 @@ import org.apache.hadoop.metrics2.annotation.Metrics;
|
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
|
import org.apache.hadoop.metrics2.lib.MutableCounterInt;
|
|
|
import org.apache.hadoop.metrics2.util.MBeans;
|
|
|
-import org.apache.hadoop.net.CachedDNSToSwitchMapping;
|
|
|
-import org.apache.hadoop.net.DNSToSwitchMapping;
|
|
|
import org.apache.hadoop.net.NetworkTopology;
|
|
|
import org.apache.hadoop.net.Node;
|
|
|
-import org.apache.hadoop.net.ScriptBasedMapping;
|
|
|
import org.apache.hadoop.security.AccessControlException;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.UserGroupInformation.AuthenticationMethod;
|
|
@@ -145,8 +142,6 @@ import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.security.token.delegation.DelegationKey;
|
|
|
import org.apache.hadoop.util.Daemon;
|
|
|
-import org.apache.hadoop.util.HostsFileReader;
|
|
|
-import org.apache.hadoop.util.ReflectionUtils;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
|
import org.apache.hadoop.util.VersionInfo;
|
|
|
import org.mortbay.util.ajax.JSON;
|
|
@@ -313,10 +308,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
ReplaceDatanodeOnFailure.DEFAULT;
|
|
|
|
|
|
private volatile SafeModeInfo safeMode; // safe mode information
|
|
|
-
|
|
|
- private DNSToSwitchMapping dnsToSwitchMapping;
|
|
|
-
|
|
|
- private HostsFileReader hostsReader;
|
|
|
|
|
|
private long maxFsObjects = 0; // maximum number of fs objects
|
|
|
|
|
@@ -376,9 +367,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
this.dir = new FSDirectory(fsImage, this, conf);
|
|
|
}
|
|
|
this.safeMode = new SafeModeInfo(conf);
|
|
|
- this.hostsReader = new HostsFileReader(
|
|
|
- conf.get(DFSConfigKeys.DFS_HOSTS,""),
|
|
|
- conf.get(DFSConfigKeys.DFS_HOSTS_EXCLUDE,""));
|
|
|
if (isBlockTokenEnabled) {
|
|
|
blockTokenSecretManager = new BlockTokenSecretManager(true,
|
|
|
blockKeyUpdateInterval, blockTokenLifetime);
|
|
@@ -407,19 +395,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
this.nnrmthread = new Daemon(new NameNodeResourceMonitor());
|
|
|
nnrmthread.start();
|
|
|
|
|
|
- this.dnsToSwitchMapping = ReflectionUtils.newInstance(
|
|
|
- conf.getClass(DFSConfigKeys.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
|
|
|
- ScriptBasedMapping.class,
|
|
|
- DNSToSwitchMapping.class), conf);
|
|
|
-
|
|
|
- /* If the dns to switch mapping supports cache, resolve network
|
|
|
- * locations of those hosts in the include list,
|
|
|
- * and store the mapping in the cache; so future calls to resolve
|
|
|
- * will be fast.
|
|
|
- */
|
|
|
- if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
|
|
|
- dnsToSwitchMapping.resolve(new ArrayList<String>(hostsReader.getHosts()));
|
|
|
- }
|
|
|
registerMXBean();
|
|
|
DefaultMetricsSystem.instance().register(this);
|
|
|
}
|
|
@@ -768,7 +743,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
*
|
|
|
* @return current access keys
|
|
|
*/
|
|
|
- ExportedBlockKeys getBlockKeys() {
|
|
|
+ public ExportedBlockKeys getBlockKeys() {
|
|
|
return isBlockTokenEnabled ? blockTokenSecretManager.exportKeys()
|
|
|
: ExportedBlockKeys.DUMMY_KEYS;
|
|
|
}
|
|
@@ -1836,8 +1811,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
// choose new datanodes.
|
|
|
- final DatanodeInfo[] targets = blockManager.replicator.chooseTarget(
|
|
|
- src, numAdditionalNodes, clientnode, chosen, true,
|
|
|
+ final DatanodeInfo[] targets = blockManager.getBlockPlacementPolicy(
|
|
|
+ ).chooseTarget(src, numAdditionalNodes, clientnode, chosen, true,
|
|
|
excludes, preferredblocksize);
|
|
|
final LocatedBlock lb = new LocatedBlock(blk, targets);
|
|
|
if (isBlockTokenEnabled) {
|
|
@@ -2780,162 +2755,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
- registerDatanodeInternal(nodeReg);
|
|
|
+ getBlockManager().getDatanodeManager().registerDatanode(nodeReg);
|
|
|
+ checkSafeMode();
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- /** @see #registerDatanode(DatanodeRegistration) */
|
|
|
- public void registerDatanodeInternal(DatanodeRegistration nodeReg)
|
|
|
- throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
- String dnAddress = Server.getRemoteAddress();
|
|
|
- if (dnAddress == null) {
|
|
|
- // Mostly called inside an RPC.
|
|
|
- // But if not, use address passed by the data-node.
|
|
|
- dnAddress = nodeReg.getHost();
|
|
|
- }
|
|
|
-
|
|
|
- // check if the datanode is allowed to be connect to the namenode
|
|
|
- if (!verifyNodeRegistration(nodeReg, dnAddress)) {
|
|
|
- throw new DisallowedDatanodeException(nodeReg);
|
|
|
- }
|
|
|
-
|
|
|
- String hostName = nodeReg.getHost();
|
|
|
-
|
|
|
- // update the datanode's name with ip:port
|
|
|
- DatanodeID dnReg = new DatanodeID(dnAddress + ":" + nodeReg.getPort(),
|
|
|
- nodeReg.getStorageID(),
|
|
|
- nodeReg.getInfoPort(),
|
|
|
- nodeReg.getIpcPort());
|
|
|
- nodeReg.updateRegInfo(dnReg);
|
|
|
- nodeReg.exportedKeys = getBlockKeys();
|
|
|
-
|
|
|
- NameNode.stateChangeLog.info(
|
|
|
- "BLOCK* NameSystem.registerDatanode: "
|
|
|
- + "node registration from " + nodeReg.getName()
|
|
|
- + " storage " + nodeReg.getStorageID());
|
|
|
-
|
|
|
- DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
|
|
|
- DatanodeDescriptor nodeN =
|
|
|
- blockManager.getDatanodeManager().getDatanodeByHost(nodeReg.getName());
|
|
|
-
|
|
|
- if (nodeN != null && nodeN != nodeS) {
|
|
|
- NameNode.LOG.info("BLOCK* NameSystem.registerDatanode: "
|
|
|
- + "node from name: " + nodeN.getName());
|
|
|
- // nodeN previously served a different data storage,
|
|
|
- // which is not served by anybody anymore.
|
|
|
- removeDatanode(nodeN);
|
|
|
- // physically remove node from datanodeMap
|
|
|
- blockManager.getDatanodeManager().wipeDatanode(nodeN);
|
|
|
- nodeN = null;
|
|
|
- }
|
|
|
-
|
|
|
- if (nodeS != null) {
|
|
|
- if (nodeN == nodeS) {
|
|
|
- // The same datanode has been just restarted to serve the same data
|
|
|
- // storage. We do not need to remove old data blocks, the delta will
|
|
|
- // be calculated on the next block report from the datanode
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* NameSystem.registerDatanode: "
|
|
|
- + "node restarted.");
|
|
|
- }
|
|
|
- } else {
|
|
|
- // nodeS is found
|
|
|
- /* The registering datanode is a replacement node for the existing
|
|
|
- data storage, which from now on will be served by a new node.
|
|
|
- If this message repeats, both nodes might have same storageID
|
|
|
- by (insanely rare) random chance. User needs to restart one of the
|
|
|
- nodes with its data cleared (or user can just remove the StorageID
|
|
|
- value in "VERSION" file under the data directory of the datanode,
|
|
|
- but this is might not work if VERSION file format has changed
|
|
|
- */
|
|
|
- NameNode.stateChangeLog.info( "BLOCK* NameSystem.registerDatanode: "
|
|
|
- + "node " + nodeS.getName()
|
|
|
- + " is replaced by " + nodeReg.getName() +
|
|
|
- " with the same storageID " +
|
|
|
- nodeReg.getStorageID());
|
|
|
- }
|
|
|
- // update cluster map
|
|
|
- blockManager.getDatanodeManager().getNetworkTopology().remove(nodeS);
|
|
|
- nodeS.updateRegInfo(nodeReg);
|
|
|
- nodeS.setHostName(hostName);
|
|
|
- nodeS.setDisallowed(false); // Node is in the include list
|
|
|
-
|
|
|
- // resolve network location
|
|
|
- resolveNetworkLocation(nodeS);
|
|
|
- blockManager.getDatanodeManager().getNetworkTopology().add(nodeS);
|
|
|
-
|
|
|
- // also treat the registration message as a heartbeat
|
|
|
- synchronized(heartbeats) {
|
|
|
- if( !heartbeats.contains(nodeS)) {
|
|
|
- heartbeats.add(nodeS);
|
|
|
- //update its timestamp
|
|
|
- nodeS.updateHeartbeat(0L, 0L, 0L, 0L, 0, 0);
|
|
|
- nodeS.isAlive = true;
|
|
|
- }
|
|
|
- }
|
|
|
- checkDecommissioning(nodeS, dnAddress);
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- // this is a new datanode serving a new data storage
|
|
|
- if (nodeReg.getStorageID().equals("")) {
|
|
|
- // this data storage has never been registered
|
|
|
- // it is either empty or was created by pre-storageID version of DFS
|
|
|
- nodeReg.storageID = newStorageID();
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug(
|
|
|
- "BLOCK* NameSystem.registerDatanode: "
|
|
|
- + "new storageID " + nodeReg.getStorageID() + " assigned.");
|
|
|
- }
|
|
|
- }
|
|
|
- // register new datanode
|
|
|
- DatanodeDescriptor nodeDescr
|
|
|
- = new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK, hostName);
|
|
|
- resolveNetworkLocation(nodeDescr);
|
|
|
- blockManager.getDatanodeManager().addDatanode(nodeDescr);
|
|
|
- checkDecommissioning(nodeDescr, dnAddress);
|
|
|
-
|
|
|
- // also treat the registration message as a heartbeat
|
|
|
- synchronized(heartbeats) {
|
|
|
- heartbeats.add(nodeDescr);
|
|
|
- nodeDescr.isAlive = true;
|
|
|
- // no need to update its timestamp
|
|
|
- // because its is done when the descriptor is created
|
|
|
- }
|
|
|
-
|
|
|
- checkSafeMode();
|
|
|
- }
|
|
|
-
|
|
|
- /* Resolve a node's network location */
|
|
|
- private void resolveNetworkLocation (DatanodeDescriptor node) {
|
|
|
- assert hasWriteLock();
|
|
|
- List<String> names = new ArrayList<String>(1);
|
|
|
- if (dnsToSwitchMapping instanceof CachedDNSToSwitchMapping) {
|
|
|
- // get the node's IP address
|
|
|
- names.add(node.getHost());
|
|
|
- } else {
|
|
|
- // get the node's host name
|
|
|
- String hostName = node.getHostName();
|
|
|
- int colon = hostName.indexOf(":");
|
|
|
- hostName = (colon==-1)?hostName:hostName.substring(0,colon);
|
|
|
- names.add(hostName);
|
|
|
- }
|
|
|
-
|
|
|
- // resolve its network location
|
|
|
- List<String> rName = dnsToSwitchMapping.resolve(names);
|
|
|
- String networkLocation;
|
|
|
- if (rName == null) {
|
|
|
- LOG.error("The resolve call returned null! Using " +
|
|
|
- NetworkTopology.DEFAULT_RACK + " for host " + names);
|
|
|
- networkLocation = NetworkTopology.DEFAULT_RACK;
|
|
|
- } else {
|
|
|
- networkLocation = rName.get(0);
|
|
|
- }
|
|
|
- node.setNetworkLocation(networkLocation);
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Get registrationID for datanodes based on the namespaceID.
|
|
@@ -2946,26 +2771,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
public String getRegistrationID() {
|
|
|
return Storage.getRegistrationID(dir.fsImage.getStorage());
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Generate new storage ID.
|
|
|
- *
|
|
|
- * @return unique storage ID
|
|
|
- *
|
|
|
- * Note: that collisions are still possible if somebody will try
|
|
|
- * to bring in a data storage from a different cluster.
|
|
|
- */
|
|
|
- private String newStorageID() {
|
|
|
- String newID = null;
|
|
|
- while(newID == null) {
|
|
|
- newID = "DS" + Integer.toString(DFSUtil.getRandom().nextInt());
|
|
|
- if (datanodeMap.get(newID) != null)
|
|
|
- newID = null;
|
|
|
- }
|
|
|
- return newID;
|
|
|
- }
|
|
|
-
|
|
|
- private boolean isDatanodeDead(DatanodeDescriptor node) {
|
|
|
+
|
|
|
+ public boolean isDatanodeDead(DatanodeDescriptor node) {
|
|
|
return (node.getLastUpdate() <
|
|
|
(now() - heartbeatExpireInterval));
|
|
|
}
|
|
@@ -3078,7 +2885,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
- private void updateStats(DatanodeDescriptor node, boolean isAdded) {
|
|
|
+ public void updateStats(DatanodeDescriptor node, boolean isAdded) {
|
|
|
//
|
|
|
// The statistics are protected by the heartbeat lock
|
|
|
// For decommissioning/decommissioned nodes, only used capacity
|
|
@@ -3280,10 +3087,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
/**
|
|
|
* Remove a datanode descriptor.
|
|
|
* @param nodeID datanode ID.
|
|
|
- * @throws IOException
|
|
|
+ * @throws UnregisteredNodeException
|
|
|
*/
|
|
|
- public void removeDatanode(DatanodeID nodeID)
|
|
|
- throws IOException {
|
|
|
+ public void removeDatanode(final DatanodeID nodeID
|
|
|
+ ) throws UnregisteredNodeException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
DatanodeDescriptor nodeInfo = getDatanode(nodeID);
|
|
@@ -3664,83 +3471,23 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
int getNumberOfDatanodes(DatanodeReportType type) {
|
|
|
- return getDatanodeListForReport(type).size();
|
|
|
- }
|
|
|
-
|
|
|
- private ArrayList<DatanodeDescriptor> getDatanodeListForReport(
|
|
|
- DatanodeReportType type) {
|
|
|
readLock();
|
|
|
- try {
|
|
|
- boolean listLiveNodes = type == DatanodeReportType.ALL ||
|
|
|
- type == DatanodeReportType.LIVE;
|
|
|
- boolean listDeadNodes = type == DatanodeReportType.ALL ||
|
|
|
- type == DatanodeReportType.DEAD;
|
|
|
-
|
|
|
- HashMap<String, String> mustList = new HashMap<String, String>();
|
|
|
-
|
|
|
- if (listDeadNodes) {
|
|
|
- //first load all the nodes listed in include and exclude files.
|
|
|
- Iterator<String> it = hostsReader.getHosts().iterator();
|
|
|
- while (it.hasNext()) {
|
|
|
- mustList.put(it.next(), "");
|
|
|
- }
|
|
|
- it = hostsReader.getExcludedHosts().iterator();
|
|
|
- while (it.hasNext()) {
|
|
|
- mustList.put(it.next(), "");
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- ArrayList<DatanodeDescriptor> nodes = null;
|
|
|
-
|
|
|
- synchronized (datanodeMap) {
|
|
|
- nodes = new ArrayList<DatanodeDescriptor>(datanodeMap.size() +
|
|
|
- mustList.size());
|
|
|
- Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator();
|
|
|
- while (it.hasNext()) {
|
|
|
- DatanodeDescriptor dn = it.next();
|
|
|
- boolean isDead = isDatanodeDead(dn);
|
|
|
- if ( (isDead && listDeadNodes) || (!isDead && listLiveNodes) ) {
|
|
|
- nodes.add(dn);
|
|
|
- }
|
|
|
- //Remove any form of the this datanode in include/exclude lists.
|
|
|
- try {
|
|
|
- InetAddress inet = InetAddress.getByName(dn.getHost());
|
|
|
- // compare hostname(:port)
|
|
|
- mustList.remove(inet.getHostName());
|
|
|
- mustList.remove(inet.getHostName()+":"+dn.getPort());
|
|
|
- // compare ipaddress(:port)
|
|
|
- mustList.remove(inet.getHostAddress().toString());
|
|
|
- mustList.remove(inet.getHostAddress().toString()+ ":" +dn.getPort());
|
|
|
- } catch ( UnknownHostException e ) {
|
|
|
- mustList.remove(dn.getName());
|
|
|
- mustList.remove(dn.getHost());
|
|
|
- LOG.warn(e);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- if (listDeadNodes) {
|
|
|
- Iterator<String> it = mustList.keySet().iterator();
|
|
|
- while (it.hasNext()) {
|
|
|
- DatanodeDescriptor dn =
|
|
|
- new DatanodeDescriptor(new DatanodeID(it.next()));
|
|
|
- dn.setLastUpdate(0);
|
|
|
- nodes.add(dn);
|
|
|
- }
|
|
|
- }
|
|
|
- return nodes;
|
|
|
+ try {
|
|
|
+ return getBlockManager().getDatanodeManager().getDatanodeListForReport(
|
|
|
+ type).size();
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- public DatanodeInfo[] datanodeReport( DatanodeReportType type)
|
|
|
- throws AccessControlException {
|
|
|
+ DatanodeInfo[] datanodeReport(final DatanodeReportType type
|
|
|
+ ) throws AccessControlException {
|
|
|
+ checkSuperuserPrivilege();
|
|
|
readLock();
|
|
|
try {
|
|
|
- checkSuperuserPrivilege();
|
|
|
-
|
|
|
- ArrayList<DatanodeDescriptor> results = getDatanodeListForReport(type);
|
|
|
+ final DatanodeManager dm = getBlockManager().getDatanodeManager();
|
|
|
+ final List<DatanodeDescriptor> results = dm.getDatanodeListForReport(type);
|
|
|
+
|
|
|
DatanodeInfo[] arr = new DatanodeInfo[results.size()];
|
|
|
for (int i=0; i<arr.length; i++) {
|
|
|
arr[i] = new DatanodeInfo(results.get(i));
|
|
@@ -3804,8 +3551,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
ArrayList<DatanodeDescriptor> dead) {
|
|
|
readLock();
|
|
|
try {
|
|
|
- ArrayList<DatanodeDescriptor> results =
|
|
|
- getDatanodeListForReport(DatanodeReportType.ALL);
|
|
|
+ final List<DatanodeDescriptor> results = getBlockManager(
|
|
|
+ ).getDatanodeManager().getDatanodeListForReport(DatanodeReportType.ALL);
|
|
|
for(Iterator<DatanodeDescriptor> it = results.iterator(); it.hasNext();) {
|
|
|
DatanodeDescriptor node = it.next();
|
|
|
if (isDatanodeDead(node))
|
|
@@ -3836,44 +3583,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Start decommissioning the specified datanode.
|
|
|
- */
|
|
|
- private void startDecommission(DatanodeDescriptor node)
|
|
|
- throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
- if (!node.isDecommissionInProgress() && !node.isDecommissioned()) {
|
|
|
- LOG.info("Start Decommissioning node " + node.getName() + " with " +
|
|
|
- node.numBlocks() + " blocks.");
|
|
|
- synchronized (heartbeats) {
|
|
|
- updateStats(node, false);
|
|
|
- node.startDecommission();
|
|
|
- updateStats(node, true);
|
|
|
- }
|
|
|
- node.decommissioningStatus.setStartTime(now());
|
|
|
-
|
|
|
- // all the blocks that reside on this node have to be replicated.
|
|
|
- checkDecommissionStateInternal(node);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Stop decommissioning the specified datanodes.
|
|
|
- */
|
|
|
- public void stopDecommission(DatanodeDescriptor node)
|
|
|
- throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
- if (node.isDecommissionInProgress() || node.isDecommissioned()) {
|
|
|
- LOG.info("Stop Decommissioning node " + node.getName());
|
|
|
- synchronized (heartbeats) {
|
|
|
- updateStats(node, false);
|
|
|
- node.stopDecommission();
|
|
|
- updateStats(node, true);
|
|
|
- }
|
|
|
- blockManager.processOverReplicatedBlocksOnReCommission(node);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
public Date getStartTime() {
|
|
|
return new Date(systemStart);
|
|
|
}
|
|
@@ -3898,85 +3607,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return replication;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Change, if appropriate, the admin state of a datanode to
|
|
|
- * decommission completed. Return true if decommission is complete.
|
|
|
- */
|
|
|
- public boolean checkDecommissionStateInternal(DatanodeDescriptor node) {
|
|
|
- assert hasWriteLock();
|
|
|
- //
|
|
|
- // Check to see if all blocks in this decommissioned
|
|
|
- // node has reached their target replication factor.
|
|
|
- //
|
|
|
- if (node.isDecommissionInProgress()) {
|
|
|
- if (!blockManager.isReplicationInProgress(node)) {
|
|
|
- node.setDecommissioned();
|
|
|
- LOG.info("Decommission complete for node " + node.getName());
|
|
|
- }
|
|
|
- }
|
|
|
- return node.isDecommissioned();
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Keeps track of which datanodes/ipaddress are allowed to connect to the namenode.
|
|
|
- */
|
|
|
- private boolean inHostsList(DatanodeID node, String ipAddr) {
|
|
|
- Set<String> hostsList = hostsReader.getHosts();
|
|
|
- return checkInList(node, ipAddr, hostsList, false);
|
|
|
- }
|
|
|
-
|
|
|
- private boolean inExcludedHostsList(DatanodeID node, String ipAddr) {
|
|
|
- Set<String> excludeList = hostsReader.getExcludedHosts();
|
|
|
- return checkInList(node, ipAddr, excludeList, true);
|
|
|
- }
|
|
|
-
|
|
|
-
|
|
|
- /**
|
|
|
- * Check if the given node (of DatanodeID or ipAddress) is in the (include or
|
|
|
- * exclude) list. If ipAddress in null, check only based upon the given
|
|
|
- * DatanodeID. If ipAddress is not null, the ipAddress should refers to the
|
|
|
- * same host that given DatanodeID refers to.
|
|
|
- *
|
|
|
- * @param node, DatanodeID, the host DatanodeID
|
|
|
- * @param ipAddress, if not null, should refers to the same host
|
|
|
- * that DatanodeID refers to
|
|
|
- * @param hostsList, the list of hosts in the include/exclude file
|
|
|
- * @param isExcludeList, boolean, true if this is the exclude list
|
|
|
- * @return boolean, if in the list
|
|
|
- */
|
|
|
- private boolean checkInList(DatanodeID node, String ipAddress,
|
|
|
- Set<String> hostsList, boolean isExcludeList) {
|
|
|
- InetAddress iaddr = null;
|
|
|
- try {
|
|
|
- if (ipAddress != null) {
|
|
|
- iaddr = InetAddress.getByName(ipAddress);
|
|
|
- } else {
|
|
|
- iaddr = InetAddress.getByName(node.getHost());
|
|
|
- }
|
|
|
- }catch (UnknownHostException e) {
|
|
|
- LOG.warn("Unknown host in host list: "+ipAddress);
|
|
|
- // can't resolve the host name.
|
|
|
- if (isExcludeList){
|
|
|
- return true;
|
|
|
- } else {
|
|
|
- return false;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- // if include list is empty, host is in include list
|
|
|
- if ( (!isExcludeList) && (hostsList.isEmpty()) ){
|
|
|
- return true;
|
|
|
- }
|
|
|
- return // compare ipaddress(:port)
|
|
|
- (hostsList.contains(iaddr.getHostAddress().toString()))
|
|
|
- || (hostsList.contains(iaddr.getHostAddress().toString() + ":"
|
|
|
- + node.getPort()))
|
|
|
- // compare hostname(:port)
|
|
|
- || (hostsList.contains(iaddr.getHostName()))
|
|
|
- || (hostsList.contains(iaddr.getHostName() + ":" + node.getPort()))
|
|
|
- || ((node instanceof DatanodeInfo) && hostsList
|
|
|
- .contains(((DatanodeInfo) node).getHostName()));
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Rereads the config to get hosts and exclude list file names.
|
|
@@ -3989,29 +3619,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
*/
|
|
|
public void refreshNodes(Configuration conf) throws IOException {
|
|
|
checkSuperuserPrivilege();
|
|
|
- // Reread the config to get dfs.hosts and dfs.hosts.exclude filenames.
|
|
|
- // Update the file names and refresh internal includes and excludes list
|
|
|
- if (conf == null)
|
|
|
- conf = new HdfsConfiguration();
|
|
|
- hostsReader.updateFileNames(conf.get(DFSConfigKeys.DFS_HOSTS,""),
|
|
|
- conf.get(DFSConfigKeys.DFS_HOSTS_EXCLUDE, ""));
|
|
|
- hostsReader.refresh();
|
|
|
+ getBlockManager().getDatanodeManager().refreshHostsReader(conf);
|
|
|
writeLock();
|
|
|
try {
|
|
|
- for (Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator();
|
|
|
- it.hasNext();) {
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
- // Check if not include.
|
|
|
- if (!inHostsList(node, null)) {
|
|
|
- node.setDisallowed(true); // case 2.
|
|
|
- } else {
|
|
|
- if (inExcludedHostsList(node, null)) {
|
|
|
- startDecommission(node); // case 3.
|
|
|
- } else {
|
|
|
- stopDecommission(node); // case 4.
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ getBlockManager().getDatanodeManager().refreshDatanodes();
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -4021,27 +3632,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
checkSuperuserPrivilege();
|
|
|
getFSImage().finalizeUpgrade();
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Checks if the node is not on the hosts list. If it is not, then
|
|
|
- * it will be disallowed from registering.
|
|
|
- */
|
|
|
- private boolean verifyNodeRegistration(DatanodeID nodeReg, String ipAddr) {
|
|
|
- assert hasWriteLock();
|
|
|
- return inHostsList(nodeReg, ipAddr);
|
|
|
- }
|
|
|
|
|
|
- /**
|
|
|
- * Decommission the node if it is in exclude list.
|
|
|
- */
|
|
|
- private void checkDecommissioning(DatanodeDescriptor nodeReg, String ipAddr)
|
|
|
- throws IOException {
|
|
|
- assert hasWriteLock();
|
|
|
- // If the registered node is in exclude list, then decommission it
|
|
|
- if (inExcludedHostsList(nodeReg, ipAddr)) {
|
|
|
- startDecommission(nodeReg);
|
|
|
- }
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* Get data node by storage ID.
|
|
@@ -4050,7 +3641,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* @return DatanodeDescriptor or null if the node is not found.
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- public DatanodeDescriptor getDatanode(DatanodeID nodeID) throws IOException {
|
|
|
+ public DatanodeDescriptor getDatanode(DatanodeID nodeID
|
|
|
+ ) throws UnregisteredNodeException {
|
|
|
assert hasReadOrWriteLock();
|
|
|
UnregisteredNodeException e = null;
|
|
|
DatanodeDescriptor node = datanodeMap.get(nodeID.getStorageID());
|
|
@@ -5411,8 +5003,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
try {
|
|
|
ArrayList<DatanodeDescriptor> decommissioningNodes =
|
|
|
new ArrayList<DatanodeDescriptor>();
|
|
|
- ArrayList<DatanodeDescriptor> results =
|
|
|
- getDatanodeListForReport(DatanodeReportType.LIVE);
|
|
|
+ final List<DatanodeDescriptor> results = getBlockManager(
|
|
|
+ ).getDatanodeManager().getDatanodeListForReport(DatanodeReportType.LIVE);
|
|
|
for (Iterator<DatanodeDescriptor> it = results.iterator(); it.hasNext();) {
|
|
|
DatanodeDescriptor node = it.next();
|
|
|
if (node.isDecommissionInProgress()) {
|
|
@@ -5824,49 +5416,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
public BlockManager getBlockManager() {
|
|
|
return blockManager;
|
|
|
}
|
|
|
-
|
|
|
- /**
|
|
|
- * Remove an already decommissioned data node who is neither in include nor
|
|
|
- * exclude hosts lists from the the list of live or dead nodes. This is used
|
|
|
- * to not display an already decommssioned data node to the operators.
|
|
|
- * The operation procedure of making a already decommissioned data node not
|
|
|
- * to be displayed is as following:
|
|
|
- * <ol>
|
|
|
- * <li>
|
|
|
- * Host must have been in the include hosts list and the include hosts list
|
|
|
- * must not be empty.
|
|
|
- * </li>
|
|
|
- * <li>
|
|
|
- * Host is decommissioned by remaining in the include hosts list and added
|
|
|
- * into the exclude hosts list. Name node is updated with the new
|
|
|
- * information by issuing dfsadmin -refreshNodes command.
|
|
|
- * </li>
|
|
|
- * <li>
|
|
|
- * Host is removed from both include hosts and exclude hosts lists. Name
|
|
|
- * node is updated with the new informationby issuing dfsamin -refreshNodes
|
|
|
- * command.
|
|
|
- * <li>
|
|
|
- * </ol>
|
|
|
- *
|
|
|
- * @param nodeList
|
|
|
- * , array list of live or dead nodes.
|
|
|
- */
|
|
|
- void removeDecomNodeFromList(ArrayList<DatanodeDescriptor> nodeList) {
|
|
|
- // If the include list is empty, any nodes are welcomed and it does not
|
|
|
- // make sense to exclude any nodes from the cluster. Therefore, no remove.
|
|
|
- if (hostsReader.getHosts().isEmpty()) {
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- for (Iterator<DatanodeDescriptor> it = nodeList.iterator(); it.hasNext();) {
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
- if ((!inHostsList(node, null)) && (!inExcludedHostsList(node, null))
|
|
|
- && node.isDecommissioned()) {
|
|
|
- // Include list is not empty, an existing datanode does not appear
|
|
|
- // in both include or exclude lists and it has been decommissioned.
|
|
|
- // Remove it from the node list.
|
|
|
- it.remove();
|
|
|
- }
|
|
|
- }
|
|
|
+
|
|
|
+ void removeDecomNodeFromList(List<DatanodeDescriptor> nodeList) {
|
|
|
+ getBlockManager().getDatanodeManager().removeDecomNodeFromList(nodeList);
|
|
|
}
|
|
|
}
|