|
@@ -2008,10 +2008,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
long mtime)
|
|
|
throws IOException, UnresolvedLinkException {
|
|
|
String src = srcArg;
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.truncate: src="
|
|
|
- + src + " newLength=" + newLength);
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug(
|
|
|
+ "DIR* NameSystem.truncate: src={} newLength={}", src, newLength);
|
|
|
if (newLength < 0) {
|
|
|
throw new HadoopIllegalArgumentException(
|
|
|
"Cannot truncate to a negative file size: " + newLength + ".");
|
|
@@ -2161,10 +2159,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock));
|
|
|
getBlockManager().addBlockCollection(truncatedBlockUC, file);
|
|
|
|
|
|
- NameNode.stateChangeLog.info("BLOCK* prepareFileForTruncate: "
|
|
|
- + "Scheduling copy-on-truncate to new size "
|
|
|
- + truncatedBlockUC.getNumBytes() + " new block " + newBlock
|
|
|
- + " old block " + truncatedBlockUC.getTruncateBlock());
|
|
|
+ NameNode.stateChangeLog.debug(
|
|
|
+ "BLOCK* prepareFileForTruncate: Scheduling copy-on-truncate to new" +
|
|
|
+ " size {} new block {} old block {}", truncatedBlockUC.getNumBytes(),
|
|
|
+ newBlock, truncatedBlockUC.getTruncateBlock());
|
|
|
} else {
|
|
|
// Use new generation stamp for in-place truncate recovery
|
|
|
blockManager.convertLastBlockToUnderConstruction(file, lastBlockDelta);
|
|
@@ -2177,10 +2175,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
truncatedBlockUC.getTruncateBlock().setGenerationStamp(
|
|
|
newBlock.getGenerationStamp());
|
|
|
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* prepareFileForTruncate: "
|
|
|
- + "Scheduling in-place block truncate to new size "
|
|
|
- + truncatedBlockUC.getTruncateBlock().getNumBytes()
|
|
|
- + " block=" + truncatedBlockUC);
|
|
|
+ NameNode.stateChangeLog.debug(
|
|
|
+ "BLOCK* prepareFileForTruncate: {} Scheduling in-place block " +
|
|
|
+ "truncate to new size {}",
|
|
|
+ truncatedBlockUC.getTruncateBlock().getNumBytes(), truncatedBlockUC);
|
|
|
}
|
|
|
if (shouldRecoverNow) {
|
|
|
truncatedBlockUC.initializeBlockRecovery(newBlock.getGenerationStamp());
|
|
@@ -2597,10 +2595,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
|
|
|
// record file record in log, record new generation stamp
|
|
|
getEditLog().logOpenFile(src, newNode, overwrite, logRetryEntry);
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added " +
|
|
|
- src + " inode " + newNode.getId() + " " + holder);
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added {}" +
|
|
|
+ " inode {} holder {}", src, newNode.getId(), holder);
|
|
|
return toRemoveBlocks;
|
|
|
} catch (IOException ie) {
|
|
|
NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: " + src + " " +
|
|
@@ -2961,11 +2957,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
String clientMachine, boolean newBlock, boolean logRetryCache)
|
|
|
throws IOException {
|
|
|
String src = srcArg;
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: src=" + src
|
|
|
- + ", holder=" + holder
|
|
|
- + ", clientMachine=" + clientMachine);
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug(
|
|
|
+ "DIR* NameSystem.appendFile: src={}, holder={}, clientMachine={}",
|
|
|
+ src, holder, clientMachine);
|
|
|
boolean skipSync = false;
|
|
|
if (!supportAppends) {
|
|
|
throw new UnsupportedOperationException(
|
|
@@ -3000,12 +2994,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
}
|
|
|
if (lb != null) {
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: file "
|
|
|
- +src+" for "+holder+" at "+clientMachine
|
|
|
- +" block " + lb.getBlock()
|
|
|
- +" block size " + lb.getBlock().getNumBytes());
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug(
|
|
|
+ "DIR* NameSystem.appendFile: file {} for {} at {} block {} block" +
|
|
|
+ " size {}", src, holder, clientMachine, lb.getBlock(),
|
|
|
+ lb.getBlock().getNumBytes());
|
|
|
}
|
|
|
logAuditEvent(true, "append", srcArg);
|
|
|
return new LastBlockWithStatus(lb, stat);
|
|
@@ -3065,10 +3057,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
Node clientNode = null;
|
|
|
String clientMachine = null;
|
|
|
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: "
|
|
|
- + src + " inodeId " + fileId + " for " + clientName);
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: {} inodeId {}" +
|
|
|
+ " for {}", src, fileId, clientName);
|
|
|
|
|
|
checkOperation(OperationCategory.READ);
|
|
|
byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
@@ -3274,12 +3264,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
lastBlockInFile.getNumBytes() >= pendingFile.getPreferredBlockSize() &&
|
|
|
lastBlockInFile.isComplete()) {
|
|
|
// Case 1
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug(
|
|
|
- "BLOCK* NameSystem.allocateBlock: handling block allocation" +
|
|
|
- " writing to a file with a complete previous block: src=" +
|
|
|
- src + " lastBlock=" + lastBlockInFile);
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug(
|
|
|
+ "BLOCK* NameSystem.allocateBlock: handling block allocation" +
|
|
|
+ " writing to a file with a complete previous block: src={}" +
|
|
|
+ " lastBlock={}", src, lastBlockInFile);
|
|
|
} else if (Block.matchingIdAndGenStamp(penultimateBlock, previousBlock)) {
|
|
|
if (lastBlockInFile.getNumBytes() != 0) {
|
|
|
throw new IOException(
|
|
@@ -3384,10 +3372,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
*/
|
|
|
boolean abandonBlock(ExtendedBlock b, long fileId, String src, String holder)
|
|
|
throws IOException {
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: " + b
|
|
|
- + "of file " + src);
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug(
|
|
|
+ "BLOCK* NameSystem.abandonBlock: {} of file {}", b, src);
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
|
byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
|
|
|
FSPermissionChecker pc = getPermissionChecker();
|
|
@@ -3421,10 +3407,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (!removed) {
|
|
|
return true;
|
|
|
}
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
|
|
|
- + b + " is removed from pendingCreates");
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: {} is " +
|
|
|
+ "removed from pendingCreates", b);
|
|
|
persistBlocks(src, file, false);
|
|
|
} finally {
|
|
|
writeUnlock();
|
|
@@ -3484,10 +3468,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
ExtendedBlock last, long fileId)
|
|
|
throws SafeModeException, UnresolvedLinkException, IOException {
|
|
|
String src = srcArg;
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " +
|
|
|
- src + " for " + holder);
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: {} for {}",
|
|
|
+ src, holder);
|
|
|
checkBlock(last);
|
|
|
boolean success = false;
|
|
|
checkOperation(OperationCategory.WRITE);
|
|
@@ -4622,11 +4604,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
assert hasWriteLock();
|
|
|
Preconditions.checkArgument(file.isUnderConstruction());
|
|
|
getEditLog().logUpdateBlocks(path, file, logRetryCache);
|
|
|
- if(NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("persistBlocks: " + path
|
|
|
- + " with " + file.getBlocks().length + " blocks is persisted to" +
|
|
|
- " the file system");
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug("persistBlocks: {} with {} blocks is" +
|
|
|
+ " peristed to the file system", path, file.getBlocks().length);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -4639,11 +4618,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
waitForLoadingFSImage();
|
|
|
// file is closed
|
|
|
getEditLog().logCloseFile(path, file);
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("closeFile: "
|
|
|
- +path+" with "+ file.getBlocks().length
|
|
|
- +" blocks is persisted to the file system");
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug("closeFile: {} with {} blocks is persisted" +
|
|
|
+ " to the file system", path, file.getBlocks().length);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -5094,11 +5070,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
private void persistNewBlock(String path, INodeFile file) {
|
|
|
Preconditions.checkArgument(file.isUnderConstruction());
|
|
|
getEditLog().logAddBlock(path, file);
|
|
|
- if (NameNode.stateChangeLog.isDebugEnabled()) {
|
|
|
- NameNode.stateChangeLog.debug("persistNewBlock: "
|
|
|
- + path + " with new block " + file.getLastBlock().toString()
|
|
|
- + ", current total block count is " + file.getBlocks().length);
|
|
|
- }
|
|
|
+ NameNode.stateChangeLog.debug("persistNewBlock: {} with new block {}," +
|
|
|
+ " current total block count is {}", path,
|
|
|
+ file.getLastBlock().toString(), file.getBlocks().length);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -6553,7 +6527,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
if (cookieTab[0] == null) {
|
|
|
cookieTab[0] = String.valueOf(getIntCookie(cookieTab[0]));
|
|
|
}
|
|
|
- LOG.info("there are no corrupt file blocks.");
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("there are no corrupt file blocks.");
|
|
|
+ }
|
|
|
return corruptFiles;
|
|
|
}
|
|
|
|
|
@@ -6588,7 +6564,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
|
|
|
}
|
|
|
}
|
|
|
cookieTab[0] = String.valueOf(skip);
|
|
|
- LOG.info("list corrupt file blocks returned: " + count);
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ LOG.debug("list corrupt file blocks returned: " + count);
|
|
|
+ }
|
|
|
return corruptFiles;
|
|
|
} finally {
|
|
|
readUnlock();
|