|
@@ -237,35 +237,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
|
|
|
// Block pool ID used by this namenode
|
|
|
String blockPoolId;
|
|
|
-
|
|
|
- /**
|
|
|
- * Stores the datanode -> block map.
|
|
|
- * <p>
|
|
|
- * Done by storing a set of {@link DatanodeDescriptor} objects, sorted by
|
|
|
- * storage id. In order to keep the storage map consistent it tracks
|
|
|
- * all storages ever registered with the namenode.
|
|
|
- * A descriptor corresponding to a specific storage id can be
|
|
|
- * <ul>
|
|
|
- * <li>added to the map if it is a new storage id;</li>
|
|
|
- * <li>updated with a new datanode started as a replacement for the old one
|
|
|
- * with the same storage id; and </li>
|
|
|
- * <li>removed if and only if an existing datanode is restarted to serve a
|
|
|
- * different storage id.</li>
|
|
|
- * </ul> <br>
|
|
|
- * The list of the {@link DatanodeDescriptor}s in the map is checkpointed
|
|
|
- * in the namespace image file. Only the {@link DatanodeInfo} part is
|
|
|
- * persistent, the list of blocks is restored from the datanode block
|
|
|
- * reports.
|
|
|
- * <p>
|
|
|
- * Mapping: StorageID -> DatanodeDescriptor
|
|
|
- */
|
|
|
- public final NavigableMap<String, DatanodeDescriptor> datanodeMap =
|
|
|
- new TreeMap<String, DatanodeDescriptor>();
|
|
|
|
|
|
/**
|
|
|
- * Stores a set of DatanodeDescriptor objects.
|
|
|
- * This is a subset of {@link #datanodeMap}, containing nodes that are
|
|
|
- * considered alive.
|
|
|
+ * Stores a subset of datanodeMap, containing nodes that are considered alive.
|
|
|
* The HeartbeatMonitor periodically checks for out-dated entries,
|
|
|
* and removes them from the list.
|
|
|
*/
|
|
@@ -289,9 +263,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
|
|
|
// heartbeatRecheckInterval is how often namenode checks for expired datanodes
|
|
|
private long heartbeatRecheckInterval;
|
|
|
- // heartbeatExpireInterval is how long namenode waits for datanode to report
|
|
|
- // heartbeat
|
|
|
- private long heartbeatExpireInterval;
|
|
|
|
|
|
//resourceRecheckInterval is how often namenode checks for the disk space availability
|
|
|
private long resourceRecheckInterval;
|
|
@@ -314,9 +285,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
*/
|
|
|
private final GenerationStamp generationStamp = new GenerationStamp();
|
|
|
|
|
|
- // Ask Datanode only up to this many blocks to delete.
|
|
|
- public int blockInvalidateLimit = DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_DEFAULT;
|
|
|
-
|
|
|
// precision of access times.
|
|
|
private long accessTimePrecision = 0;
|
|
|
|
|
@@ -513,14 +481,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
this.defaultPermission = PermissionStatus.createImmutable(
|
|
|
fsOwner.getShortUserName(), supergroup, new FsPermission(filePermission));
|
|
|
|
|
|
- long heartbeatInterval = conf.getLong(
|
|
|
- DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
|
|
|
- DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_DEFAULT) * 1000;
|
|
|
this.heartbeatRecheckInterval = conf.getInt(
|
|
|
DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_KEY,
|
|
|
DFSConfigKeys.DFS_NAMENODE_HEARTBEAT_RECHECK_INTERVAL_DEFAULT); // 5 minutes
|
|
|
- this.heartbeatExpireInterval = 2 * heartbeatRecheckInterval +
|
|
|
- 10 * heartbeatInterval;
|
|
|
|
|
|
this.serverDefaults = new FsServerDefaults(
|
|
|
conf.getLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_BLOCK_SIZE),
|
|
@@ -531,14 +494,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
this.maxFsObjects = conf.getLong(DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY,
|
|
|
DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT);
|
|
|
|
|
|
- //default limit
|
|
|
- this.blockInvalidateLimit = Math.max(this.blockInvalidateLimit,
|
|
|
- 20*(int)(heartbeatInterval/1000));
|
|
|
- //use conf value if it is set.
|
|
|
- this.blockInvalidateLimit = conf.getInt(
|
|
|
- DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY, this.blockInvalidateLimit);
|
|
|
- LOG.info(DFSConfigKeys.DFS_BLOCK_INVALIDATE_LIMIT_KEY + "=" + this.blockInvalidateLimit);
|
|
|
-
|
|
|
this.accessTimePrecision = conf.getLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, 0);
|
|
|
this.supportAppends = conf.getBoolean(DFSConfigKeys.DFS_SUPPORT_APPEND_KEY,
|
|
|
DFSConfigKeys.DFS_SUPPORT_APPEND_DEFAULT);
|
|
@@ -642,12 +597,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
out.println("Live Datanodes: "+live.size());
|
|
|
out.println("Dead Datanodes: "+dead.size());
|
|
|
blockManager.metaSave(out);
|
|
|
-
|
|
|
- //
|
|
|
- // Dump all datanodes
|
|
|
- //
|
|
|
- datanodeDump(out);
|
|
|
-
|
|
|
+
|
|
|
out.flush();
|
|
|
out.close();
|
|
|
} finally {
|
|
@@ -688,45 +638,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
readLock();
|
|
|
try {
|
|
|
checkSuperuserPrivilege();
|
|
|
-
|
|
|
- DatanodeDescriptor node = getDatanode(datanode);
|
|
|
- if (node == null) {
|
|
|
- NameNode.stateChangeLog.warn("BLOCK* NameSystem.getBlocks: "
|
|
|
- + "Asking for blocks from an unrecorded node " + datanode.getName());
|
|
|
- throw new IllegalArgumentException(
|
|
|
- "Unexpected exception. Got getBlocks message for datanode " +
|
|
|
- datanode.getName() + ", but there is no info for it");
|
|
|
- }
|
|
|
-
|
|
|
- int numBlocks = node.numBlocks();
|
|
|
- if(numBlocks == 0) {
|
|
|
- return new BlocksWithLocations(new BlockWithLocations[0]);
|
|
|
- }
|
|
|
- Iterator<BlockInfo> iter = node.getBlockIterator();
|
|
|
- int startBlock = DFSUtil.getRandom().nextInt(numBlocks); // starting from a random block
|
|
|
- // skip blocks
|
|
|
- for(int i=0; i<startBlock; i++) {
|
|
|
- iter.next();
|
|
|
- }
|
|
|
- List<BlockWithLocations> results = new ArrayList<BlockWithLocations>();
|
|
|
- long totalSize = 0;
|
|
|
- BlockInfo curBlock;
|
|
|
- while(totalSize<size && iter.hasNext()) {
|
|
|
- curBlock = iter.next();
|
|
|
- if(!curBlock.isComplete()) continue;
|
|
|
- totalSize += addBlock(curBlock, results);
|
|
|
- }
|
|
|
- if(totalSize<size) {
|
|
|
- iter = node.getBlockIterator(); // start from the beginning
|
|
|
- for(int i=0; i<startBlock&&totalSize<size; i++) {
|
|
|
- curBlock = iter.next();
|
|
|
- if(!curBlock.isComplete()) continue;
|
|
|
- totalSize += addBlock(curBlock, results);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- return new BlocksWithLocations(
|
|
|
- results.toArray(new BlockWithLocations[results.size()]));
|
|
|
+ return blockManager.getBlocksWithLocations(datanode, size);
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
@@ -742,22 +654,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
: ExportedBlockKeys.DUMMY_KEYS;
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Get all valid locations of the block & add the block to results
|
|
|
- * return the length of the added block; 0 if the block is not added
|
|
|
- */
|
|
|
- private long addBlock(Block block, List<BlockWithLocations> results) {
|
|
|
- assert hasReadOrWriteLock();
|
|
|
- ArrayList<String> machineSet = blockManager.getValidLocations(block);
|
|
|
- if(machineSet.size() == 0) {
|
|
|
- return 0;
|
|
|
- } else {
|
|
|
- results.add(new BlockWithLocations(block,
|
|
|
- machineSet.toArray(new String[machineSet.size()])));
|
|
|
- return block.getNumBytes();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/////////////////////////////////////////////////////////
|
|
|
//
|
|
|
// These methods are called by HadoopFS clients
|
|
@@ -1795,7 +1691,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
//find datanode descriptors
|
|
|
chosen = new ArrayList<DatanodeDescriptor>();
|
|
|
for(DatanodeInfo d : existings) {
|
|
|
- final DatanodeDescriptor descriptor = getDatanode(d);
|
|
|
+ final DatanodeDescriptor descriptor = blockManager.getDatanodeManager(
|
|
|
+ ).getDatanode(d);
|
|
|
if (descriptor != null) {
|
|
|
chosen.add(descriptor);
|
|
|
}
|
|
@@ -2622,7 +2519,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
if (newtargets.length > 0) {
|
|
|
descriptors = new DatanodeDescriptor[newtargets.length];
|
|
|
for(int i = 0; i < newtargets.length; i++) {
|
|
|
- descriptors[i] = getDatanode(newtargets[i]);
|
|
|
+ descriptors[i] = blockManager.getDatanodeManager().getDatanode(
|
|
|
+ newtargets[i]);
|
|
|
}
|
|
|
}
|
|
|
if (closeFile) {
|
|
@@ -2766,15 +2664,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return Storage.getRegistrationID(dir.fsImage.getStorage());
|
|
|
}
|
|
|
|
|
|
- public boolean isDatanodeDead(DatanodeDescriptor node) {
|
|
|
- return (node.getLastUpdate() <
|
|
|
- (now() - heartbeatExpireInterval));
|
|
|
- }
|
|
|
-
|
|
|
- private void setDatanodeDead(DatanodeDescriptor node) throws IOException {
|
|
|
- node.setLastUpdate(0);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* The given node has reported in. This method should:
|
|
|
* 1) Record the heartbeat, so the datanode isn't timed out
|
|
@@ -2792,91 +2681,32 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
throws IOException {
|
|
|
readLock();
|
|
|
try {
|
|
|
- return handleHeartbeatInternal(nodeReg, capacity, dfsUsed,
|
|
|
- remaining, blockPoolUsed, xceiverCount, xmitsInProgress,
|
|
|
- failedVolumes);
|
|
|
+ final int maxTransfer = blockManager.maxReplicationStreams - xmitsInProgress;
|
|
|
+ DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
|
|
|
+ nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed,
|
|
|
+ xceiverCount, maxTransfer, failedVolumes);
|
|
|
+ if (cmds != null) {
|
|
|
+ return cmds;
|
|
|
+ }
|
|
|
+
|
|
|
+ //check distributed upgrade
|
|
|
+ DatanodeCommand cmd = getDistributedUpgradeCommand();
|
|
|
+ if (cmd != null) {
|
|
|
+ return new DatanodeCommand[] {cmd};
|
|
|
+ }
|
|
|
+ return null;
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /** @see #handleHeartbeat(DatanodeRegistration, long, long, long, long, int, int, int) */
|
|
|
- DatanodeCommand[] handleHeartbeatInternal(DatanodeRegistration nodeReg,
|
|
|
- long capacity, long dfsUsed, long remaining, long blockPoolUsed,
|
|
|
- int xceiverCount, int xmitsInProgress, int failedVolumes)
|
|
|
- throws IOException {
|
|
|
- assert hasReadLock();
|
|
|
- DatanodeCommand cmd = null;
|
|
|
- synchronized (heartbeats) {
|
|
|
- synchronized (datanodeMap) {
|
|
|
- DatanodeDescriptor nodeinfo = null;
|
|
|
- try {
|
|
|
- nodeinfo = getDatanode(nodeReg);
|
|
|
- } catch(UnregisteredNodeException e) {
|
|
|
- return new DatanodeCommand[]{DatanodeCommand.REGISTER};
|
|
|
- }
|
|
|
-
|
|
|
- // Check if this datanode should actually be shutdown instead.
|
|
|
- if (nodeinfo != null && nodeinfo.isDisallowed()) {
|
|
|
- setDatanodeDead(nodeinfo);
|
|
|
- throw new DisallowedDatanodeException(nodeinfo);
|
|
|
- }
|
|
|
-
|
|
|
- if (nodeinfo == null || !nodeinfo.isAlive) {
|
|
|
- return new DatanodeCommand[]{DatanodeCommand.REGISTER};
|
|
|
- }
|
|
|
-
|
|
|
- updateStats(nodeinfo, false);
|
|
|
- nodeinfo.updateHeartbeat(capacity, dfsUsed, remaining, blockPoolUsed,
|
|
|
- xceiverCount, failedVolumes);
|
|
|
- updateStats(nodeinfo, true);
|
|
|
-
|
|
|
- //check lease recovery
|
|
|
- BlockInfoUnderConstruction[] blocks = nodeinfo
|
|
|
- .getLeaseRecoveryCommand(Integer.MAX_VALUE);
|
|
|
- if (blocks != null) {
|
|
|
- BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
|
|
|
- blocks.length);
|
|
|
- for (BlockInfoUnderConstruction b : blocks) {
|
|
|
- brCommand.add(new RecoveringBlock(
|
|
|
- new ExtendedBlock(blockPoolId, b), b.getExpectedLocations(), b
|
|
|
- .getBlockRecoveryId()));
|
|
|
- }
|
|
|
- return new DatanodeCommand[] { brCommand };
|
|
|
- }
|
|
|
-
|
|
|
- ArrayList<DatanodeCommand> cmds = new ArrayList<DatanodeCommand>(3);
|
|
|
- //check pending replication
|
|
|
- List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
|
|
|
- blockManager.maxReplicationStreams - xmitsInProgress);
|
|
|
- if (pendingList != null) {
|
|
|
- cmd = new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId,
|
|
|
- pendingList);
|
|
|
- cmds.add(cmd);
|
|
|
- }
|
|
|
- //check block invalidation
|
|
|
- Block[] blks = nodeinfo.getInvalidateBlocks(blockInvalidateLimit);
|
|
|
- if (blks != null) {
|
|
|
- cmd = new BlockCommand(DatanodeProtocol.DNA_INVALIDATE, blockPoolId, blks);
|
|
|
- cmds.add(cmd);
|
|
|
- }
|
|
|
- // check access key update
|
|
|
- if (isBlockTokenEnabled && nodeinfo.needKeyUpdate) {
|
|
|
- cmds.add(new KeyUpdateCommand(blockTokenSecretManager.exportKeys()));
|
|
|
- nodeinfo.needKeyUpdate = false;
|
|
|
- }
|
|
|
- if (!cmds.isEmpty()) {
|
|
|
- return cmds.toArray(new DatanodeCommand[cmds.size()]);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- //check distributed upgrade
|
|
|
- cmd = getDistributedUpgradeCommand();
|
|
|
- if (cmd != null) {
|
|
|
- return new DatanodeCommand[] {cmd};
|
|
|
+ public void addKeyUpdateCommand(final List<DatanodeCommand> cmds,
|
|
|
+ final DatanodeDescriptor nodeinfo) {
|
|
|
+ // check access key update
|
|
|
+ if (isBlockTokenEnabled && nodeinfo.needKeyUpdate) {
|
|
|
+ cmds.add(new KeyUpdateCommand(blockTokenSecretManager.exportKeys()));
|
|
|
+ nodeinfo.needKeyUpdate = false;
|
|
|
}
|
|
|
- return null;
|
|
|
}
|
|
|
|
|
|
public void updateStats(DatanodeDescriptor node, boolean isAdded) {
|
|
@@ -3017,7 +2847,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
) throws UnregisteredNodeException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
- DatanodeDescriptor nodeInfo = getDatanode(nodeID);
|
|
|
+ DatanodeDescriptor nodeInfo = getBlockManager().getDatanodeManager(
|
|
|
+ ).getDatanode(nodeID);
|
|
|
if (nodeInfo != null) {
|
|
|
removeDatanode(nodeInfo);
|
|
|
} else {
|
|
@@ -3033,7 +2864,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* Remove a datanode descriptor.
|
|
|
* @param nodeInfo datanode descriptor.
|
|
|
*/
|
|
|
- private void removeDatanode(DatanodeDescriptor nodeInfo) {
|
|
|
+ public void removeDatanode(DatanodeDescriptor nodeInfo) {
|
|
|
assert hasWriteLock();
|
|
|
synchronized (heartbeats) {
|
|
|
if (nodeInfo.isAlive) {
|
|
@@ -3064,6 +2895,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
* effect causes more datanodes to be declared dead.
|
|
|
*/
|
|
|
void heartbeatCheck() {
|
|
|
+ final DatanodeManager datanodeManager = getBlockManager().getDatanodeManager();
|
|
|
// It's OK to check safe mode w/o taking the lock here, we re-check
|
|
|
// for safe mode after taking the lock before removing a datanode.
|
|
|
if (isInSafeMode()) {
|
|
@@ -3079,7 +2911,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
for (Iterator<DatanodeDescriptor> it = heartbeats.iterator();
|
|
|
it.hasNext();) {
|
|
|
DatanodeDescriptor nodeInfo = it.next();
|
|
|
- if (isDatanodeDead(nodeInfo)) {
|
|
|
+ if (datanodeManager.isDatanodeDead(nodeInfo)) {
|
|
|
expiredHeartbeats.incr();
|
|
|
foundDead = true;
|
|
|
nodeID = nodeInfo;
|
|
@@ -3095,21 +2927,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return;
|
|
|
}
|
|
|
try {
|
|
|
- synchronized(heartbeats) {
|
|
|
- synchronized (datanodeMap) {
|
|
|
- DatanodeDescriptor nodeInfo = null;
|
|
|
- try {
|
|
|
- nodeInfo = getDatanode(nodeID);
|
|
|
- } catch (IOException e) {
|
|
|
- nodeInfo = null;
|
|
|
- }
|
|
|
- if (nodeInfo != null && isDatanodeDead(nodeInfo)) {
|
|
|
- NameNode.stateChangeLog.info("BLOCK* NameSystem.heartbeatCheck: "
|
|
|
- + "lost heartbeat from " + nodeInfo.getName());
|
|
|
- removeDatanode(nodeInfo);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ datanodeManager.removeDeadDatanode(nodeID);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
|
}
|
|
@@ -3129,7 +2947,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
writeLock();
|
|
|
startTime = now(); //after acquiring write lock
|
|
|
try {
|
|
|
- DatanodeDescriptor node = getDatanode(nodeID);
|
|
|
+ final DatanodeDescriptor node = blockManager.getDatanodeManager(
|
|
|
+ ).getDatanode(nodeID);
|
|
|
if (node == null || !node.isAlive) {
|
|
|
throw new IOException("ProcessReport from dead or unregistered node: "
|
|
|
+ nodeID.getName());
|
|
@@ -3269,7 +3088,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
) throws IOException {
|
|
|
writeLock();
|
|
|
try {
|
|
|
- DatanodeDescriptor node = getDatanode(nodeID);
|
|
|
+ final DatanodeDescriptor node = blockManager.getDatanodeManager(
|
|
|
+ ).getDatanode(nodeID);
|
|
|
if (node == null || !node.isAlive) {
|
|
|
NameNode.stateChangeLog.warn("BLOCK* NameSystem.blockReceived: " + block
|
|
|
+ " is received from dead or unregistered node " + nodeID.getName());
|
|
@@ -3475,33 +3295,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
ArrayList<DatanodeDescriptor> dead) {
|
|
|
readLock();
|
|
|
try {
|
|
|
- final List<DatanodeDescriptor> results = getBlockManager(
|
|
|
- ).getDatanodeManager().getDatanodeListForReport(DatanodeReportType.ALL);
|
|
|
- for(Iterator<DatanodeDescriptor> it = results.iterator(); it.hasNext();) {
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
- if (isDatanodeDead(node))
|
|
|
- dead.add(node);
|
|
|
- else
|
|
|
- live.add(node);
|
|
|
- }
|
|
|
- } finally {
|
|
|
- readUnlock();
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Prints information about all datanodes.
|
|
|
- */
|
|
|
- private void datanodeDump(PrintWriter out) {
|
|
|
- readLock();
|
|
|
- try {
|
|
|
- synchronized (datanodeMap) {
|
|
|
- out.println("Metasave: Number of datanodes: " + datanodeMap.size());
|
|
|
- for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator(); it.hasNext();) {
|
|
|
- DatanodeDescriptor node = it.next();
|
|
|
- out.println(node.dumpDatanode());
|
|
|
- }
|
|
|
- }
|
|
|
+ getBlockManager().getDatanodeManager().fetchDatanodess(live, dead);
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
@@ -3556,30 +3350,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
checkSuperuserPrivilege();
|
|
|
getFSImage().finalizeUpgrade();
|
|
|
}
|
|
|
-
|
|
|
-
|
|
|
- /**
|
|
|
- * Get data node by storage ID.
|
|
|
- *
|
|
|
- * @param nodeID
|
|
|
- * @return DatanodeDescriptor or null if the node is not found.
|
|
|
- * @throws IOException
|
|
|
- */
|
|
|
- public DatanodeDescriptor getDatanode(DatanodeID nodeID
|
|
|
- ) throws UnregisteredNodeException {
|
|
|
- assert hasReadOrWriteLock();
|
|
|
- UnregisteredNodeException e = null;
|
|
|
- DatanodeDescriptor node = datanodeMap.get(nodeID.getStorageID());
|
|
|
- if (node == null)
|
|
|
- return null;
|
|
|
- if (!node.getName().equals(nodeID.getName())) {
|
|
|
- e = new UnregisteredNodeException(nodeID, node);
|
|
|
- NameNode.stateChangeLog.fatal("BLOCK* NameSystem.getDatanode: "
|
|
|
- + e.getLocalizedMessage());
|
|
|
- throw e;
|
|
|
- }
|
|
|
- return node;
|
|
|
- }
|
|
|
|
|
|
/**
|
|
|
* SafeModeInfo contains information related to the safe mode.
|
|
@@ -4503,43 +4273,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
}
|
|
|
|
|
|
|
|
|
- /**
|
|
|
- * Number of live data nodes
|
|
|
- * @return Number of live data nodes
|
|
|
- */
|
|
|
@Override // FSNamesystemMBean
|
|
|
public int getNumLiveDataNodes() {
|
|
|
- int numLive = 0;
|
|
|
- synchronized (datanodeMap) {
|
|
|
- for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator();
|
|
|
- it.hasNext();) {
|
|
|
- DatanodeDescriptor dn = it.next();
|
|
|
- if (!isDatanodeDead(dn) ) {
|
|
|
- numLive++;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- return numLive;
|
|
|
+ return getBlockManager().getDatanodeManager().getNumLiveDataNodes();
|
|
|
}
|
|
|
-
|
|
|
|
|
|
- /**
|
|
|
- * Number of dead data nodes
|
|
|
- * @return Number of dead data nodes
|
|
|
- */
|
|
|
@Override // FSNamesystemMBean
|
|
|
public int getNumDeadDataNodes() {
|
|
|
- int numDead = 0;
|
|
|
- synchronized (datanodeMap) {
|
|
|
- for(Iterator<DatanodeDescriptor> it = datanodeMap.values().iterator();
|
|
|
- it.hasNext();) {
|
|
|
- DatanodeDescriptor dn = it.next();
|
|
|
- if (isDatanodeDead(dn) ) {
|
|
|
- numDead++;
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
- return numDead;
|
|
|
+ return getBlockManager().getDatanodeManager().getNumDeadDataNodes();
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -4699,11 +4440,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
blockinfo.setNumBytes(newBlock.getNumBytes());
|
|
|
|
|
|
// find the DatanodeDescriptor objects
|
|
|
+ final DatanodeManager dm = getBlockManager().getDatanodeManager();
|
|
|
DatanodeDescriptor[] descriptors = null;
|
|
|
if (newNodes.length > 0) {
|
|
|
descriptors = new DatanodeDescriptor[newNodes.length];
|
|
|
for(int i = 0; i < newNodes.length; i++) {
|
|
|
- descriptors[i] = getDatanode(newNodes[i]);
|
|
|
+ descriptors[i] = dm.getDatanode(newNodes[i]);
|
|
|
}
|
|
|
}
|
|
|
blockinfo.setExpectedLocations(descriptors);
|
|
@@ -4832,12 +4574,6 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
|
|
|
return blockManager.numCorruptReplicas(blk);
|
|
|
}
|
|
|
|
|
|
- /** Get a datanode descriptor given corresponding storageID */
|
|
|
- public DatanodeDescriptor getDatanode(String nodeID) {
|
|
|
- assert hasReadOrWriteLock();
|
|
|
- return datanodeMap.get(nodeID);
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Return a range of corrupt replica block ids. Up to numExpectedBlocks
|
|
|
* blocks starting at the next block after startingBlockId are returned
|