|
@@ -66,6 +66,7 @@ import org.apache.hadoop.ipc.Server;
|
|
|
import org.apache.hadoop.net.CachedDNSToSwitchMapping;
|
|
|
import org.apache.hadoop.net.DNSToSwitchMapping;
|
|
|
import org.apache.hadoop.net.NetworkTopology;
|
|
|
+import org.apache.hadoop.net.NetworkTopology.InvalidTopologyException;
|
|
|
import org.apache.hadoop.net.Node;
|
|
|
import org.apache.hadoop.net.NodeBase;
|
|
|
import org.apache.hadoop.net.ScriptBasedMapping;
|
|
@@ -431,8 +432,8 @@ public class DatanodeManager {
|
|
|
host2DatanodeMap.remove(datanodeMap.put(node.getStorageID(), node));
|
|
|
}
|
|
|
|
|
|
+ networktopology.add(node); // may throw InvalidTopologyException
|
|
|
host2DatanodeMap.add(node);
|
|
|
- networktopology.add(node);
|
|
|
checkIfClusterIsNowMultiRack(node);
|
|
|
|
|
|
if (LOG.isDebugEnabled()) {
|
|
@@ -647,92 +648,122 @@ public class DatanodeManager {
|
|
|
nodeReg.setIpAddr(ip);
|
|
|
nodeReg.setPeerHostName(hostname);
|
|
|
}
|
|
|
-
|
|
|
- nodeReg.setExportedKeys(blockManager.getBlockKeys());
|
|
|
-
|
|
|
- // Checks if the node is not on the hosts list. If it is not, then
|
|
|
- // it will be disallowed from registering.
|
|
|
- if (!inHostsList(nodeReg)) {
|
|
|
- throw new DisallowedDatanodeException(nodeReg);
|
|
|
- }
|
|
|
-
|
|
|
- NameNode.stateChangeLog.info("BLOCK* registerDatanode: from "
|
|
|
- + nodeReg + " storage " + nodeReg.getStorageID());
|
|
|
-
|
|
|
- DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
|
|
|
- DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
|
|
|
- nodeReg.getIpAddr(), nodeReg.getXferPort());
|
|
|
-
|
|
|
- if (nodeN != null && nodeN != nodeS) {
|
|
|
- NameNode.LOG.info("BLOCK* registerDatanode: " + nodeN);
|
|
|
- // nodeN previously served a different data storage,
|
|
|
- // which is not served by anybody anymore.
|
|
|
- removeDatanode(nodeN);
|
|
|
- // physically remove node from datanodeMap
|
|
|
- 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* registerDatanode: "
|
|
|
- + "node restarted.");
|
|
|
+
|
|
|
+ try {
|
|
|
+ nodeReg.setExportedKeys(blockManager.getBlockKeys());
|
|
|
+
|
|
|
+ // Checks if the node is not on the hosts list. If it is not, then
|
|
|
+ // it will be disallowed from registering.
|
|
|
+ if (!inHostsList(nodeReg)) {
|
|
|
+ throw new DisallowedDatanodeException(nodeReg);
|
|
|
+ }
|
|
|
+
|
|
|
+ NameNode.stateChangeLog.info("BLOCK* registerDatanode: from "
|
|
|
+ + nodeReg + " storage " + nodeReg.getStorageID());
|
|
|
+
|
|
|
+ DatanodeDescriptor nodeS = datanodeMap.get(nodeReg.getStorageID());
|
|
|
+ DatanodeDescriptor nodeN = host2DatanodeMap.getDatanodeByXferAddr(
|
|
|
+ nodeReg.getIpAddr(), nodeReg.getXferPort());
|
|
|
+
|
|
|
+ if (nodeN != null && nodeN != nodeS) {
|
|
|
+ NameNode.LOG.info("BLOCK* registerDatanode: " + nodeN);
|
|
|
+ // nodeN previously served a different data storage,
|
|
|
+ // which is not served by anybody anymore.
|
|
|
+ removeDatanode(nodeN);
|
|
|
+ // physically remove node from datanodeMap
|
|
|
+ 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* 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* registerDatanode: " + nodeS
|
|
|
+ + " is replaced by " + nodeReg + " with the same storageID "
|
|
|
+ + nodeReg.getStorageID());
|
|
|
+ }
|
|
|
+
|
|
|
+ boolean success = false;
|
|
|
+ try {
|
|
|
+ // update cluster map
|
|
|
+ getNetworkTopology().remove(nodeS);
|
|
|
+ nodeS.updateRegInfo(nodeReg);
|
|
|
+ nodeS.setDisallowed(false); // Node is in the include list
|
|
|
+
|
|
|
+ // resolve network location
|
|
|
+ resolveNetworkLocation(nodeS);
|
|
|
+ getNetworkTopology().add(nodeS);
|
|
|
+
|
|
|
+ // also treat the registration message as a heartbeat
|
|
|
+ heartbeatManager.register(nodeS);
|
|
|
+ checkDecommissioning(nodeS);
|
|
|
+ success = true;
|
|
|
+ } finally {
|
|
|
+ if (!success) {
|
|
|
+ removeDatanode(nodeS);
|
|
|
+ wipeDatanode(nodeS);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return;
|
|
|
+ }
|
|
|
+
|
|
|
+ // this is a new datanode serving a new data storage
|
|
|
+ if ("".equals(nodeReg.getStorageID())) {
|
|
|
+ // this data storage has never been registered
|
|
|
+ // it is either empty or was created by pre-storageID version of DFS
|
|
|
+ nodeReg.setStorageID(newStorageID());
|
|
|
+ if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
+ NameNode.stateChangeLog.debug(
|
|
|
+ "BLOCK* NameSystem.registerDatanode: "
|
|
|
+ + "new storageID " + nodeReg.getStorageID() + " assigned.");
|
|
|
}
|
|
|
- } 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* registerDatanode: " + nodeS
|
|
|
- + " is replaced by " + nodeReg + " with the same storageID "
|
|
|
- + nodeReg.getStorageID());
|
|
|
}
|
|
|
- // update cluster map
|
|
|
- getNetworkTopology().remove(nodeS);
|
|
|
- nodeS.updateRegInfo(nodeReg);
|
|
|
- nodeS.setDisallowed(false); // Node is in the include list
|
|
|
|
|
|
- // resolve network location
|
|
|
- resolveNetworkLocation(nodeS);
|
|
|
- getNetworkTopology().add(nodeS);
|
|
|
+ DatanodeDescriptor nodeDescr
|
|
|
+ = new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK);
|
|
|
+ boolean success = false;
|
|
|
+ try {
|
|
|
+ resolveNetworkLocation(nodeDescr);
|
|
|
+ networktopology.add(nodeDescr);
|
|
|
+
|
|
|
+ // register new datanode
|
|
|
+ addDatanode(nodeDescr);
|
|
|
+ checkDecommissioning(nodeDescr);
|
|
|
|
|
|
- // also treat the registration message as a heartbeat
|
|
|
- heartbeatManager.register(nodeS);
|
|
|
- checkDecommissioning(nodeS);
|
|
|
- return;
|
|
|
- }
|
|
|
-
|
|
|
- // this is a new datanode serving a new data storage
|
|
|
- if ("".equals(nodeReg.getStorageID())) {
|
|
|
- // this data storage has never been registered
|
|
|
- // it is either empty or was created by pre-storageID version of DFS
|
|
|
- nodeReg.setStorageID(newStorageID());
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug(
|
|
|
- "BLOCK* NameSystem.registerDatanode: "
|
|
|
- + "new storageID " + nodeReg.getStorageID() + " assigned.");
|
|
|
+ // also treat the registration message as a heartbeat
|
|
|
+ // no need to update its timestamp
|
|
|
+ // because its is done when the descriptor is created
|
|
|
+ heartbeatManager.addDatanode(nodeDescr);
|
|
|
+ success = true;
|
|
|
+ } finally {
|
|
|
+ if (!success) {
|
|
|
+ removeDatanode(nodeDescr);
|
|
|
+ wipeDatanode(nodeDescr);
|
|
|
+ }
|
|
|
}
|
|
|
+ } catch (InvalidTopologyException e) {
|
|
|
+ // If the network location is invalid, clear the cached mappings
|
|
|
+ // so that we have a chance to re-add this DataNode with the
|
|
|
+ // correct network location later.
|
|
|
+ dnsToSwitchMapping.reloadCachedMappings();
|
|
|
+ throw e;
|
|
|
}
|
|
|
- // register new datanode
|
|
|
- DatanodeDescriptor nodeDescr
|
|
|
- = new DatanodeDescriptor(nodeReg, NetworkTopology.DEFAULT_RACK);
|
|
|
- resolveNetworkLocation(nodeDescr);
|
|
|
- addDatanode(nodeDescr);
|
|
|
- checkDecommissioning(nodeDescr);
|
|
|
-
|
|
|
- // also treat the registration message as a heartbeat
|
|
|
- // no need to update its timestamp
|
|
|
- // because its is done when the descriptor is created
|
|
|
- heartbeatManager.addDatanode(nodeDescr);
|
|
|
}
|
|
|
|
|
|
/**
|