|
@@ -77,6 +77,7 @@ import com.google.common.annotations.VisibleForTesting;
|
|
|
@InterfaceAudience.Private
|
|
|
public class BlockManager {
|
|
|
static final Log LOG = LogFactory.getLog(BlockManager.class);
|
|
|
+ static final Log blockLog = NameNode.blockStateChangeLog;
|
|
|
|
|
|
/** Default load factor of map */
|
|
|
public static final float DEFAULT_MAP_LOAD_FACTOR = 0.75f;
|
|
@@ -729,7 +730,7 @@ public class BlockManager {
|
|
|
final long size) throws UnregisteredNodeException {
|
|
|
final DatanodeDescriptor node = getDatanodeManager().getDatanode(datanode);
|
|
|
if (node == null) {
|
|
|
- NameNode.stateChangeLog.warn("BLOCK* getBlocks: "
|
|
|
+ blockLog.warn("BLOCK* getBlocks: "
|
|
|
+ "Asking for blocks from an unrecorded node " + datanode.getName());
|
|
|
throw new HadoopIllegalArgumentException(
|
|
|
"Datanode " + datanode.getName() + " not found.");
|
|
@@ -799,7 +800,7 @@ public class BlockManager {
|
|
|
datanodes.append(node.getName()).append(" ");
|
|
|
}
|
|
|
if (datanodes.length() != 0) {
|
|
|
- NameNode.stateChangeLog.info("BLOCK* addToInvalidates: "
|
|
|
+ blockLog.info("BLOCK* addToInvalidates: "
|
|
|
+ b + " to " + datanodes.toString());
|
|
|
}
|
|
|
}
|
|
@@ -821,7 +822,7 @@ public class BlockManager {
|
|
|
// ignore the request for now. This could happen when BlockScanner
|
|
|
// thread of Datanode reports bad block before Block reports are sent
|
|
|
// by the Datanode on startup
|
|
|
- NameNode.stateChangeLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
|
|
|
+ blockLog.info("BLOCK* findAndMarkBlockAsCorrupt: "
|
|
|
+ blk + " not found.");
|
|
|
return;
|
|
|
}
|
|
@@ -845,7 +846,7 @@ public class BlockManager {
|
|
|
|
|
|
INodeFile inode = storedBlock.getINode();
|
|
|
if (inode == null) {
|
|
|
- NameNode.stateChangeLog.info("BLOCK markBlockAsCorrupt: " +
|
|
|
+ blockLog.info("BLOCK markBlockAsCorrupt: " +
|
|
|
"block " + storedBlock +
|
|
|
" could not be marked as corrupt as it" +
|
|
|
" does not belong to any file");
|
|
@@ -872,7 +873,7 @@ public class BlockManager {
|
|
|
*/
|
|
|
private void invalidateBlock(Block blk, DatanodeInfo dn)
|
|
|
throws IOException {
|
|
|
- NameNode.stateChangeLog.info("BLOCK* invalidateBlock: "
|
|
|
+ blockLog.info("BLOCK* invalidateBlock: "
|
|
|
+ blk + " on " + dn.getName());
|
|
|
DatanodeDescriptor node = getDatanodeManager().getDatanode(dn);
|
|
|
if (node == null) {
|
|
@@ -886,12 +887,12 @@ public class BlockManager {
|
|
|
if (count >= 1) {
|
|
|
addToInvalidates(blk, dn);
|
|
|
removeStoredBlock(blk, node);
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* invalidateBlocks: "
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* invalidateBlocks: "
|
|
|
+ blk + " on " + dn.getName() + " listed for deletion.");
|
|
|
}
|
|
|
} else {
|
|
|
- NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: " + blk + " on "
|
|
|
+ blockLog.info("BLOCK* invalidateBlocks: " + blk + " on "
|
|
|
+ dn.getName() + " is the only copy and was not deleted.");
|
|
|
}
|
|
|
}
|
|
@@ -1060,7 +1061,7 @@ public class BlockManager {
|
|
|
(blockHasEnoughRacks(block)) ) {
|
|
|
neededReplications.remove(block, priority); // remove from neededReplications
|
|
|
replIndex--;
|
|
|
- NameNode.stateChangeLog.info("BLOCK* "
|
|
|
+ blockLog.info("BLOCK* "
|
|
|
+ "Removing block " + block
|
|
|
+ " from neededReplications as it has enough replicas.");
|
|
|
return false;
|
|
@@ -1118,7 +1119,7 @@ public class BlockManager {
|
|
|
(blockHasEnoughRacks(block)) ) {
|
|
|
neededReplications.remove(block, priority); // remove from neededReplications
|
|
|
replIndex--;
|
|
|
- NameNode.stateChangeLog.info("BLOCK* "
|
|
|
+ blockLog.info("BLOCK* "
|
|
|
+ "Removing block " + block
|
|
|
+ " from neededReplications as it has enough replicas.");
|
|
|
return false;
|
|
@@ -1144,8 +1145,8 @@ public class BlockManager {
|
|
|
// The reason we use 'pending' is so we can retry
|
|
|
// replications that fail after an appropriate amount of time.
|
|
|
pendingReplications.add(block, targets.length);
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug(
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug(
|
|
|
"BLOCK* block " + block
|
|
|
+ " is moved from neededReplications to pendingReplications");
|
|
|
}
|
|
@@ -1155,18 +1156,18 @@ public class BlockManager {
|
|
|
neededReplications.remove(block, priority); // remove from neededReplications
|
|
|
replIndex--;
|
|
|
}
|
|
|
- if (NameNode.stateChangeLog.isInfoEnabled()) {
|
|
|
+ if (blockLog.isInfoEnabled()) {
|
|
|
StringBuilder targetList = new StringBuilder("datanode(s)");
|
|
|
for (int k = 0; k < targets.length; k++) {
|
|
|
targetList.append(' ');
|
|
|
targetList.append(targets[k].getName());
|
|
|
}
|
|
|
- NameNode.stateChangeLog.info(
|
|
|
+ blockLog.info(
|
|
|
"BLOCK* ask "
|
|
|
+ srcNode.getName() + " to replicate "
|
|
|
+ block + " to " + targetList);
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug(
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug(
|
|
|
"BLOCK* neededReplications = " + neededReplications.size()
|
|
|
+ " pendingReplications = " + pendingReplications.size());
|
|
|
}
|
|
@@ -1354,7 +1355,7 @@ public class BlockManager {
|
|
|
// To minimize startup time, we discard any second (or later) block reports
|
|
|
// that we receive while still in startup phase.
|
|
|
if (namesystem.isInStartupSafeMode() && node.numBlocks() > 0) {
|
|
|
- NameNode.stateChangeLog.info("BLOCK* processReport: "
|
|
|
+ blockLog.info("BLOCK* processReport: "
|
|
|
+ "discarded non-initial block report from " + nodeID.getName()
|
|
|
+ " because namenode still in startup phase");
|
|
|
return;
|
|
@@ -1374,7 +1375,7 @@ public class BlockManager {
|
|
|
|
|
|
// Log the block report processing stats from Namenode perspective
|
|
|
NameNode.getNameNodeMetrics().addBlockReport((int) (endTime - startTime));
|
|
|
- NameNode.stateChangeLog.info("BLOCK* processReport: from "
|
|
|
+ blockLog.info("BLOCK* processReport: from "
|
|
|
+ nodeID.getName() + ", blocks: " + newReport.getNumberOfBlocks()
|
|
|
+ ", processing time: " + (endTime - startTime) + " msecs");
|
|
|
}
|
|
@@ -1403,7 +1404,7 @@ public class BlockManager {
|
|
|
addStoredBlock(b, node, null, true);
|
|
|
}
|
|
|
for (Block b : toInvalidate) {
|
|
|
- NameNode.stateChangeLog.info("BLOCK* processReport: block "
|
|
|
+ blockLog.info("BLOCK* processReport: block "
|
|
|
+ b + " on " + node.getName() + " size " + b.getNumBytes()
|
|
|
+ " does not belong to any file.");
|
|
|
addToInvalidates(b, node);
|
|
@@ -1737,7 +1738,7 @@ public class BlockManager {
|
|
|
}
|
|
|
if (storedBlock == null || storedBlock.getINode() == null) {
|
|
|
// If this block does not belong to anyfile, then we are done.
|
|
|
- NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on "
|
|
|
+ blockLog.info("BLOCK* addStoredBlock: " + block + " on "
|
|
|
+ node.getName() + " size " + block.getNumBytes()
|
|
|
+ " but it does not belong to any file.");
|
|
|
// we could add this block to invalidate set of this datanode.
|
|
@@ -1755,13 +1756,13 @@ public class BlockManager {
|
|
|
if (added) {
|
|
|
curReplicaDelta = 1;
|
|
|
if (logEveryBlock) {
|
|
|
- NameNode.stateChangeLog.info("BLOCK* addStoredBlock: "
|
|
|
+ blockLog.info("BLOCK* addStoredBlock: "
|
|
|
+ "blockMap updated: " + node.getName() + " is added to " +
|
|
|
storedBlock + " size " + storedBlock.getNumBytes());
|
|
|
}
|
|
|
} else {
|
|
|
curReplicaDelta = 0;
|
|
|
- NameNode.stateChangeLog.warn("BLOCK* addStoredBlock: "
|
|
|
+ blockLog.warn("BLOCK* addStoredBlock: "
|
|
|
+ "Redundant addStoredBlock request received for " + storedBlock
|
|
|
+ " on " + node.getName() + " size " + storedBlock.getNumBytes());
|
|
|
}
|
|
@@ -1842,7 +1843,7 @@ public class BlockManager {
|
|
|
try {
|
|
|
invalidateBlock(blk, node);
|
|
|
} catch (IOException e) {
|
|
|
- NameNode.stateChangeLog.info("NameNode.invalidateCorruptReplicas " +
|
|
|
+ blockLog.info("NameNode.invalidateCorruptReplicas " +
|
|
|
"error in deleting bad block " + blk +
|
|
|
" on " + node + e);
|
|
|
gotException = true;
|
|
@@ -2057,7 +2058,7 @@ public class BlockManager {
|
|
|
// upon giving instructions to the namenode.
|
|
|
//
|
|
|
addToInvalidates(b, cur);
|
|
|
- NameNode.stateChangeLog.info("BLOCK* chooseExcessReplicates: "
|
|
|
+ blockLog.info("BLOCK* chooseExcessReplicates: "
|
|
|
+"("+cur.getName()+", "+b+") is added to invalidated blocks set.");
|
|
|
}
|
|
|
}
|
|
@@ -2071,8 +2072,8 @@ public class BlockManager {
|
|
|
}
|
|
|
if (excessBlocks.add(block)) {
|
|
|
excessBlocksCount++;
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* addToExcessReplicate:"
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* addToExcessReplicate:"
|
|
|
+ " (" + dn.getName() + ", " + block
|
|
|
+ ") is added to excessReplicateMap");
|
|
|
}
|
|
@@ -2084,15 +2085,15 @@ public class BlockManager {
|
|
|
* removed block is still valid.
|
|
|
*/
|
|
|
private void removeStoredBlock(Block block, DatanodeDescriptor node) {
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* removeStoredBlock: "
|
|
|
+ block + " from " + node.getName());
|
|
|
}
|
|
|
assert (namesystem.hasWriteLock());
|
|
|
{
|
|
|
if (!blocksMap.removeNode(block, node)) {
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* removeStoredBlock: "
|
|
|
+ block + " has already been removed from node " + node);
|
|
|
}
|
|
|
return;
|
|
@@ -2119,8 +2120,8 @@ public class BlockManager {
|
|
|
if (excessBlocks != null) {
|
|
|
if (excessBlocks.remove(block)) {
|
|
|
excessBlocksCount--;
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
|
|
|
+ if(blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* removeStoredBlock: "
|
|
|
+ block + " is removed from excessBlocks");
|
|
|
}
|
|
|
if (excessBlocks.size() == 0) {
|
|
@@ -2163,7 +2164,7 @@ public class BlockManager {
|
|
|
if (delHint != null && delHint.length() != 0) {
|
|
|
delHintNode = datanodeManager.getDatanode(delHint);
|
|
|
if (delHintNode == null) {
|
|
|
- NameNode.stateChangeLog.warn("BLOCK* blockReceived: " + block
|
|
|
+ blockLog.warn("BLOCK* blockReceived: " + block
|
|
|
+ " is expected to be removed from an unrecorded node " + delHint);
|
|
|
}
|
|
|
}
|
|
@@ -2192,7 +2193,7 @@ public class BlockManager {
|
|
|
addStoredBlock(b, node, delHintNode, true);
|
|
|
}
|
|
|
for (Block b : toInvalidate) {
|
|
|
- NameNode.stateChangeLog.info("BLOCK* addBlock: block "
|
|
|
+ blockLog.info("BLOCK* addBlock: block "
|
|
|
+ b + " on " + node.getName() + " size " + b.getNumBytes()
|
|
|
+ " does not belong to any file.");
|
|
|
addToInvalidates(b, node);
|
|
@@ -2211,12 +2212,12 @@ public class BlockManager {
|
|
|
if (node == null || !node.isAlive) {
|
|
|
final String s = block + " is received from dead or unregistered node "
|
|
|
+ nodeID.getName();
|
|
|
- NameNode.stateChangeLog.warn("BLOCK* blockReceived: " + s);
|
|
|
+ blockLog.warn("BLOCK* blockReceived: " + s);
|
|
|
throw new IOException(s);
|
|
|
}
|
|
|
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* blockReceived: " + block
|
|
|
+ if (blockLog.isDebugEnabled()) {
|
|
|
+ blockLog.debug("BLOCK* blockReceived: " + block
|
|
|
+ " is received from " + nodeID.getName());
|
|
|
}
|
|
|
|