|
@@ -656,11 +656,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
editLog.recoverUnclosedStreams();
|
|
|
|
|
|
LOG.info("Catching up to latest edits from old active before " +
|
|
|
- "taking over writer role in edits logs.");
|
|
|
+ "taking over writer role in edits logs");
|
|
|
editLogTailer.catchupDuringFailover();
|
|
|
blockManager.setPostponeBlocksFromFuture(false);
|
|
|
|
|
|
- LOG.info("Reprocessing replication and invalidation queues...");
|
|
|
+ LOG.info("Reprocessing replication and invalidation queues");
|
|
|
blockManager.getDatanodeManager().markAllDatanodesStale();
|
|
|
blockManager.clearQueues();
|
|
|
blockManager.processAllPendingDNMessages();
|
|
@@ -1969,7 +1969,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
if (force) {
|
|
|
// close now: no need to wait for soft lease expiration and
|
|
|
// close only the file src
|
|
|
- LOG.info("recoverLease: recover lease " + lease + ", src=" + src +
|
|
|
+ LOG.info("recoverLease: " + lease + ", src=" + src +
|
|
|
" from client " + pendingFile.getClientName());
|
|
|
internalReleaseLease(lease, src, holder);
|
|
|
} else {
|
|
@@ -1981,8 +1981,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// period, then start lease recovery.
|
|
|
//
|
|
|
if (lease.expiredSoftLimit()) {
|
|
|
- LOG.info("startFile: recover lease " + lease + ", src=" + src +
|
|
|
- " from client " + pendingFile.getClientName());
|
|
|
+ LOG.info("startFile: recover " + lease + ", src=" + src + " client "
|
|
|
+ + pendingFile.getClientName());
|
|
|
boolean isClosed = internalReleaseLease(lease, src, null);
|
|
|
if(!isClosed)
|
|
|
throw new RecoveryInProgressException(
|
|
@@ -2158,7 +2158,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
|
|
|
// The retry case ("b" above) -- abandon the old block.
|
|
|
- NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: " +
|
|
|
+ NameNode.stateChangeLog.info("BLOCK* allocateBlock: " +
|
|
|
"caught retry for allocation of a new block in " +
|
|
|
src + ". Abandoning old block " + lastBlockInFile);
|
|
|
dir.removeBlock(src, pendingFile, lastBlockInFile);
|
|
@@ -2394,10 +2394,10 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
// See HDFS-3031.
|
|
|
final Block realLastBlock = ((INodeFile)inode).getLastBlock();
|
|
|
if (Block.matchingIdAndGenStamp(last, realLastBlock)) {
|
|
|
- NameNode.stateChangeLog.info("DIR* NameSystem.completeFile: " +
|
|
|
- "received request from " + holder + " to complete file " + src +
|
|
|
+ NameNode.stateChangeLog.info("DIR* completeFile: " +
|
|
|
+ "request from " + holder + " to complete " + src +
|
|
|
" which is already closed. But, it appears to be an RPC " +
|
|
|
- "retry. Returning success.");
|
|
|
+ "retry. Returning success");
|
|
|
return true;
|
|
|
}
|
|
|
}
|
|
@@ -2412,8 +2412,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
|
|
|
finalizeINodeFileUnderConstruction(src, pendingFile);
|
|
|
|
|
|
- NameNode.stateChangeLog.info("DIR* NameSystem.completeFile: file " + src
|
|
|
- + " is closed by " + holder);
|
|
|
+ NameNode.stateChangeLog.info("DIR* completeFile: " + src + " is closed by "
|
|
|
+ + holder);
|
|
|
return true;
|
|
|
}
|
|
|
|
|
@@ -2438,8 +2438,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
nextGenerationStamp();
|
|
|
b.setGenerationStamp(getGenerationStamp());
|
|
|
b = dir.addBlock(src, inodes, b, targets);
|
|
|
- NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: "
|
|
|
- +src+ ". " + blockPoolId + " "+ b);
|
|
|
+ NameNode.stateChangeLog.info("BLOCK* allocateBlock: " + src + ". "
|
|
|
+ + blockPoolId + " " + b);
|
|
|
return b;
|
|
|
}
|
|
|
|
|
@@ -2457,8 +2457,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
//
|
|
|
for (BlockInfo block: v.getBlocks()) {
|
|
|
if (!block.isComplete()) {
|
|
|
- LOG.info("BLOCK* NameSystem.checkFileProgress: "
|
|
|
- + "block " + block + " has not reached minimal replication "
|
|
|
+ LOG.info("BLOCK* checkFileProgress: " + block
|
|
|
+ + " has not reached minimal replication "
|
|
|
+ blockManager.minReplication);
|
|
|
return false;
|
|
|
}
|
|
@@ -2469,8 +2469,8 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
//
|
|
|
BlockInfo b = v.getPenultimateBlock();
|
|
|
if (b != null && !b.isComplete()) {
|
|
|
- LOG.info("BLOCK* NameSystem.checkFileProgress: "
|
|
|
- + "block " + b + " has not reached minimal replication "
|
|
|
+ LOG.info("BLOCK* checkFileProgress: " + b
|
|
|
+ + " has not reached minimal replication "
|
|
|
+ blockManager.minReplication);
|
|
|
return false;
|
|
|
}
|
|
@@ -2943,8 +2943,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
*/
|
|
|
void fsync(String src, String clientName)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
- NameNode.stateChangeLog.info("BLOCK* NameSystem.fsync: file "
|
|
|
- + src + " for " + clientName);
|
|
|
+ NameNode.stateChangeLog.info("BLOCK* fsync: " + src + " for " + clientName);
|
|
|
writeLock();
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -2975,7 +2974,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
boolean internalReleaseLease(Lease lease, String src,
|
|
|
String recoveryLeaseHolder) throws AlreadyBeingCreatedException,
|
|
|
IOException, UnresolvedLinkException {
|
|
|
- LOG.info("Recovering lease=" + lease + ", src=" + src);
|
|
|
+ LOG.info("Recovering " + lease + ", src=" + src);
|
|
|
assert !isInSafeMode();
|
|
|
assert hasWriteLock();
|
|
|
|
|
@@ -3616,7 +3615,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
"in order to create namespace image.");
|
|
|
}
|
|
|
getFSImage().saveNamespace(this);
|
|
|
- LOG.info("New namespace image has been created.");
|
|
|
+ LOG.info("New namespace image has been created");
|
|
|
} finally {
|
|
|
readUnlock();
|
|
|
}
|
|
@@ -3834,11 +3833,11 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
long timeInSafemode = now() - startTime;
|
|
|
NameNode.stateChangeLog.info("STATE* Leaving safe mode after "
|
|
|
- + timeInSafemode/1000 + " secs.");
|
|
|
+ + timeInSafemode/1000 + " secs");
|
|
|
NameNode.getNameNodeMetrics().setSafeModeTime((int) timeInSafemode);
|
|
|
|
|
|
if (reached >= 0) {
|
|
|
- NameNode.stateChangeLog.info("STATE* Safe mode is OFF.");
|
|
|
+ NameNode.stateChangeLog.info("STATE* Safe mode is OFF");
|
|
|
}
|
|
|
reached = -1;
|
|
|
safeMode = null;
|
|
@@ -4158,7 +4157,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
}
|
|
|
}
|
|
|
if (!fsRunning) {
|
|
|
- LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread. ");
|
|
|
+ LOG.info("NameNode is being shutdown, exit SafeModeMonitor thread");
|
|
|
} else {
|
|
|
// leave safe mode and stop the monitor
|
|
|
leaveSafeMode();
|
|
@@ -4343,7 +4342,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
if (isEditlogOpenForWrite) {
|
|
|
getEditLog().logSyncAll();
|
|
|
}
|
|
|
- NameNode.stateChangeLog.info("STATE* Safe mode is ON. "
|
|
|
+ NameNode.stateChangeLog.info("STATE* Safe mode is ON"
|
|
|
+ safeMode.getTurnOffTip());
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -4358,7 +4357,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
writeLock();
|
|
|
try {
|
|
|
if (!isInSafeMode()) {
|
|
|
- NameNode.stateChangeLog.info("STATE* Safe mode is already OFF.");
|
|
|
+ NameNode.stateChangeLog.info("STATE* Safe mode is already OFF");
|
|
|
return;
|
|
|
}
|
|
|
safeMode.leave();
|
|
@@ -4712,7 +4711,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
|
|
|
try {
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
|
|
|
- NameNode.stateChangeLog.info("*DIR* NameNode.reportBadBlocks");
|
|
|
+ NameNode.stateChangeLog.info("*DIR* reportBadBlocks");
|
|
|
for (int i = 0; i < blocks.length; i++) {
|
|
|
ExtendedBlock blk = blocks[i].getBlock();
|
|
|
DatanodeInfo[] nodes = blocks[i].getLocations();
|