diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt index af5d1fdd0e3..6d34dd7958b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt @@ -152,6 +152,9 @@ Trunk (Unreleased) HDFS-4110. Refine a log printed in JNStorage. (Liang Xie via suresh) + HDFS-4122. Cleanup HDFS logs and reduce the size of logged messages. + (suresh) + OPTIMIZATIONS BUG FIXES diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 3331b3a27a6..2df1d3dc815 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -652,7 +652,7 @@ public class DFSClient implements java.io.Closeable { // if there is no more clients under the renewer. getLeaseRenewer().closeClient(this); } catch (IOException ioe) { - LOG.info("Exception occurred while aborting the client. " + ioe); + LOG.info("Exception occurred while aborting the client " + ioe); } closeConnectionToNamenode(); } @@ -2111,7 +2111,7 @@ public class DFSClient implements java.io.Closeable { reportBadBlocks(lblocks); } catch (IOException ie) { LOG.info("Found corruption while reading " + file - + ". Error repairing corrupt blocks. Bad blocks remain.", ie); + + ". Error repairing corrupt blocks. Bad blocks remain.", ie); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index f14704053c4..891afa33f33 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -457,7 +457,7 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable buffersize, verifyChecksum, dfsClient.clientName); if(connectFailedOnce) { DFSClient.LOG.info("Successfully connected to " + targetAddr + - " for block " + blk.getBlockId()); + " for " + blk); } return chosenNode; } catch (IOException ex) { @@ -736,9 +736,9 @@ public class DFSInputStream extends FSInputStream implements ByteBufferReadable } if (nodes == null || nodes.length == 0) { - DFSClient.LOG.info("No node available for block: " + blockInfo); + DFSClient.LOG.info("No node available for " + blockInfo); } - DFSClient.LOG.info("Could not obtain block " + block.getBlock() + DFSClient.LOG.info("Could not obtain " + block.getBlock() + " from any node: " + ie + ". Will get new block locations from namenode and retry..."); try { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 8e51d7c0493..9a8769cc40a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -735,7 +735,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable { // private boolean processDatanodeError() throws IOException { if (response != null) { - DFSClient.LOG.info("Error Recovery for block " + block + + DFSClient.LOG.info("Error Recovery for " + block + " waiting for responder to exit. "); return true; } @@ -1008,7 +1008,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable { success = createBlockOutputStream(nodes, 0L, false); if (!success) { - DFSClient.LOG.info("Abandoning block " + block); + DFSClient.LOG.info("Abandoning " + block); dfsClient.namenode.abandonBlock(block, src, dfsClient.clientName); block = null; DFSClient.LOG.info("Excluding datanode " + nodes[errorIndex]); @@ -1773,7 +1773,7 @@ public class DFSOutputStream extends FSOutputSummer implements Syncable { try { Thread.sleep(400); if (Time.now() - localstart > 5000) { - DFSClient.LOG.info("Could not complete file " + src + " retrying..."); + DFSClient.LOG.info("Could not complete " + src + " retrying..."); } } catch (InterruptedException ie) { DFSClient.LOG.warn("Caught exception ", ie); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index 0e30245ce53..1507db72866 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -702,7 +702,7 @@ public class DistributedFileSystem extends FileSystem { } DatanodeInfo[] dataNode = {dfsIn.getCurrentDatanode()}; lblocks[0] = new LocatedBlock(dataBlock, dataNode); - LOG.info("Found checksum error in data stream at block=" + LOG.info("Found checksum error in data stream at " + dataBlock + " on datanode=" + dataNode[0]); @@ -715,7 +715,7 @@ public class DistributedFileSystem extends FileSystem { } DatanodeInfo[] sumsNode = {dfsSums.getCurrentDatanode()}; lblocks[1] = new LocatedBlock(sumsBlock, sumsNode); - LOG.info("Found checksum error in checksum stream at block=" + LOG.info("Found checksum error in checksum stream at " + sumsBlock + " on datanode=" + sumsNode[0]); // Ask client to delete blocks. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java index e7aff0d61e9..b2f344d17a0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java @@ -950,8 +950,8 @@ public class BlockManager { datanodes.append(node).append(" "); } if (datanodes.length() != 0) { - NameNode.stateChangeLog.info("BLOCK* addToInvalidates: " - + b + " to " + datanodes.toString()); + NameNode.stateChangeLog.info("BLOCK* addToInvalidates: " + b + " " + + datanodes); } } @@ -972,7 +972,7 @@ public class BlockManager { // thread of Datanode reports bad block before Block reports are sent // by the Datanode on startup NameNode.stateChangeLog.info("BLOCK* findAndMarkBlockAsCorrupt: " - + blk + " not found."); + + blk + " not found"); return; } markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, reason), dn); @@ -1026,7 +1026,7 @@ public class BlockManager { NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: postponing " + "invalidation of " + b + " on " + dn + " because " + nr.replicasOnStaleNodes() + " replica(s) are located on nodes " + - "with potentially out-of-date block reports."); + "with potentially out-of-date block reports"); postponeBlock(b.corrupted); } else if (nr.liveReplicas() >= 1) { @@ -1039,7 +1039,7 @@ public class BlockManager { } } else { NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: " + b - + " on " + dn + " is the only copy and was not deleted."); + + " on " + dn + " is the only copy and was not deleted"); } } @@ -1160,9 +1160,8 @@ public class BlockManager { (blockHasEnoughRacks(block)) ) { neededReplications.remove(block, priority); // remove from neededReplications neededReplications.decrementReplicationIndex(priority); - NameNode.stateChangeLog.info("BLOCK* " - + "Removing block " + block - + " from neededReplications as it has enough replicas."); + NameNode.stateChangeLog.info("BLOCK* Removing " + block + + " from neededReplications as it has enough replicas"); continue; } } @@ -1236,9 +1235,8 @@ public class BlockManager { neededReplications.remove(block, priority); // remove from neededReplications neededReplications.decrementReplicationIndex(priority); rw.targets = null; - NameNode.stateChangeLog.info("BLOCK* " - + "Removing block " + block - + " from neededReplications as it has enough replicas."); + NameNode.stateChangeLog.info("BLOCK* Removing " + block + + " from neededReplications as it has enough replicas"); continue; } } @@ -1290,10 +1288,8 @@ public class BlockManager { targetList.append(' '); targetList.append(targets[k]); } - NameNode.stateChangeLog.info( - "BLOCK* ask " - + rw.srcNode + " to replicate " - + rw.block + " to " + targetList); + NameNode.stateChangeLog.info("BLOCK* ask " + rw.srcNode + + " to replicate " + rw.block + " to " + targetList); } } } @@ -1527,10 +1523,9 @@ public class BlockManager { boolean staleBefore = node.areBlockContentsStale(); node.receivedBlockReport(); if (staleBefore && !node.areBlockContentsStale()) { - LOG.info("BLOCK* processReport: " + - "Received first block report from " + node + - " after becoming active. Its block contents are no longer" + - " considered stale."); + LOG.info("BLOCK* processReport: Received first block report from " + + node + " after becoming active. Its block contents are no longer" + + " considered stale"); rescanPostponedMisreplicatedBlocks(); } @@ -1601,9 +1596,9 @@ public class BlockManager { addStoredBlock(b, node, null, true); } for (Block b : toInvalidate) { - NameNode.stateChangeLog.info("BLOCK* processReport: block " + NameNode.stateChangeLog.info("BLOCK* processReport: " + b + " on " + node + " size " + b.getNumBytes() - + " does not belong to any file."); + + " does not belong to any file"); addToInvalidates(b, node); } for (BlockToMarkCorrupt b : toCorrupt) { @@ -1870,7 +1865,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block int count = pendingDNMessages.count(); if (count > 0) { LOG.info("Processing " + count + " messages from DataNodes " + - "that were previously queued during standby state."); + "that were previously queued during standby state"); } processQueuedMessages(pendingDNMessages.takeAll()); assert pendingDNMessages.count() == 0; @@ -1927,9 +1922,9 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block // the block report got a little bit delayed after the pipeline // closed. So, ignore this report, assuming we will get a // FINALIZED replica later. See HDFS-2791 - LOG.info("Received an RBW replica for block " + storedBlock + - " on " + dn + ": ignoring it, since the block is " + - "complete with the same generation stamp."); + LOG.info("Received an RBW replica for " + storedBlock + + " on " + dn + ": ignoring it, since it is " + + "complete with the same genstamp"); return null; } else { return new BlockToMarkCorrupt(storedBlock, @@ -2041,7 +2036,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block // If this block does not belong to anyfile, then we are done. NameNode.stateChangeLog.info("BLOCK* addStoredBlock: " + block + " on " + node + " size " + block.getNumBytes() - + " but it does not belong to any file."); + + " but it does not belong to any file"); // we could add this block to invalidate set of this datanode. // it will happen in next block report otherwise. return block; @@ -2158,9 +2153,8 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block try { invalidateBlock(new BlockToMarkCorrupt(blk, null), node); } catch (IOException e) { - NameNode.stateChangeLog.info("NameNode.invalidateCorruptReplicas " + - "error in deleting bad block " + blk + - " on " + node, e); + NameNode.stateChangeLog.info("invalidateCorruptReplicas " + + "error in deleting bad block " + blk + " on " + node, e); gotException = true; } } @@ -2308,7 +2302,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block DatanodeDescriptor cur = it.next(); if (cur.areBlockContentsStale()) { LOG.info("BLOCK* processOverReplicatedBlock: " + - "Postponing processing of over-replicated block " + + "Postponing processing of over-replicated " + block + " since datanode " + cur + " does not yet have up-to-date " + "block information."); postponeBlock(block); @@ -2398,7 +2392,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block // addToInvalidates(b, cur); NameNode.stateChangeLog.info("BLOCK* chooseExcessReplicates: " - +"("+cur+", "+b+") is added to invalidated blocks set."); + +"("+cur+", "+b+") is added to invalidated blocks set"); } } @@ -2540,7 +2534,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block for (Block b : toInvalidate) { NameNode.stateChangeLog.info("BLOCK* addBlock: block " + b + " on " + node + " size " + b.getNumBytes() - + " does not belong to any file."); + + " does not belong to any file"); addToInvalidates(b, node); } for (BlockToMarkCorrupt b : toCorrupt) { @@ -2651,7 +2645,7 @@ assert storedBlock.findDatanode(dn) < 0 : "Block " + block * of live nodes. If in startup safemode (or its 30-sec extension period), * then it gains speed by ignoring issues of excess replicas or nodes * that are decommissioned or in process of becoming decommissioned. - * If not in startup, then it calls {@link countNodes()} instead. + * If not in startup, then it calls {@link #countNodes(Block)} instead. * * @param b - the block being tested * @return count of live nodes for this block diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java index f09e18709b4..a0f445a93b0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java @@ -362,8 +362,7 @@ public class DatanodeDescriptor extends DatanodeInfo { void addBlockToBeRecovered(BlockInfoUnderConstruction block) { if(recoverBlocks.contains(block)) { // this prevents adding the same block twice to the recovery queue - BlockManager.LOG.info("Block " + block + - " is already in the recovery queue."); + BlockManager.LOG.info(block + " is already in the recovery queue"); return; } recoverBlocks.offer(block); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java index 094c7623d0a..a8d31392156 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java @@ -584,7 +584,7 @@ public class DatanodeManager { if (node.isDecommissionInProgress()) { if (!blockManager.isReplicationInProgress(node)) { node.setDecommissioned(); - LOG.info("Decommission complete for node " + node); + LOG.info("Decommission complete for " + node); } } return node.isDecommissioned(); @@ -593,8 +593,8 @@ public class DatanodeManager { /** Start decommissioning the specified datanode. */ private void startDecommission(DatanodeDescriptor node) { if (!node.isDecommissionInProgress() && !node.isDecommissioned()) { - LOG.info("Start Decommissioning node " + node + " with " + - node.numBlocks() + " blocks."); + LOG.info("Start Decommissioning " + node + " with " + + node.numBlocks() + " blocks"); heartbeatManager.startDecommission(node); node.decommissioningStatus.setStartTime(now()); @@ -606,7 +606,7 @@ public class DatanodeManager { /** Stop decommissioning the specified datanodes. */ void stopDecommission(DatanodeDescriptor node) { if (node.isDecommissionInProgress() || node.isDecommissioned()) { - LOG.info("Stop Decommissioning node " + node); + LOG.info("Stop Decommissioning " + node); heartbeatManager.stopDecommission(node); blockManager.processOverReplicatedBlocksOnReCommission(node); } @@ -658,17 +658,15 @@ public class DatanodeManager { throw new DisallowedDatanodeException(nodeReg); } - NameNode.stateChangeLog.info("BLOCK* NameSystem.registerDatanode: " - + "node registration from " + nodeReg - + " storage " + nodeReg.getStorageID()); + 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* NameSystem.registerDatanode: " - + "node from name: " + nodeN); + NameNode.LOG.info("BLOCK* registerDatanode: " + nodeN); // nodeN previously served a different data storage, // which is not served by anybody anymore. removeDatanode(nodeN); @@ -683,8 +681,8 @@ public class DatanodeManager { // 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* NameSystem.registerDatanode: " - + "node restarted."); + NameNode.stateChangeLog.debug("BLOCK* registerDatanode: " + + "node restarted."); } } else { // nodeS is found @@ -696,11 +694,9 @@ public class DatanodeManager { 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* NameSystem.registerDatanode: " - + "node " + nodeS - + " is replaced by " + nodeReg + - " with the same storageID " + - nodeReg.getStorageID()); + NameNode.stateChangeLog.info("BLOCK* registerDatanode: " + nodeS + + " is replaced by " + nodeReg + " with the same storageID " + + nodeReg.getStorageID()); } // update cluster map getNetworkTopology().remove(nodeS); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java index 623d5499185..4a019b73fbe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/Storage.java @@ -433,7 +433,7 @@ public abstract class Storage extends StorageInfo { if (!root.exists()) { // storage directory does not exist if (startOpt != StartupOption.FORMAT) { - LOG.info("Storage directory " + rootPath + " does not exist."); + LOG.info("Storage directory " + rootPath + " does not exist"); return StorageState.NON_EXISTENT; } LOG.info(rootPath + " does not exist. Creating ..."); @@ -442,7 +442,7 @@ public abstract class Storage extends StorageInfo { } // or is inaccessible if (!root.isDirectory()) { - LOG.info(rootPath + "is not a directory."); + LOG.info(rootPath + "is not a directory"); return StorageState.NON_EXISTENT; } if (!root.canWrite()) { @@ -539,34 +539,34 @@ public abstract class Storage extends StorageInfo { switch(curState) { case COMPLETE_UPGRADE: // mv previous.tmp -> previous LOG.info("Completing previous upgrade for storage directory " - + rootPath + "."); + + rootPath); rename(getPreviousTmp(), getPreviousDir()); return; case RECOVER_UPGRADE: // mv previous.tmp -> current LOG.info("Recovering storage directory " + rootPath - + " from previous upgrade."); + + " from previous upgrade"); if (curDir.exists()) deleteDir(curDir); rename(getPreviousTmp(), curDir); return; case COMPLETE_ROLLBACK: // rm removed.tmp LOG.info("Completing previous rollback for storage directory " - + rootPath + "."); + + rootPath); deleteDir(getRemovedTmp()); return; case RECOVER_ROLLBACK: // mv removed.tmp -> current LOG.info("Recovering storage directory " + rootPath - + " from previous rollback."); + + " from previous rollback"); rename(getRemovedTmp(), curDir); return; case COMPLETE_FINALIZE: // rm finalized.tmp LOG.info("Completing previous finalize for storage directory " - + rootPath + "."); + + rootPath); deleteDir(getFinalizedTmp()); return; case COMPLETE_CHECKPOINT: // mv lastcheckpoint.tmp -> previous.checkpoint LOG.info("Completing previous checkpoint for storage directory " - + rootPath + "."); + + rootPath); File prevCkptDir = getPreviousCheckpoint(); if (prevCkptDir.exists()) deleteDir(prevCkptDir); @@ -574,7 +574,7 @@ public abstract class Storage extends StorageInfo { return; case RECOVER_CHECKPOINT: // mv lastcheckpoint.tmp -> current LOG.info("Recovering storage directory " + rootPath - + " from failed checkpoint."); + + " from failed checkpoint"); if (curDir.exists()) deleteDir(curDir); rename(getLastCheckpointTmp(), curDir); @@ -629,7 +629,7 @@ public abstract class Storage extends StorageInfo { FileLock newLock = tryLock(); if (newLock == null) { String msg = "Cannot lock storage " + this.root - + ". The directory is already locked."; + + ". The directory is already locked"; LOG.info(msg); throw new IOException(msg); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java index 4bbada2c029..ad4a78e30fe 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPServiceActor.java @@ -641,8 +641,7 @@ class BPServiceActor implements Runnable { try { Thread.sleep(millis); } catch (InterruptedException ie) { - LOG.info("BPOfferService " + this + - " interrupted while " + stateString); + LOG.info("BPOfferService " + this + " interrupted while " + stateString); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java index e0582e47600..2d618910f7d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceScanner.java @@ -154,7 +154,7 @@ class BlockPoolSliceScanner { } this.scanPeriod = hours * 3600 * 1000; LOG.info("Periodic Block Verification Scanner initialized with interval " - + hours + " hours for block pool " + bpid + "."); + + hours + " hours for block pool " + bpid); // get the list of blocks and arrange them in random order List arr = dataset.getFinalizedBlocks(blockPoolId); @@ -310,12 +310,12 @@ class BlockPoolSliceScanner { } private void handleScanFailure(ExtendedBlock block) { - LOG.info("Reporting bad block " + block); + LOG.info("Reporting bad " + block); try { datanode.reportBadBlocks(block); } catch (IOException ie) { // it is bad, but not bad enough to shutdown the scanner - LOG.warn("Cannot report bad block=" + block.getBlockId()); + LOG.warn("Cannot report bad " + block.getBlockId()); } } @@ -411,7 +411,7 @@ class BlockPoolSliceScanner { // If the block does not exists anymore, then its not an error if (!dataset.contains(block)) { - LOG.info(block + " is no longer in the dataset."); + LOG.info(block + " is no longer in the dataset"); deleteBlock(block.getLocalBlock()); return; } @@ -424,7 +424,7 @@ class BlockPoolSliceScanner { // is a block really deleted by mistake, DirectoryScan should catch it. if (e instanceof FileNotFoundException ) { LOG.info("Verification failed for " + block + - ". It may be due to race with write."); + " - may be due to race with write"); deleteBlock(block.getLocalBlock()); return; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java index 335b2d6e823..da43ad8f9a4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockPoolSliceStorage.java @@ -332,7 +332,7 @@ public class BlockPoolSliceStorage extends Storage { // 4.rename /curernt//previous.tmp to /curernt//previous rename(bpTmpDir, bpPrevDir); LOG.info("Upgrade of block pool " + blockpoolID + " at " + bpSd.getRoot() - + " is complete."); + + " is complete"); } /** @@ -409,7 +409,7 @@ public class BlockPoolSliceStorage extends Storage { // 3. delete removed.tmp dir deleteDir(tmpDir); - LOG.info("Rollback of " + bpSd.getRoot() + " is complete."); + LOG.info("Rollback of " + bpSd.getRoot() + " is complete"); } /* diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java index fa9a014f495..0f1ccb94351 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java @@ -357,7 +357,7 @@ class BlockReceiver implements Closeable { private void handleMirrorOutError(IOException ioe) throws IOException { String bpid = block.getBlockPoolId(); LOG.info(datanode.getDNRegistrationForBP(bpid) - + ":Exception writing block " + block + " to mirror " + mirrorAddr, ioe); + + ":Exception writing " + block + " to mirror " + mirrorAddr, ioe); if (Thread.interrupted()) { // shut down if the thread is interrupted throw ioe; } else { // encounter an error while writing to mirror @@ -379,16 +379,16 @@ class BlockReceiver implements Closeable { LOG.warn("Checksum error in block " + block + " from " + inAddr, ce); if (srcDataNode != null) { try { - LOG.info("report corrupt block " + block + " from datanode " + + LOG.info("report corrupt " + block + " from datanode " + srcDataNode + " to namenode"); datanode.reportRemoteBadBlock(srcDataNode, block); } catch (IOException e) { - LOG.warn("Failed to report bad block " + block + + LOG.warn("Failed to report bad " + block + " from datanode " + srcDataNode + " to namenode"); } } - throw new IOException("Unexpected checksum mismatch " + - "while writing " + block + " from " + inAddr); + throw new IOException("Unexpected checksum mismatch while writing " + + block + " from " + inAddr); } } @@ -518,7 +518,7 @@ class BlockReceiver implements Closeable { // If this is a partial chunk, then read in pre-existing checksum if (firstByteInBlock % bytesPerChecksum != 0) { LOG.info("Packet starts at " + firstByteInBlock + - " for block " + block + + " for " + block + " which is not a multiple of bytesPerChecksum " + bytesPerChecksum); long offsetInChecksum = BlockMetadataHeader.getHeaderSize() + @@ -662,7 +662,7 @@ class BlockReceiver implements Closeable { } } catch (IOException ioe) { - LOG.info("Exception in receiveBlock for " + block, ioe); + LOG.info("Exception for " + block, ioe); throw ioe; } finally { if (!responderClosed) { // Abnormal termination of the flow above @@ -733,10 +733,9 @@ class BlockReceiver implements Closeable { int checksumSize = diskChecksum.getChecksumSize(); blkoff = blkoff - sizePartialChunk; LOG.info("computePartialChunkCrc sizePartialChunk " + - sizePartialChunk + - " block " + block + - " offset in block " + blkoff + - " offset in metafile " + ckoff); + sizePartialChunk + " " + block + + " block offset " + blkoff + + " metafile offset " + ckoff); // create an input stream from the block file // and read in partial crc chunk into temporary buffer @@ -758,7 +757,7 @@ class BlockReceiver implements Closeable { partialCrc = DataChecksum.newDataChecksum( diskChecksum.getChecksumType(), diskChecksum.getBytesPerChecksum()); partialCrc.update(buf, 0, sizePartialChunk); - LOG.info("Read in partial CRC chunk from disk for block " + block); + LOG.info("Read in partial CRC chunk from disk for " + block); // paranoia! verify that the pre-computed crc matches what we // recalculated just now @@ -973,7 +972,7 @@ class BlockReceiver implements Closeable { "HDFS_WRITE", clientname, offset, dnR.getStorageID(), block, endTime-startTime)); } else { - LOG.info("Received block " + block + " of size " + LOG.info("Received " + block + " size " + block.getNumBytes() + " from " + inAddr); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java index fe68c4076d8..49eb7dc3acc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BlockSender.java @@ -503,7 +503,7 @@ class BlockSender implements java.io.Closeable { * part of a block and then decides not to read the rest (but leaves * the socket open). */ - LOG.info("BlockSender.sendChunks() exception: ", e); + LOG.info("exception: ", e); } else { /* Exception while writing to the client. Connection closure from * the other end is mostly the case and we do not care much about diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index b64c41b1ca7..ab6551f57f5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -481,8 +481,7 @@ public class DataNode extends Configured blockScanner = new DataBlockScanner(this, data, conf); blockScanner.start(); } else { - LOG.info("Periodic Block Verification scan is disabled because " + - reason + "."); + LOG.info("Periodic Block Verification scan disabled because " + reason); } } @@ -511,7 +510,7 @@ public class DataNode extends Configured directoryScanner.start(); } else { LOG.info("Periodic Directory Tree Verification scan is disabled because " + - reason + "."); + reason); } } @@ -1262,7 +1261,7 @@ public class DataNode extends Configured xfersBuilder.append(xferTargets[i]); xfersBuilder.append(" "); } - LOG.info(bpReg + " Starting thread to transfer block " + + LOG.info(bpReg + " Starting thread to transfer " + block + " to " + xfersBuilder); } @@ -2049,7 +2048,7 @@ public class DataNode extends Configured ExtendedBlock block = rb.getBlock(); DatanodeInfo[] targets = rb.getLocations(); - LOG.info(who + " calls recoverBlock(block=" + block + LOG.info(who + " calls recoverBlock(" + block + ", targets=[" + Joiner.on(", ").join(targets) + "]" + ", newGenerationStamp=" + rb.getNewGenerationStamp() + ")"); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java index b0675ef09cb..870d71f1592 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataStorage.java @@ -155,11 +155,11 @@ public class DataStorage extends Storage { break; case NON_EXISTENT: // ignore this storage - LOG.info("Storage directory " + dataDir + " does not exist."); + LOG.info("Storage directory " + dataDir + " does not exist"); it.remove(); continue; case NOT_FORMATTED: // format - LOG.info("Storage directory " + dataDir + " is not formatted."); + LOG.info("Storage directory " + dataDir + " is not formatted"); LOG.info("Formatting ..."); format(sd, nsInfo); break; @@ -482,7 +482,7 @@ public class DataStorage extends Storage { // 5. Rename /previous.tmp to /previous rename(tmpDir, prevDir); - LOG.info("Upgrade of " + sd.getRoot()+ " is complete."); + LOG.info("Upgrade of " + sd.getRoot()+ " is complete"); addBlockPoolStorage(nsInfo.getBlockPoolID(), bpStorage); } @@ -556,7 +556,7 @@ public class DataStorage extends Storage { rename(prevDir, curDir); // delete tmp dir deleteDir(tmpDir); - LOG.info("Rollback of " + sd.getRoot() + " is complete."); + LOG.info("Rollback of " + sd.getRoot() + " is complete"); } /** @@ -596,9 +596,9 @@ public class DataStorage extends Storage { deleteDir(bbwDir); } } catch(IOException ex) { - LOG.error("Finalize upgrade for " + dataDirPath + " failed.", ex); + LOG.error("Finalize upgrade for " + dataDirPath + " failed", ex); } - LOG.info("Finalize upgrade for " + dataDirPath + " is complete."); + LOG.info("Finalize upgrade for " + dataDirPath + " is complete"); } @Override public String toString() { return "Finalize " + dataDirPath; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java index 5669d8bf90c..31b896caf93 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java @@ -170,7 +170,7 @@ class DataXceiver extends Receiver implements Runnable { } catch (InvalidMagicNumberException imne) { LOG.info("Failed to read expected encryption handshake from client " + "at " + s.getInetAddress() + ". Perhaps the client is running an " + - "older version of Hadoop which does not support encryption."); + "older version of Hadoop which does not support encryption"); return; } input = encryptedStreams.in; @@ -367,9 +367,8 @@ class DataXceiver extends Receiver implements Runnable { // make a copy here. final ExtendedBlock originalBlock = new ExtendedBlock(block); block.setNumBytes(dataXceiverServer.estimateBlockSize); - LOG.info("Receiving block " + block + - " src: " + remoteAddress + - " dest: " + localAddress); + LOG.info("Receiving " + block + " src: " + remoteAddress + " dest: " + + localAddress); // reply to upstream datanode or client final DataOutputStream replyOut = new DataOutputStream( @@ -478,9 +477,9 @@ class DataXceiver extends Receiver implements Runnable { block + " to mirror " + mirrorNode + ": " + e); throw e; } else { - LOG.info(datanode + ":Exception transfering block " + + LOG.info(datanode + ":Exception transfering " + block + " to mirror " + mirrorNode + - ". continuing without the mirror.", e); + "- continuing without the mirror", e); } } } @@ -528,10 +527,8 @@ class DataXceiver extends Receiver implements Runnable { if (isDatanode || stage == BlockConstructionStage.PIPELINE_CLOSE_RECOVERY) { datanode.closeBlock(block, DataNode.EMPTY_DEL_HINT); - LOG.info("Received block " + block + - " src: " + remoteAddress + - " dest: " + localAddress + - " of size " + block.getNumBytes()); + LOG.info("Received " + block + " src: " + remoteAddress + " dest: " + + localAddress + " of size " + block.getNumBytes()); } @@ -674,7 +671,7 @@ class DataXceiver extends Receiver implements Runnable { datanode.metrics.incrBytesRead((int) read); datanode.metrics.incrBlocksRead(); - LOG.info("Copied block " + block + " to " + s.getRemoteSocketAddress()); + LOG.info("Copied " + block + " to " + s.getRemoteSocketAddress()); } catch (IOException ioe) { isOpSuccess = false; LOG.info("opCopyBlock " + block + " received exception " + ioe); @@ -797,8 +794,7 @@ class DataXceiver extends Receiver implements Runnable { // notify name node datanode.notifyNamenodeReceivedBlock(block, delHint); - LOG.info("Moved block " + block + - " from " + s.getRemoteSocketAddress()); + LOG.info("Moved " + block + " from " + s.getRemoteSocketAddress()); } catch (IOException ioe) { opStatus = ERROR; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java index 0f171805ab6..806921d2263 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetAsyncDiskService.java @@ -136,7 +136,7 @@ class FsDatasetAsyncDiskService { if (executors == null) { LOG.warn("AsyncDiskService has already shut down."); } else { - LOG.info("Shutting down all async disk service threads..."); + LOG.info("Shutting down all async disk service threads"); for (Map.Entry e : executors.entrySet()) { e.getValue().shutdown(); @@ -144,7 +144,7 @@ class FsDatasetAsyncDiskService { // clear the executor map so that calling execute again will fail. executors = null; - LOG.info("All async disk service threads have been shut down."); + LOG.info("All async disk service threads have been shut down"); } } @@ -154,7 +154,7 @@ class FsDatasetAsyncDiskService { */ void deleteAsync(FsVolumeImpl volume, File blockFile, File metaFile, ExtendedBlock block) { - LOG.info("Scheduling block " + block.getLocalBlock() + LOG.info("Scheduling " + block.getLocalBlock() + " file " + blockFile + " for deletion"); ReplicaFileDeleteTask deletionTask = new ReplicaFileDeleteTask( volume, blockFile, metaFile, block); @@ -198,8 +198,8 @@ class FsDatasetAsyncDiskService { datanode.notifyNamenodeDeletedBlock(block); } volume.decDfsUsed(block.getBlockPoolId(), dfsBytes); - LOG.info("Deleted block " + block.getBlockPoolId() + " " - + block.getLocalBlock() + " at file " + blockFile); + LOG.info("Deleted " + block.getBlockPoolId() + " " + + block.getLocalBlock() + " file " + blockFile); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java index 0f9172247ba..a0a7347ea88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/fsdataset/impl/FsDatasetImpl.java @@ -425,7 +425,7 @@ class FsDatasetImpl implements FsDatasetSpi { return; } if (newlen > oldlen) { - throw new IOException("Cannout truncate block to from oldlen (=" + oldlen + throw new IOException("Cannot truncate block to from oldlen (=" + oldlen + ") to newlen (=" + newlen + ")"); } @@ -481,7 +481,7 @@ class FsDatasetImpl implements FsDatasetSpi { " should be greater than the replica " + b + "'s generation stamp"); } ReplicaInfo replicaInfo = getReplicaInfo(b); - LOG.info("Appending to replica " + replicaInfo); + LOG.info("Appending to " + replicaInfo); if (replicaInfo.getState() != ReplicaState.FINALIZED) { throw new ReplicaNotFoundException( ReplicaNotFoundException.UNFINALIZED_REPLICA + b); @@ -689,7 +689,7 @@ class FsDatasetImpl implements FsDatasetSpi { public synchronized ReplicaInPipeline recoverRbw(ExtendedBlock b, long newGS, long minBytesRcvd, long maxBytesRcvd) throws IOException { - LOG.info("Recover the RBW replica " + b); + LOG.info("Recover RBW replica " + b); ReplicaInfo replicaInfo = getReplicaInfo(b.getBlockPoolId(), b.getBlockId()); @@ -700,7 +700,7 @@ class FsDatasetImpl implements FsDatasetSpi { } ReplicaBeingWritten rbw = (ReplicaBeingWritten)replicaInfo; - LOG.info("Recovering replica " + rbw); + LOG.info("Recovering " + rbw); // Stop the previous writer rbw.stopWriter(); @@ -736,8 +736,8 @@ class FsDatasetImpl implements FsDatasetSpi { final long blockId = b.getBlockId(); final long expectedGs = b.getGenerationStamp(); final long visible = b.getNumBytes(); - LOG.info("Convert replica " + b - + " from Temporary to RBW, visible length=" + visible); + LOG.info("Convert " + b + " from Temporary to RBW, visible length=" + + visible); final ReplicaInPipeline temp; { @@ -1415,8 +1415,7 @@ class FsDatasetImpl implements FsDatasetSpi { static ReplicaRecoveryInfo initReplicaRecovery(String bpid, ReplicaMap map, Block block, long recoveryId) throws IOException { final ReplicaInfo replica = map.get(bpid, block.getBlockId()); - LOG.info("initReplicaRecovery: block=" + block - + ", recoveryId=" + recoveryId + LOG.info("initReplicaRecovery: " + block + ", recoveryId=" + recoveryId + ", replica=" + replica); //check replica @@ -1485,7 +1484,7 @@ class FsDatasetImpl implements FsDatasetSpi { //get replica final String bpid = oldBlock.getBlockPoolId(); final ReplicaInfo replica = volumeMap.get(bpid, oldBlock.getBlockId()); - LOG.info("updateReplica: block=" + oldBlock + LOG.info("updateReplica: " + oldBlock + ", recoveryId=" + recoveryId + ", length=" + newlength + ", replica=" + replica); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java index f47a7c299ab..19955075ab6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java @@ -140,7 +140,7 @@ public class FSDirectory implements Closeable { DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_KEY, DFSConfigKeys.DFS_NAMENODE_NAME_CACHE_THRESHOLD_DEFAULT); NameNode.LOG.info("Caching file names occuring more than " + threshold - + " times "); + + " times"); nameCache = new NameCache(threshold); namesystem = ns; } @@ -253,15 +253,12 @@ public class FSDirectory implements Closeable { writeUnlock(); } if (newNode == null) { - NameNode.stateChangeLog.info("DIR* FSDirectory.addFile: " - +"failed to add "+path - +" to the file system"); + NameNode.stateChangeLog.info("DIR* addFile: failed to add " + path); return null; } if(NameNode.stateChangeLog.isDebugEnabled()) { - NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: " - +path+" is added to the file system"); + NameNode.stateChangeLog.debug("DIR* addFile: " + path + " is added"); } return newNode; } @@ -2119,16 +2116,13 @@ public class FSDirectory implements Closeable { writeUnlock(); } if (newNode == null) { - NameNode.stateChangeLog.info("DIR* FSDirectory.addSymlink: " - +"failed to add "+path - +" to the file system"); + NameNode.stateChangeLog.info("DIR* addSymlink: failed to add " + path); return null; } fsImage.getEditLog().logSymlink(path, target, modTime, modTime, newNode); if(NameNode.stateChangeLog.isDebugEnabled()) { - NameNode.stateChangeLog.debug("DIR* FSDirectory.addSymlink: " - +path+" is added to the file system"); + NameNode.stateChangeLog.debug("DIR* addSymlink: " + path + " is added"); } return newNode; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java index db8a08509e2..129ae1592bf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java @@ -900,7 +900,7 @@ public class FSEditLog implements LogsPurgeable { * in the new log. */ synchronized long rollEditLog() throws IOException { - LOG.info("Rolling edit logs."); + LOG.info("Rolling edit logs"); endCurrentLogSegment(true); long nextTxId = getLastWrittenTxId() + 1; @@ -915,7 +915,7 @@ public class FSEditLog implements LogsPurgeable { */ public synchronized void startLogSegment(long txid, boolean abortCurrentLogSegment) throws IOException { - LOG.info("Namenode started a new log segment at txid " + txid); + LOG.info("Started a new log segment at txid " + txid); if (isSegmentOpen()) { if (getLastWrittenTxId() == txid - 1) { //In sync with the NN, so end and finalize the current segment` diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java index fbb1c271b30..0c5e44eda22 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java @@ -90,7 +90,7 @@ public class FSEditLogLoader { expectedStartingTxId, recovery); FSImage.LOG.info("Edits file " + edits.getName() + " of size " + edits.length() + " edits # " + numEdits - + " loaded in " + (now()-startTime)/1000 + " seconds."); + + " loaded in " + (now()-startTime)/1000 + " seconds"); return numEdits; } finally { edits.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java index 64dc27b9634..02b00db836f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java @@ -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(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetDelegationTokenServlet.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetDelegationTokenServlet.java index 46dd25fd2de..55bca6f3f6d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetDelegationTokenServlet.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/GetDelegationTokenServlet.java @@ -77,7 +77,7 @@ public class GetDelegationTokenServlet extends DfsServlet { }); } catch(Exception e) { - LOG.info("Exception while sending token. Re-throwing. ", e); + LOG.info("Exception while sending token. Re-throwing ", e); resp.sendError(HttpServletResponse.SC_INTERNAL_SERVER_ERROR); } finally { if(dos != null) dos.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java index b74e61f85b3..4dbee88d6fa 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java @@ -429,7 +429,7 @@ public class LeaseManager { return; } - LOG.info("Lease " + oldest + " has expired hard limit"); + LOG.info(oldest + " has expired hard limit"); final List removing = new ArrayList(); // need to create a copy of the oldest lease paths, becuase @@ -441,15 +441,14 @@ public class LeaseManager { for(String p : leasePaths) { try { if(fsnamesystem.internalReleaseLease(oldest, p, HdfsServerConstants.NAMENODE_LEASE_HOLDER)) { - LOG.info("Lease recovery for file " + p + - " is complete. File closed."); + LOG.info("Lease recovery for " + p + " is complete. File closed."); removing.add(p); } else { - LOG.info("Started block recovery for file " + p + - " lease " + oldest); + LOG.info("Started block recovery " + p + " lease " + oldest); } } catch (IOException e) { - LOG.error("Cannot release the path "+p+" in the lease "+oldest, e); + LOG.error("Cannot release the path " + p + " in the lease " + + oldest, e); removing.add(p); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/MetaRecoveryContext.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/MetaRecoveryContext.java index b4bd119eb58..80312b8856e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/MetaRecoveryContext.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/MetaRecoveryContext.java @@ -102,7 +102,7 @@ public final class MetaRecoveryContext { "without prompting. " + "(c/s/q/a)\n", "c", "s", "q", "a"); if (answer.equals("c")) { - LOG.info("Continuing."); + LOG.info("Continuing"); return; } else if (answer.equals("s")) { throw new RequestStopException("user requested stop");