Browse Source

HDFS-4122. Cleanup HDFS logs and reduce the size of logged messages. Contributed by Suresh Srinivas.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-1@1403148 13f79535-47bb-0310-9956-ffa450edef68
Suresh Srinivas 12 years ago
parent
commit
4076f5a445

+ 3 - 0
CHANGES.txt

@@ -110,6 +110,9 @@ Release 1.2.0 - unreleased
     HADOOP-8968. Add a flag to completely disable the worker version check.
     HADOOP-8968. Add a flag to completely disable the worker version check.
     (tucu via eli)
     (tucu via eli)
 
 
+    HDFS-4122. Cleanup HDFS logs and reduce the size of logged messages.
+    (suresh)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-2533. Backport: Remove needless synchronization on some FSDataSet
     HDFS-2533. Backport: Remove needless synchronization on some FSDataSet

+ 8 - 8
src/hdfs/org/apache/hadoop/hdfs/DFSClient.java

@@ -2036,7 +2036,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
             serverSupportsHdfs200 = false;
             serverSupportsHdfs200 = false;
           } else {
           } else {
             LOG.info("Failed to get block info from "
             LOG.info("Failed to get block info from "
-                + datanode.getHostName() + " probably does not have block "
+                + datanode.getHostName() + " probably does not have "
                 + last.getBlock(), e);
                 + last.getBlock(), e);
           }
           }
         } finally {
         } finally {
@@ -2247,7 +2247,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
               fetchBlockAt(target);
               fetchBlockAt(target);
               continue;
               continue;
             } else {
             } else {
-              LOG.info("Failed to read block " + targetBlock.getBlock()
+              LOG.info("Failed to read " + targetBlock.getBlock()
                   + " on local machine" + StringUtils.stringifyException(ex));
                   + " on local machine" + StringUtils.stringifyException(ex));
               LOG.info("Try reading via the datanode on " + targetAddr);
               LOG.info("Try reading via the datanode on " + targetAddr);
             }
             }
@@ -2435,9 +2435,9 @@ public class DFSClient implements FSConstants, java.io.Closeable {
           }
           }
           
           
           if (nodes == null || nodes.length == 0) {
           if (nodes == null || nodes.length == 0) {
-            LOG.info("No node available for block: " + blockInfo);
+            LOG.info("No node available for: " + blockInfo);
           }
           }
-          LOG.info("Could not obtain block " + block.getBlock()
+          LOG.info("Could not obtain " + block.getBlock()
               + " from any node: " + ie
               + " from any node: " + ie
               + ". Will get new block locations from namenode and retry...");
               + ". Will get new block locations from namenode and retry...");
           try {
           try {
@@ -3211,12 +3211,12 @@ public class DFSClient implements FSConstants, java.io.Closeable {
         return false;
         return false;
       }
       }
       if (response != null) {
       if (response != null) {
-        LOG.info("Error Recovery for block " + block +
+        LOG.info("Error Recovery for " + block +
                  " waiting for responder to exit. ");
                  " waiting for responder to exit. ");
         return true;
         return true;
       }
       }
       if (errorIndex >= 0) {
       if (errorIndex >= 0) {
-        LOG.warn("Error Recovery for block " + block
+        LOG.warn("Error Recovery for " + block
             + " bad datanode[" + errorIndex + "] "
             + " bad datanode[" + errorIndex + "] "
             + (nodes == null? "nodes == null": nodes[errorIndex].getName()));
             + (nodes == null? "nodes == null": nodes[errorIndex].getName()));
       }
       }
@@ -3562,7 +3562,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
         success = createBlockOutputStream(nodes, clientName, false);
         success = createBlockOutputStream(nodes, clientName, false);
 
 
         if (!success) {
         if (!success) {
-          LOG.info("Abandoning block " + block);
+          LOG.info("Abandoning " + block);
           namenode.abandonBlock(block, src, clientName);
           namenode.abandonBlock(block, src, clientName);
 
 
           if (errorIndex < nodes.length) {
           if (errorIndex < nodes.length) {
@@ -4083,7 +4083,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
             try {
             try {
               Thread.sleep(400);
               Thread.sleep(400);
               if (System.currentTimeMillis() - localstart > 5000) {
               if (System.currentTimeMillis() - localstart > 5000) {
-                LOG.info("Could not complete file " + src + " retrying...");
+                LOG.info("Could not complete " + src + " retrying...");
               }
               }
             } catch (InterruptedException ie) {
             } catch (InterruptedException ie) {
             }
             }

+ 3 - 4
src/hdfs/org/apache/hadoop/hdfs/server/balancer/Balancer.java

@@ -338,8 +338,7 @@ public class Balancer implements Tool {
         LOG.info( "Moving block " + block.getBlock().getBlockId() +
         LOG.info( "Moving block " + block.getBlock().getBlockId() +
               " from "+ source.getName() + " to " +
               " from "+ source.getName() + " to " +
               target.getName() + " through " +
               target.getName() + " through " +
-              proxySource.getName() +
-              " is succeeded." );
+              proxySource.getName() + " is succeeded." );
       } catch (IOException e) {
       } catch (IOException e) {
         LOG.warn("Error moving block "+block.getBlockId()+
         LOG.warn("Error moving block "+block.getBlockId()+
             " from " + source.getName() + " to " +
             " from " + source.getName() + " to " +
@@ -1515,8 +1514,8 @@ public class Balancer implements Tool {
           System.out.println("The cluster is balanced. Exiting...");
           System.out.println("The cluster is balanced. Exiting...");
           return SUCCESS;
           return SUCCESS;
         } else {
         } else {
-          LOG.info( "Need to move "+ StringUtils.byteDesc(bytesLeftToMove)
-              +" bytes to make the cluster balanced." );
+          LOG.info("Need to move "+ StringUtils.byteDesc(bytesLeftToMove)
+              +" bytes to make the cluster balanced" );
         }
         }
         
         
         /* Decide all the nodes that will participate in the block move and
         /* Decide all the nodes that will participate in the block move and

+ 10 - 10
src/hdfs/org/apache/hadoop/hdfs/server/common/Storage.java

@@ -428,16 +428,16 @@ public abstract class Storage extends StorageInfo {
         if (!root.exists()) {
         if (!root.exists()) {
           // storage directory does not exist
           // storage directory does not exist
           if (startOpt != StartupOption.FORMAT) {
           if (startOpt != StartupOption.FORMAT) {
-            LOG.info("Storage directory " + rootPath + " does not exist.");
+            LOG.info("Storage directory " + rootPath + " does not exist");
             return StorageState.NON_EXISTENT;
             return StorageState.NON_EXISTENT;
           }
           }
-          LOG.info(rootPath + " does not exist. Creating ...");
+          LOG.info(rootPath + " does not exist. Creating...");
           if (!root.mkdirs())
           if (!root.mkdirs())
             throw new IOException("Cannot create directory " + rootPath);
             throw new IOException("Cannot create directory " + rootPath);
         }
         }
         // or is inaccessible
         // or is inaccessible
         if (!root.isDirectory()) {
         if (!root.isDirectory()) {
-          LOG.info(rootPath + "is not a directory.");
+          LOG.info(rootPath + "is not a directory");
           return StorageState.NON_EXISTENT;
           return StorageState.NON_EXISTENT;
         }
         }
         if (!root.canWrite()) {
         if (!root.canWrite()) {
@@ -534,34 +534,34 @@ public abstract class Storage extends StorageInfo {
       switch(curState) {
       switch(curState) {
       case COMPLETE_UPGRADE:  // mv previous.tmp -> previous
       case COMPLETE_UPGRADE:  // mv previous.tmp -> previous
         LOG.info("Completing previous upgrade for storage directory " 
         LOG.info("Completing previous upgrade for storage directory " 
-                 + rootPath + ".");
+                 + rootPath);
         rename(getPreviousTmp(), getPreviousDir());
         rename(getPreviousTmp(), getPreviousDir());
         return;
         return;
       case RECOVER_UPGRADE:   // mv previous.tmp -> current
       case RECOVER_UPGRADE:   // mv previous.tmp -> current
         LOG.info("Recovering storage directory " + rootPath
         LOG.info("Recovering storage directory " + rootPath
-                 + " from previous upgrade.");
+                 + " from previous upgrade");
         if (curDir.exists())
         if (curDir.exists())
           deleteDir(curDir);
           deleteDir(curDir);
         rename(getPreviousTmp(), curDir);
         rename(getPreviousTmp(), curDir);
         return;
         return;
       case COMPLETE_ROLLBACK: // rm removed.tmp
       case COMPLETE_ROLLBACK: // rm removed.tmp
         LOG.info("Completing previous rollback for storage directory "
         LOG.info("Completing previous rollback for storage directory "
-                 + rootPath + ".");
+                 + rootPath);
         deleteDir(getRemovedTmp());
         deleteDir(getRemovedTmp());
         return;
         return;
       case RECOVER_ROLLBACK:  // mv removed.tmp -> current
       case RECOVER_ROLLBACK:  // mv removed.tmp -> current
         LOG.info("Recovering storage directory " + rootPath
         LOG.info("Recovering storage directory " + rootPath
-                 + " from previous rollback.");
+                 + " from previous rollback");
         rename(getRemovedTmp(), curDir);
         rename(getRemovedTmp(), curDir);
         return;
         return;
       case COMPLETE_FINALIZE: // rm finalized.tmp
       case COMPLETE_FINALIZE: // rm finalized.tmp
         LOG.info("Completing previous finalize for storage directory "
         LOG.info("Completing previous finalize for storage directory "
-                 + rootPath + ".");
+                 + rootPath);
         deleteDir(getFinalizedTmp());
         deleteDir(getFinalizedTmp());
         return;
         return;
       case COMPLETE_CHECKPOINT: // mv lastcheckpoint.tmp -> previous.checkpoint
       case COMPLETE_CHECKPOINT: // mv lastcheckpoint.tmp -> previous.checkpoint
         LOG.info("Completing previous checkpoint for storage directory " 
         LOG.info("Completing previous checkpoint for storage directory " 
-                 + rootPath + ".");
+                 + rootPath);
         File prevCkptDir = getPreviousCheckpoint();
         File prevCkptDir = getPreviousCheckpoint();
         if (prevCkptDir.exists())
         if (prevCkptDir.exists())
           deleteDir(prevCkptDir);
           deleteDir(prevCkptDir);
@@ -569,7 +569,7 @@ public abstract class Storage extends StorageInfo {
         return;
         return;
       case RECOVER_CHECKPOINT:  // mv lastcheckpoint.tmp -> current
       case RECOVER_CHECKPOINT:  // mv lastcheckpoint.tmp -> current
         LOG.info("Recovering storage directory " + rootPath
         LOG.info("Recovering storage directory " + rootPath
-                 + " from failed checkpoint.");
+                 + " from failed checkpoint");
         if (curDir.exists())
         if (curDir.exists())
           deleteDir(curDir);
           deleteDir(curDir);
         rename(getLastCheckpointTmp(), curDir);
         rename(getLastCheckpointTmp(), curDir);

+ 13 - 17
src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockReceiver.java

@@ -201,7 +201,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
    * affect this datanode.
    * affect this datanode.
    */
    */
   private void handleMirrorOutError(IOException ioe) throws IOException {
   private void handleMirrorOutError(IOException ioe) throws IOException {
-    LOG.info(datanode.dnRegistration + ": Exception writing block " +
+    LOG.info(datanode.dnRegistration + ": Exception writing " +
              block + " to mirror " + mirrorAddr + "\n" +
              block + " to mirror " + mirrorAddr + "\n" +
              StringUtils.stringifyException(ioe));
              StringUtils.stringifyException(ioe));
     if (Thread.interrupted()) { // shut down if the thread is interrupted
     if (Thread.interrupted()) { // shut down if the thread is interrupted
@@ -228,13 +228,13 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
       if (!checksum.compare(checksumBuf, checksumOff)) {
       if (!checksum.compare(checksumBuf, checksumOff)) {
         if (srcDataNode != null) {
         if (srcDataNode != null) {
           try {
           try {
-            LOG.info("report corrupt block " + block + " from datanode " +
+            LOG.info("report corrupt " + block + " from datanode " +
                       srcDataNode + " to namenode");
                       srcDataNode + " to namenode");
             LocatedBlock lb = new LocatedBlock(block, 
             LocatedBlock lb = new LocatedBlock(block, 
                                             new DatanodeInfo[] {srcDataNode});
                                             new DatanodeInfo[] {srcDataNode});
             datanode.namenode.reportBadBlocks(new LocatedBlock[] {lb});
             datanode.namenode.reportBadBlocks(new LocatedBlock[] {lb});
           } catch (IOException e) {
           } catch (IOException e) {
-            LOG.warn("Failed to report bad block " + block + 
+            LOG.warn("Failed to report bad " + block + 
                       " from datanode " + srcDataNode + " to namenode");
                       " from datanode " + srcDataNode + " to namenode");
           }
           }
         }
         }
@@ -410,7 +410,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     buf.reset();
     buf.reset();
     
     
     if (LOG.isDebugEnabled()){
     if (LOG.isDebugEnabled()){
-      LOG.debug("Receiving one packet for block " + block +
+      LOG.debug("Receiving one packet for " + block +
                 " of length " + payloadLen +
                 " of length " + payloadLen +
                 " seqno " + seqno +
                 " seqno " + seqno +
                 " offsetInBlock " + offsetInBlock +
                 " offsetInBlock " + offsetInBlock +
@@ -439,7 +439,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     } 
     } 
 
 
     if (len == 0) {
     if (len == 0) {
-      LOG.debug("Receiving empty packet for block " + block);
+      LOG.debug("Receiving empty packet for " + block);
     } else {
     } else {
       offsetInBlock += len;
       offsetInBlock += len;
 
 
@@ -603,8 +603,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
       }
       }
 
 
     } catch (IOException ioe) {
     } catch (IOException ioe) {
-      LOG.info("Exception in receiveBlock for block " + block + 
-               " " + ioe);
+      LOG.info("Exception in receiveBlock for " + block + " " + ioe);
       IOUtils.closeStream(this);
       IOUtils.closeStream(this);
       if (responder != null) {
       if (responder != null) {
         responder.interrupt();
         responder.interrupt();
@@ -666,8 +665,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     // If this is a partial chunk, then read in pre-existing checksum
     // If this is a partial chunk, then read in pre-existing checksum
     if (offsetInBlock % bytesPerChecksum != 0) {
     if (offsetInBlock % bytesPerChecksum != 0) {
       LOG.info("setBlockPosition trying to set position to " +
       LOG.info("setBlockPosition trying to set position to " +
-               offsetInBlock +
-               " for block " + block +
+               offsetInBlock + " for " + block +
                " which is not a multiple of bytesPerChecksum " +
                " which is not a multiple of bytesPerChecksum " +
                bytesPerChecksum);
                bytesPerChecksum);
       computePartialChunkCrc(offsetInBlock, offsetInChecksum, bytesPerChecksum);
       computePartialChunkCrc(offsetInBlock, offsetInChecksum, bytesPerChecksum);
@@ -697,8 +695,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     int checksumSize = checksum.getChecksumSize();
     int checksumSize = checksum.getChecksumSize();
     blkoff = blkoff - sizePartialChunk;
     blkoff = blkoff - sizePartialChunk;
     LOG.info("computePartialChunkCrc sizePartialChunk " + 
     LOG.info("computePartialChunkCrc sizePartialChunk " + 
-              sizePartialChunk +
-              " block " + block +
+              sizePartialChunk + " " + block +
               " offset in block " + blkoff +
               " offset in block " + blkoff +
               " offset in metafile " + ckoff);
               " offset in metafile " + ckoff);
 
 
@@ -721,7 +718,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     // compute crc of partial chunk from data read in the block file.
     // compute crc of partial chunk from data read in the block file.
     partialCrc = new PureJavaCrc32();
     partialCrc = new PureJavaCrc32();
     partialCrc.update(buf, 0, sizePartialChunk);
     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
     // paranoia! verify that the pre-computed crc matches what we
     // recalculated just now
     // recalculated just now
@@ -754,7 +751,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
     private Thread receiverThread; // the thread that spawns this responder
     private Thread receiverThread; // the thread that spawns this responder
 
 
     public String toString() {
     public String toString() {
-      return "PacketResponder " + numTargets + " for Block " + this.block;
+      return "PacketResponder " + numTargets + " for " + this.block;
     }
     }
 
 
     PacketResponder(BlockReceiver receiver, Block b, DataInputStream in, 
     PacketResponder(BlockReceiver receiver, Block b, DataInputStream in, 
@@ -906,8 +903,7 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
                       "HDFS_WRITE", receiver.clientName, offset, 
                       "HDFS_WRITE", receiver.clientName, offset, 
                       datanode.dnRegistration.getStorageID(), block, endTime-startTime));
                       datanode.dnRegistration.getStorageID(), block, endTime-startTime));
               } else {
               } else {
-                LOG.info("Received block " + block + 
-                         " of size " + block.getNumBytes() + 
+                LOG.info("Received " + block + " of size " + block.getNumBytes() +
                          " from " + receiver.inAddr);
                          " from " + receiver.inAddr);
               }
               }
             }
             }
@@ -948,8 +944,8 @@ class BlockReceiver implements java.io.Closeable, FSConstants {
           }
           }
         }
         }
       }
       }
-      LOG.info("PacketResponder " + numTargets + 
-               " for block " + block + " terminating");
+      LOG.info("PacketResponder " + numTargets + " for " + block +
+          " terminating");
     }
     }
   }
   }
   
   

+ 1 - 1
src/hdfs/org/apache/hadoop/hdfs/server/datanode/BlockSender.java

@@ -160,7 +160,7 @@ class BlockSender implements java.io.Closeable, FSConstants {
       if (startOffset < 0 || startOffset > endOffset
       if (startOffset < 0 || startOffset > endOffset
           || (length + startOffset) > endOffset) {
           || (length + startOffset) > endOffset) {
         String msg = " Offset " + startOffset + " and length " + length
         String msg = " Offset " + startOffset + " and length " + length
-        + " don't match block " + block + " ( blockLen " + endOffset + " )";
+        + " don't match " + block + " ( blockLen " + endOffset + " )";
         LOG.warn(datanode.dnRegistration + ":sendBlock() : " + msg);
         LOG.warn(datanode.dnRegistration + ":sendBlock() : " + msg);
         throw new IOException(msg);
         throw new IOException(msg);
       }
       }

+ 8 - 9
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataBlockScanner.java

@@ -262,7 +262,7 @@ class DataBlockScanner implements Runnable {
     
     
     BlockScanInfo info = blockMap.get(block);
     BlockScanInfo info = blockMap.get(block);
     if ( info != null ) {
     if ( info != null ) {
-      LOG.warn("Adding an already existing block " + block);
+      LOG.warn("Adding an already existing " + block);
       delBlockInfo(info);
       delBlockInfo(info);
     }
     }
     
     
@@ -369,8 +369,7 @@ class DataBlockScanner implements Runnable {
   }
   }
   
   
   private void handleScanFailure(Block block) {
   private void handleScanFailure(Block block) {
-    
-    LOG.info("Reporting bad block " + block + " to namenode.");
+    LOG.info("Reporting bad " + block + " to namenode.");
     
     
     try {
     try {
       DatanodeInfo[] dnArr = { new DatanodeInfo(datanode.dnRegistration) };
       DatanodeInfo[] dnArr = { new DatanodeInfo(datanode.dnRegistration) };
@@ -380,7 +379,7 @@ class DataBlockScanner implements Runnable {
       /* One common reason is that NameNode could be in safe mode.
       /* One common reason is that NameNode could be in safe mode.
        * Should we keep on retrying in that case?
        * Should we keep on retrying in that case?
        */
        */
-      LOG.warn("Failed to report bad block " + block + " to namenode : " +
+      LOG.warn("Failed to report bad " + block + " to namenode : " +
                " Exception : " + StringUtils.stringifyException(e));
                " Exception : " + StringUtils.stringifyException(e));
     }
     }
   }
   }
@@ -460,8 +459,8 @@ class DataBlockScanner implements Runnable {
         
         
         blockSender.sendBlock(out, null, throttler);
         blockSender.sendBlock(out, null, throttler);
 
 
-        LOG.info((second ? "Second " : "") +
-                 "Verification succeeded for " + block);
+        LOG.info((second ? "Second verification" : "Verification") +
+                 " succeeded " + block);
         
         
         if ( second ) {
         if ( second ) {
           totalTransientErrors++;
           totalTransientErrors++;
@@ -477,8 +476,8 @@ class DataBlockScanner implements Runnable {
 
 
         // If the block does not exists anymore, then its not an error
         // If the block does not exists anymore, then its not an error
         if ( dataset.getFile(block) == null ) {
         if ( dataset.getFile(block) == null ) {
-          LOG.info("Verification failed for " + block + ". Its ok since " +
-          "it not in datanode dataset anymore.");
+          LOG.info("Verification failed for " + block + ". Its ok since "
+              + "it is not in datanode dataset anymore.");
           deleteBlock(block);
           deleteBlock(block);
           return;
           return;
         }
         }
@@ -629,7 +628,7 @@ class DataBlockScanner implements Runnable {
       throw e;
       throw e;
     } finally {
     } finally {
       shutdown();
       shutdown();
-      LOG.info("Exiting DataBlockScanner thread.");
+      LOG.info("Exiting DataBlockScanner thread");
     }
     }
   }
   }
   
   

+ 11 - 13
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataNode.java

@@ -479,7 +479,7 @@ public class DataNode extends Configured
       blockScanner = new DataBlockScanner(this, (FSDataset)data, conf);
       blockScanner = new DataBlockScanner(this, (FSDataset)data, conf);
     } else {
     } else {
       LOG.info("Periodic Block Verification is disabled because " +
       LOG.info("Periodic Block Verification is disabled because " +
-               reason + ".");
+               reason);
     }
     }
 
 
     readaheadPool = ReadaheadPool.getInstance();
     readaheadPool = ReadaheadPool.getInstance();
@@ -986,7 +986,6 @@ public class DataNode extends Configured
    * forever calling remote NameNode functions.
    * forever calling remote NameNode functions.
    */
    */
   public void offerService() throws Exception {
   public void offerService() throws Exception {
-     
     LOG.info("using BLOCKREPORT_INTERVAL of " + blockReportInterval + "msec" + 
     LOG.info("using BLOCKREPORT_INTERVAL of " + blockReportInterval + "msec" + 
        " Initial delay: " + initialBlockReportDelay + "msec");
        " Initial delay: " + initialBlockReportDelay + "msec");
 
 
@@ -1018,7 +1017,6 @@ public class DataNode extends Configured
                                                        xmitsInProgress.get(),
                                                        xmitsInProgress.get(),
                                                        getXceiverCount());
                                                        getXceiverCount());
           myMetrics.addHeartBeat(now() - startTime);
           myMetrics.addHeartBeat(now() - startTime);
-          //LOG.info("Just sent heartbeat, with name " + localName);
           if (!processCommand(cmds))
           if (!processCommand(cmds))
             continue;
             continue;
         }
         }
@@ -1117,7 +1115,7 @@ public class DataNode extends Configured
         // start block scanner
         // start block scanner
         if (blockScanner != null && blockScannerThread == null &&
         if (blockScanner != null && blockScannerThread == null &&
             upgradeManager.isUpgradeCompleted()) {
             upgradeManager.isUpgradeCompleted()) {
-          LOG.info("Starting Periodic block scanner.");
+          LOG.info("Starting Periodic block scanner");
           blockScannerThread = new Daemon(blockScanner);
           blockScannerThread = new Daemon(blockScanner);
           blockScannerThread.start();
           blockScannerThread.start();
         }
         }
@@ -1301,7 +1299,7 @@ public class DataNode extends Configured
                               ) throws IOException {
                               ) throws IOException {
     if (!data.isValidBlock(block)) {
     if (!data.isValidBlock(block)) {
       // block does not exist or is under-construction
       // block does not exist or is under-construction
-      String errStr = "Can't send invalid block " + block;
+      String errStr = "Can't send invalid " + block;
       LOG.info(errStr);
       LOG.info(errStr);
       notifyNamenode(DatanodeProtocol.INVALID_BLOCK, errStr);
       notifyNamenode(DatanodeProtocol.INVALID_BLOCK, errStr);
       return;
       return;
@@ -1314,7 +1312,7 @@ public class DataNode extends Configured
       namenode.reportBadBlocks(new LocatedBlock[]{
       namenode.reportBadBlocks(new LocatedBlock[]{
           new LocatedBlock(block, new DatanodeInfo[] {
           new LocatedBlock(block, new DatanodeInfo[] {
               new DatanodeInfo(dnRegistration)})});
               new DatanodeInfo(dnRegistration)})});
-      LOG.info("Can't replicate block " + block
+      LOG.info("Can't replicate " + block
           + " because on-disk length " + onDiskLength 
           + " because on-disk length " + onDiskLength 
           + " is shorter than NameNode recorded length " + block.getNumBytes());
           + " is shorter than NameNode recorded length " + block.getNumBytes());
       return;
       return;
@@ -1328,7 +1326,7 @@ public class DataNode extends Configured
           xfersBuilder.append(xferTargets[i].getName());
           xfersBuilder.append(xferTargets[i].getName());
           xfersBuilder.append(" ");
           xfersBuilder.append(" ");
         }
         }
-        LOG.info(dnRegistration + " Starting thread to transfer block " + 
+        LOG.info(dnRegistration + " Starting thread to transfer " + 
                  block + " to " + xfersBuilder);                       
                  block + " to " + xfersBuilder);                       
       }
       }
 
 
@@ -1343,7 +1341,7 @@ public class DataNode extends Configured
       try {
       try {
         transferBlock(blocks[i], xferTargets[i]);
         transferBlock(blocks[i], xferTargets[i]);
       } catch (IOException ie) {
       } catch (IOException ie) {
-        LOG.warn("Failed to transfer block " + blocks[i], ie);
+        LOG.warn("Failed to transfer " + blocks[i], ie);
       }
       }
     }
     }
   }
   }
@@ -1532,7 +1530,7 @@ public class DataNode extends Configured
         blockSender.sendBlock(out, baseStream, null);
         blockSender.sendBlock(out, baseStream, null);
 
 
         // no response necessary
         // no response necessary
-        LOG.info(dnRegistration + ":Transmitted block " + b + " to " + curTarget);
+        LOG.info(dnRegistration + ":Transmitted " + b + " to " + curTarget);
 
 
       } catch (IOException ie) {
       } catch (IOException ie) {
         LOG.warn(dnRegistration + ":Failed to transfer " + b + " to " + targets[0].getName()
         LOG.warn(dnRegistration + ":Failed to transfer " + b + " to " + targets[0].getName()
@@ -1853,9 +1851,9 @@ public class DataNode extends Configured
       data.finalizeBlockIfNeeded(newblock);
       data.finalizeBlockIfNeeded(newblock);
       myMetrics.incrBlocksWritten();
       myMetrics.incrBlocksWritten();
       notifyNamenodeReceivedBlock(newblock, EMPTY_DEL_HINT);
       notifyNamenodeReceivedBlock(newblock, EMPTY_DEL_HINT);
-      LOG.info("Received block " + newblock +
+      LOG.info("Received " + newblock +
                 " of size " + newblock.getNumBytes() +
                 " of size " + newblock.getNumBytes() +
-                " as part of lease recovery.");
+                " as part of lease recovery");
     }
     }
   }
   }
 
 
@@ -1986,7 +1984,7 @@ public class DataNode extends Configured
     // file at the same time.
     // file at the same time.
     synchronized (ongoingRecovery) {
     synchronized (ongoingRecovery) {
       if (ongoingRecovery.get(block.getWithWildcardGS()) != null) {
       if (ongoingRecovery.get(block.getWithWildcardGS()) != null) {
-        String msg = "Block " + block + " is already being recovered, " +
+        String msg = block + " is already being recovered, " +
                      " ignoring this request to recover it.";
                      " ignoring this request to recover it.";
         LOG.info(msg);
         LOG.info(msg);
         throw new IOException(msg);
         throw new IOException(msg);
@@ -2013,7 +2011,7 @@ public class DataNode extends Configured
                 id, getConf(), socketTimeout, connectToDnViaHostname);
                 id, getConf(), socketTimeout, connectToDnViaHostname);
           BlockRecoveryInfo info = datanode.startBlockRecovery(block);
           BlockRecoveryInfo info = datanode.startBlockRecovery(block);
           if (info == null) {
           if (info == null) {
-            LOG.info("No block metadata found for block " + block + " on datanode "
+            LOG.info("No block metadata found for " + block + " on datanode "
                 + id);
                 + id);
             continue;
             continue;
           }
           }

+ 6 - 6
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataStorage.java

@@ -115,11 +115,11 @@ public class DataStorage extends Storage {
           break;
           break;
         case NON_EXISTENT:
         case NON_EXISTENT:
           // ignore this storage
           // ignore this storage
-          LOG.info("Storage directory " + dataDir + " does not exist.");
+          LOG.info("Storage directory " + dataDir + " does not exist");
           it.remove();
           it.remove();
           continue;
           continue;
         case NOT_FORMATTED: // format
         case NOT_FORMATTED: // format
-          LOG.info("Storage directory " + dataDir + " is not formatted.");
+          LOG.info("Storage directory " + dataDir + " is not formatted");
           LOG.info("Formatting ...");
           LOG.info("Formatting ...");
           format(sd, nsInfo);
           format(sd, nsInfo);
           break;
           break;
@@ -291,7 +291,7 @@ public class DataStorage extends Storage {
     // rename tmp to previous
     // rename tmp to previous
     rename(tmpDir, prevDir);
     rename(tmpDir, prevDir);
     LOG.info( hardLink.linkStats.report());
     LOG.info( hardLink.linkStats.report());
-    LOG.info("Upgrade of " + sd.getRoot()+ " is complete.");
+    LOG.info("Upgrade of " + sd.getRoot()+ " is complete");
   }
   }
 
 
   void doRollback( StorageDirectory sd,
   void doRollback( StorageDirectory sd,
@@ -327,7 +327,7 @@ public class DataStorage extends Storage {
     rename(prevDir, curDir);
     rename(prevDir, curDir);
     // delete tmp dir
     // delete tmp dir
     deleteDir(tmpDir);
     deleteDir(tmpDir);
-    LOG.info("Rollback of " + sd.getRoot() + " is complete.");
+    LOG.info("Rollback of " + sd.getRoot() + " is complete");
   }
   }
 
 
   void doFinalize(StorageDirectory sd) throws IOException {
   void doFinalize(StorageDirectory sd) throws IOException {
@@ -350,9 +350,9 @@ public class DataStorage extends Storage {
           try {
           try {
             deleteDir(tmpDir);
             deleteDir(tmpDir);
           } catch(IOException ex) {
           } 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");
         }
         }
         public String toString() { return "Finalize " + dataDirPath; }
         public String toString() { return "Finalize " + dataDirPath; }
       }).start();
       }).start();

+ 18 - 22
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DataXceiver.java

@@ -174,7 +174,7 @@ class DataXceiver implements Runnable, FSConstants {
           out.writeShort(DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN);
           out.writeShort(DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN);
           out.flush();
           out.flush();
           throw new IOException("Access token verification failed, for client "
           throw new IOException("Access token verification failed, for client "
-              + remoteAddress + " for OP_READ_BLOCK for block " + block);
+              + remoteAddress + " for OP_READ_BLOCK for " + block);
         } finally {
         } finally {
           IOUtils.closeStream(out);
           IOUtils.closeStream(out);
         }
         }
@@ -187,7 +187,7 @@ class DataXceiver implements Runnable, FSConstants {
         ? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress,
         ? String.format(DN_CLIENTTRACE_FORMAT, localAddress, remoteAddress,
             "%d", "HDFS_READ", clientName, "%d", 
             "%d", "HDFS_READ", clientName, "%d", 
             datanode.dnRegistration.getStorageID(), block, "%d")
             datanode.dnRegistration.getStorageID(), block, "%d")
-        : datanode.dnRegistration + " Served block " + block + " to " +
+        : datanode.dnRegistration + " Served " + block + " to " +
             s.getInetAddress();
             s.getInetAddress();
     try {
     try {
       try {
       try {
@@ -222,9 +222,8 @@ class DataXceiver implements Runnable, FSConstants {
        * Earlier version shutdown() datanode if there is disk error.
        * Earlier version shutdown() datanode if there is disk error.
        */
        */
       LOG.warn(datanode.dnRegistration +  ":Got exception while serving " + 
       LOG.warn(datanode.dnRegistration +  ":Got exception while serving " + 
-          block + " to " +
-                s.getInetAddress() + ":\n" + 
-                StringUtils.stringifyException(ioe) );
+          block + " to " + s.getInetAddress() + ":\n" + 
+          StringUtils.stringifyException(ioe) );
       throw ioe;
       throw ioe;
     } finally {
     } finally {
       IOUtils.closeStream(out);
       IOUtils.closeStream(out);
@@ -247,9 +246,8 @@ class DataXceiver implements Runnable, FSConstants {
     //
     //
     Block block = new Block(in.readLong(), 
     Block block = new Block(in.readLong(), 
         dataXceiverServer.estimateBlockSize, in.readLong());
         dataXceiverServer.estimateBlockSize, in.readLong());
-    LOG.info("Receiving block " + block + 
-             " src: " + remoteAddress +
-             " dest: " + localAddress);
+    LOG.info("Receiving " + block + " src: " + remoteAddress + " dest: "
+        + localAddress);
     int pipelineSize = in.readInt(); // num of datanodes in entire pipeline
     int pipelineSize = in.readInt(); // num of datanodes in entire pipeline
     boolean isRecovery = in.readBoolean(); // is this part of recovery?
     boolean isRecovery = in.readBoolean(); // is this part of recovery?
     String client = Text.readString(in); // working on behalf of this client
     String client = Text.readString(in); // working on behalf of this client
@@ -285,7 +283,7 @@ class DataXceiver implements Runnable, FSConstants {
             replyOut.flush();
             replyOut.flush();
           }
           }
           throw new IOException("Access token verification failed, for client "
           throw new IOException("Access token verification failed, for client "
-              + remoteAddress + " for OP_WRITE_BLOCK for block " + block);
+              + remoteAddress + " for OP_WRITE_BLOCK for " + block);
         } finally {
         } finally {
           IOUtils.closeStream(replyOut);
           IOUtils.closeStream(replyOut);
         }
         }
@@ -381,9 +379,9 @@ class DataXceiver implements Runnable, FSConstants {
           if (client.length() > 0) {
           if (client.length() > 0) {
             throw e;
             throw e;
           } else {
           } else {
-            LOG.info(datanode.dnRegistration + ":Exception transfering block " +
+            LOG.info(datanode.dnRegistration + ":Exception transfering " +
                      block + " to mirror " + mirrorNode +
                      block + " to mirror " + mirrorNode +
-                     ". continuing without the mirror.\n" +
+                     "- continuing without the mirror\n" +
                      StringUtils.stringifyException(e));
                      StringUtils.stringifyException(e));
           }
           }
         }
         }
@@ -411,10 +409,8 @@ class DataXceiver implements Runnable, FSConstants {
       // the block is finalized in the PacketResponder.
       // the block is finalized in the PacketResponder.
       if (client.length() == 0) {
       if (client.length() == 0) {
         datanode.notifyNamenodeReceivedBlock(block, DataNode.EMPTY_DEL_HINT);
         datanode.notifyNamenodeReceivedBlock(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 + " size " + block.getNumBytes());
       }
       }
 
 
       if (datanode.blockScanner != null) {
       if (datanode.blockScanner != null) {
@@ -454,7 +450,7 @@ class DataXceiver implements Runnable, FSConstants {
           out.flush();
           out.flush();
           throw new IOException(
           throw new IOException(
               "Access token verification failed, for client " + remoteAddress
               "Access token verification failed, for client " + remoteAddress
-                  + " for OP_BLOCK_CHECKSUM for block " + block);
+                  + " for OP_BLOCK_CHECKSUM for " + block);
         } finally {
         } finally {
           IOUtils.closeStream(out);
           IOUtils.closeStream(out);
         }
         }
@@ -512,7 +508,7 @@ class DataXceiver implements Runnable, FSConstants {
             BlockTokenSecretManager.AccessMode.COPY);
             BlockTokenSecretManager.AccessMode.COPY);
       } catch (InvalidToken e) {
       } catch (InvalidToken e) {
         LOG.warn("Invalid access token in request from "
         LOG.warn("Invalid access token in request from "
-            + remoteAddress + " for OP_COPY_BLOCK for block " + block);
+            + remoteAddress + " for OP_COPY_BLOCK for " + block);
         sendResponse(s,
         sendResponse(s,
             (short) DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN,
             (short) DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN,
             datanode.socketWriteTimeout);
             datanode.socketWriteTimeout);
@@ -522,7 +518,7 @@ class DataXceiver implements Runnable, FSConstants {
 
 
     if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
     if (!dataXceiverServer.balanceThrottler.acquire()) { // not able to start
       LOG.info("Not able to copy block " + blockId + " to " 
       LOG.info("Not able to copy block " + blockId + " to " 
-          + s.getRemoteSocketAddress() + " because threads quota is exceeded.");
+          + s.getRemoteSocketAddress() + " because threads quota is exceeded");
       sendResponse(s, (short)DataTransferProtocol.OP_STATUS_ERROR, 
       sendResponse(s, (short)DataTransferProtocol.OP_STATUS_ERROR, 
           datanode.socketWriteTimeout);
           datanode.socketWriteTimeout);
       return;
       return;
@@ -552,7 +548,7 @@ class DataXceiver implements Runnable, FSConstants {
       datanode.myMetrics.incrBytesRead((int) read);
       datanode.myMetrics.incrBytesRead((int) read);
       datanode.myMetrics.incrBlocksRead();
       datanode.myMetrics.incrBlocksRead();
       
       
-      LOG.info("Copied block " + block + " to " + s.getRemoteSocketAddress());
+      LOG.info("Copied " + block + " to " + s.getRemoteSocketAddress());
     } catch (IOException ioe) {
     } catch (IOException ioe) {
       isOpSuccess = false;
       isOpSuccess = false;
       throw ioe;
       throw ioe;
@@ -643,11 +639,11 @@ class DataXceiver implements Runnable, FSConstants {
       short status = proxyReply.readShort();
       short status = proxyReply.readShort();
       if (status != DataTransferProtocol.OP_STATUS_SUCCESS) {
       if (status != DataTransferProtocol.OP_STATUS_SUCCESS) {
         if (status == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN) {
         if (status == DataTransferProtocol.OP_STATUS_ERROR_ACCESS_TOKEN) {
-          throw new IOException("Copy block " + block + " from "
+          throw new IOException("Copy " + block + " from "
               + proxySock.getRemoteSocketAddress()
               + proxySock.getRemoteSocketAddress()
               + " failed due to access token error");
               + " failed due to access token error");
         }
         }
-        throw new IOException("Copy block " + block + " from "
+        throw new IOException("Copy " + block + " from "
             + proxySock.getRemoteSocketAddress() + " failed");
             + proxySock.getRemoteSocketAddress() + " failed");
       }
       }
       // open a block receiver and check if the block does not exist
       // open a block receiver and check if the block does not exist
@@ -663,7 +659,7 @@ class DataXceiver implements Runnable, FSConstants {
       // notify name node
       // notify name node
       datanode.notifyNamenodeReceivedBlock(block, sourceID);
       datanode.notifyNamenodeReceivedBlock(block, sourceID);
 
 
-      LOG.info("Moved block " + block + 
+      LOG.info("Moved " + block + 
           " from " + s.getRemoteSocketAddress());
           " from " + s.getRemoteSocketAddress());
       
       
     } catch (IOException ioe) {
     } catch (IOException ioe) {

+ 3 - 3
src/hdfs/org/apache/hadoop/hdfs/server/datanode/DatanodeBlockInfo.java

@@ -109,15 +109,15 @@ class DatanodeBlockInfo {
       return false;
       return false;
     }
     }
     if (file == null || volume == null) {
     if (file == null || volume == null) {
-      throw new IOException("detachBlock:Block not found. " + block);
+      throw new IOException("detachBlock: not found " + block);
     }
     }
     File meta = FSDataset.getMetaFile(file, block);
     File meta = FSDataset.getMetaFile(file, block);
     if (meta == null) {
     if (meta == null) {
-      throw new IOException("Meta file not found for block " + block);
+      throw new IOException("Meta file not found for " + block);
     }
     }
 
 
     if (HardLink.getLinkCount(file) > numLinks) {
     if (HardLink.getLinkCount(file) > numLinks) {
-      DataNode.LOG.info("CopyOnWrite for block " + block);
+      DataNode.LOG.info("CopyOnWrite for " + block);
       detachFile(file, block);
       detachFile(file, block);
     }
     }
     if (HardLink.getLinkCount(meta) > numLinks) {
     if (HardLink.getLinkCount(meta) > numLinks) {

+ 10 - 11
src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDataset.java

@@ -565,7 +565,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
             // add this block to block set
             // add this block to block set
             blockSet.add(block);
             blockSet.add(block);
             if (DataNode.LOG.isDebugEnabled()) {
             if (DataNode.LOG.isDebugEnabled()) {
-              DataNode.LOG.debug("recoverBlocksBeingWritten for block " + block);
+              DataNode.LOG.debug("recoverBlocksBeingWritten for " + block);
             }
             }
           }
           }
         }
         }
@@ -766,7 +766,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         volumes = fsvs; // replace array of volumes
         volumes = fsvs; // replace array of volumes
       }
       }
       Log.info("Completed FSVolumeSet.checkDirs. Removed=" + removed_size + 
       Log.info("Completed FSVolumeSet.checkDirs. Removed=" + removed_size + 
-          "volumes. List of current volumes: " +   toString());
+          " volumes. Current volumes: " +   toString());
       
       
       return removed_vols;
       return removed_vols;
     }
     }
@@ -1472,7 +1472,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         volumeMap.put(b, new DatanodeBlockInfo(v, f));
         volumeMap.put(b, new DatanodeBlockInfo(v, f));
       } else {
       } else {
         // reopening block for appending to it.
         // reopening block for appending to it.
-        DataNode.LOG.info("Reopen Block for append " + b);
+        DataNode.LOG.info("Reopen for append " + b);
         v = volumeMap.get(b).getVolume();
         v = volumeMap.get(b).getVolume();
         f = createTmpFile(v, b, replicationRequest);
         f = createTmpFile(v, b, replicationRequest);
         File blkfile = getBlockFile(b);
         File blkfile = getBlockFile(b);
@@ -1491,19 +1491,18 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
         DataNode.LOG.debug("Renaming " + blkfile + " to " + f);
         DataNode.LOG.debug("Renaming " + blkfile + " to " + f);
         if (!blkfile.renameTo(f)) {
         if (!blkfile.renameTo(f)) {
           if (!f.delete()) {
           if (!f.delete()) {
-            throw new IOException("Block " + b + " reopen failed. " +
+            throw new IOException(b + " reopen failed. " +
                                   " Unable to remove file " + f);
                                   " Unable to remove file " + f);
           }
           }
           if (!blkfile.renameTo(f)) {
           if (!blkfile.renameTo(f)) {
-            throw new IOException("Block " + b + " reopen failed. " +
+            throw new IOException(b + " reopen failed. " +
                                   " Unable to move block file " + blkfile +
                                   " Unable to move block file " + blkfile +
                                   " to tmp dir " + f);
                                   " to tmp dir " + f);
           }
           }
         }
         }
       }
       }
       if (f == null) {
       if (f == null) {
-        DataNode.LOG.warn("Block " + b + " reopen failed " +
-                          " Unable to locate tmp file.");
+        DataNode.LOG.warn(b + " reopen failed. Unable to locate tmp file");
         throw new IOException("Block " + b + " reopen failed " +
         throw new IOException("Block " + b + " reopen failed " +
                               " Unable to locate tmp file.");
                               " Unable to locate tmp file.");
       }
       }
@@ -1744,9 +1743,10 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       long st = System.currentTimeMillis();
       long st = System.currentTimeMillis();
       // broken out to a static method to simplify testing
       // broken out to a static method to simplify testing
       reconcileRoughBlockScan(seenOnDisk, volumeMap, ongoingCreates);
       reconcileRoughBlockScan(seenOnDisk, volumeMap, ongoingCreates);
-      DataNode.LOG.info(
-          "Reconciled asynchronous block report against current state in " +
-          (System.currentTimeMillis() - st) + " ms");
+      if (DataNode.LOG.isDebugEnabled()) {
+        DataNode.LOG.debug("Reconciled block report with current state in "
+                + (System.currentTimeMillis() - st) + "ms");
+      }
       
       
       blockReport = seenOnDisk.keySet();
       blockReport = seenOnDisk.keySet();
     }
     }
@@ -2250,7 +2250,6 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
           waitForReportRequest();
           waitForReportRequest();
           assert requested && scan == null;
           assert requested && scan == null;
           
           
-          DataNode.LOG.info("Starting asynchronous block report scan");
           long st = System.currentTimeMillis();
           long st = System.currentTimeMillis();
           HashMap<Block, File> result = fsd.roughBlockScan();
           HashMap<Block, File> result = fsd.roughBlockScan();
           DataNode.LOG.info("Finished asynchronous block report scan in "
           DataNode.LOG.info("Finished asynchronous block report scan in "

+ 5 - 5
src/hdfs/org/apache/hadoop/hdfs/server/datanode/FSDatasetAsyncDiskService.java

@@ -130,7 +130,7 @@ class FSDatasetAsyncDiskService {
       // clear the executor map so that calling execute again will fail.
       // clear the executor map so that calling execute again will fail.
       executors = null;
       executors = null;
       
       
-      LOG.info("All async disk service threads have been shut down.");
+      LOG.info("All async disk service threads have been shut down");
     }
     }
   }
   }
 
 
@@ -140,7 +140,7 @@ class FSDatasetAsyncDiskService {
    */
    */
   void deleteAsync(FSDataset.FSVolume volume, File blockFile,
   void deleteAsync(FSDataset.FSVolume volume, File blockFile,
       File metaFile, long dfsBytes, String blockName) {
       File metaFile, long dfsBytes, String blockName) {
-    DataNode.LOG.info("Scheduling block " + blockName + " file " + blockFile
+    DataNode.LOG.info("Scheduling " + blockName + " file " + blockFile
         + " for deletion");
         + " for deletion");
     ReplicaFileDeleteTask deletionTask = 
     ReplicaFileDeleteTask deletionTask = 
         new ReplicaFileDeleteTask(volume, blockFile, metaFile, dfsBytes,
         new ReplicaFileDeleteTask(volume, blockFile, metaFile, dfsBytes,
@@ -175,18 +175,18 @@ class FSDatasetAsyncDiskService {
     @Override
     @Override
     public String toString() {
     public String toString() {
       // Called in AsyncDiskService.execute for displaying error messages.
       // Called in AsyncDiskService.execute for displaying error messages.
-      return "deletion of block " + blockName + " with block file " + blockFile
+      return "deletion of " + blockName + " with file " + blockFile
           + " and meta file " + metaFile + " from volume " + volume;
           + " and meta file " + metaFile + " from volume " + volume;
     }
     }
 
 
     @Override
     @Override
     public void run() {
     public void run() {
       if ( !blockFile.delete() || ( !metaFile.delete() && metaFile.exists() ) ) {
       if ( !blockFile.delete() || ( !metaFile.delete() && metaFile.exists() ) ) {
-        DataNode.LOG.warn("Unexpected error trying to delete block "
+        DataNode.LOG.warn("Unexpected error trying to delete "
             + blockName + " at file " + blockFile + ". Ignored.");
             + blockName + " at file " + blockFile + ". Ignored.");
       } else {
       } else {
         volume.decDfsUsed(dfsBytes);
         volume.decDfsUsed(dfsBytes);
-        DataNode.LOG.info("Deleted block " + blockName + " at file " + blockFile);
+        DataNode.LOG.info("Deleted " + blockName + " at file " + blockFile);
       }
       }
     }
     }
   };
   };

+ 4 - 4
src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeManagerDatanode.java

@@ -54,7 +54,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
     DataNode.LOG.info("\n   Distributed upgrade for DataNode " 
     DataNode.LOG.info("\n   Distributed upgrade for DataNode " 
         + dataNode.dnRegistration.getName() 
         + dataNode.dnRegistration.getName() 
         + " version " + getUpgradeVersion() + " to current LV " 
         + " version " + getUpgradeVersion() + " to current LV " 
-        + FSConstants.LAYOUT_VERSION + " is initialized.");
+        + FSConstants.LAYOUT_VERSION + " is initialized");
     UpgradeObjectDatanode curUO = (UpgradeObjectDatanode)currentUpgrades.first();
     UpgradeObjectDatanode curUO = (UpgradeObjectDatanode)currentUpgrades.first();
     curUO.setDatanode(dataNode);
     curUO.setDatanode(dataNode);
     upgradeState = curUO.preUpgradeAction(nsInfo);
     upgradeState = curUO.preUpgradeAction(nsInfo);
@@ -99,7 +99,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
       DataNode.LOG.info("\n   Distributed upgrade for DataNode version " 
       DataNode.LOG.info("\n   Distributed upgrade for DataNode version " 
           + getUpgradeVersion() + " to current LV " 
           + getUpgradeVersion() + " to current LV " 
           + FSConstants.LAYOUT_VERSION + " cannot be started. "
           + FSConstants.LAYOUT_VERSION + " cannot be started. "
-          + "The upgrade object is not defined.");
+          + "The upgrade object is not defined");
       return false;
       return false;
     }
     }
     upgradeState = true;
     upgradeState = true;
@@ -111,7 +111,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
     DataNode.LOG.info("\n   Distributed upgrade for DataNode " 
     DataNode.LOG.info("\n   Distributed upgrade for DataNode " 
         + dataNode.dnRegistration.getName() 
         + dataNode.dnRegistration.getName() 
         + " version " + getUpgradeVersion() + " to current LV " 
         + " version " + getUpgradeVersion() + " to current LV " 
-        + FSConstants.LAYOUT_VERSION + " is started.");
+        + FSConstants.LAYOUT_VERSION + " is started");
     return true;
     return true;
   }
   }
 
 
@@ -141,7 +141,7 @@ class UpgradeManagerDatanode extends UpgradeManager {
     DataNode.LOG.info("\n   Distributed upgrade for DataNode " 
     DataNode.LOG.info("\n   Distributed upgrade for DataNode " 
         + dataNode.dnRegistration.getName() 
         + dataNode.dnRegistration.getName() 
         + " version " + getUpgradeVersion() + " to current LV " 
         + " version " + getUpgradeVersion() + " to current LV " 
-        + FSConstants.LAYOUT_VERSION + " is complete.");
+        + FSConstants.LAYOUT_VERSION + " is complete");
   }
   }
 
 
   synchronized void shutdownUpgrade() {
   synchronized void shutdownUpgrade() {

+ 1 - 1
src/hdfs/org/apache/hadoop/hdfs/server/datanode/UpgradeObjectDatanode.java

@@ -109,7 +109,7 @@ public abstract class UpgradeObjectDatanode extends UpgradeObject implements Run
     if(getUpgradeStatus() < 100) {
     if(getUpgradeStatus() < 100) {
       DataNode.LOG.info("\n   Distributed upgrade for DataNode version " 
       DataNode.LOG.info("\n   Distributed upgrade for DataNode version " 
           + getVersion() + " to current LV " 
           + getVersion() + " to current LV " 
-          + FSConstants.LAYOUT_VERSION + " cannot be completed.");
+          + FSConstants.LAYOUT_VERSION + " cannot be completed");
     }
     }
 
 
     // Complete the upgrade by calling the manager method
     // Complete the upgrade by calling the manager method

+ 6 - 10
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java

@@ -176,16 +176,13 @@ class FSDirectory implements FSConstants, Closeable {
       newNode = addNode(path, newNode, -1, false);
       newNode = addNode(path, newNode, -1, false);
     }
     }
     if (newNode == null) {
     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;
       return null;
     }
     }
     // add create file record to log, record new generation stamp
     // add create file record to log, record new generation stamp
     fsImage.getEditLog().logOpenFile(path, newNode);
     fsImage.getEditLog().logOpenFile(path, newNode);
 
 
-    NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
-                                  +path+" is added to the file system");
+    NameNode.stateChangeLog.debug("DIR* addFile: " + path + " is added");
     return newNode;
     return newNode;
   }
   }
 
 
@@ -291,7 +288,7 @@ class FSDirectory implements FSConstants, Closeable {
 
 
       NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
       NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
                                     + path + " with " + block
                                     + path + " with " + block
-                                    + " block is added to the in-memory "
+                                    + " is added to the in-memory "
                                     + "file system");
                                     + "file system");
     }
     }
     return block;
     return block;
@@ -308,7 +305,7 @@ class FSDirectory implements FSConstants, Closeable {
       fsImage.getEditLog().logOpenFile(path, file);
       fsImage.getEditLog().logOpenFile(path, file);
       NameNode.stateChangeLog.debug("DIR* FSDirectory.persistBlocks: "
       NameNode.stateChangeLog.debug("DIR* FSDirectory.persistBlocks: "
                                     +path+" with "+ file.getBlocks().length 
                                     +path+" with "+ file.getBlocks().length 
-                                    +" blocks is persisted to the file system");
+                                    +" blocks is persisted");
     }
     }
   }
   }
 
 
@@ -323,7 +320,7 @@ class FSDirectory implements FSConstants, Closeable {
       if (NameNode.stateChangeLog.isDebugEnabled()) {
       if (NameNode.stateChangeLog.isDebugEnabled()) {
         NameNode.stateChangeLog.debug("DIR* FSDirectory.closeFile: "
         NameNode.stateChangeLog.debug("DIR* FSDirectory.closeFile: "
                                     +path+" with "+ file.getBlocks().length 
                                     +path+" with "+ file.getBlocks().length 
-                                    +" blocks is persisted to the file system");
+                                    +" blocks is persisted");
       }
       }
     }
     }
   }
   }
@@ -345,8 +342,7 @@ class FSDirectory implements FSConstants, Closeable {
       // write modified block locations to log
       // write modified block locations to log
       fsImage.getEditLog().logOpenFile(path, fileNode);
       fsImage.getEditLog().logOpenFile(path, fileNode);
       NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
       NameNode.stateChangeLog.debug("DIR* FSDirectory.addFile: "
-                                    +path+" with "+block
-                                    +" block is added to the file system");
+          + path + " with "+ block +" is added to the");
       // update space consumed
       // update space consumed
       INode[] pathINodes = getExistingPathINodes(path);
       INode[] pathINodes = getExistingPathINodes(path);
       updateCount(pathINodes, pathINodes.length-1, 0,
       updateCount(pathINodes, pathINodes.length-1, 0,

+ 1 - 2
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -205,8 +205,7 @@ public class FSEditLog {
       int bufSize = bufCurrent.size();
       int bufSize = bufCurrent.size();
       if (bufSize != 0) {
       if (bufSize != 0) {
         throw new IOException("FSEditStream has " + bufSize +
         throw new IOException("FSEditStream has " + bufSize +
-                              " bytes still to be flushed and cannot " +
-                              "be closed.");
+           " bytes still to be flushed and cannot be closed.");
       } 
       } 
       bufCurrent.close();
       bufCurrent.close();
       bufReady.close();
       bufReady.close();

+ 128 - 140
src/hdfs/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -566,7 +566,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if (conf.getBoolean("dfs.support.append", false)) {
     if (conf.getBoolean("dfs.support.append", false)) {
       LOG.warn("The dfs.support.append option is in your configuration, " +
       LOG.warn("The dfs.support.append option is in your configuration, " +
                "however append is not supported. This configuration option " +
                "however append is not supported. This configuration option " +
-               "is no longer required to enable sync.");
+               "is no longer required to enable sync");
     }
     }
     this.durableSync = conf.getBoolean("dfs.durable.sync", true);
     this.durableSync = conf.getBoolean("dfs.durable.sync", true);
     if (!durableSync) {
     if (!durableSync) {
@@ -610,7 +610,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
           DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY
           DFSConfigKeys.DFS_NAMENODE_USE_STALE_DATANODE_FOR_WRITE_RATIO_KEY
               + " = '" + ratioUseStaleDataNodesForWrite
               + " = '" + ratioUseStaleDataNodesForWrite
               + "' is invalid. It should be a positive non-zero float value,"
               + "' is invalid. It should be a positive non-zero float value,"
-              + " not greater than 1.0f.");
+              + " not greater than 1.0f");
     }
     }
   }
   }
   
   
@@ -623,7 +623,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       throw new IllegalArgumentException(
       throw new IllegalArgumentException(
           DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY + " = '"
           DFSConfigKeys.DFS_NAMENODE_STALE_DATANODE_INTERVAL_KEY + " = '"
               + staleInterval
               + staleInterval
-              + "' is invalid. It should be a positive non-zero value.");
+              + "' is invalid. It should be a positive non-zero value");
     }
     }
     final long heartbeatIntervalSeconds = conf.getLong(
     final long heartbeatIntervalSeconds = conf.getLong(
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
         DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY,
@@ -641,13 +641,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
           + " heartbeat intervals. This may cause too frequent changes of "
           + " heartbeat intervals. This may cause too frequent changes of "
           + "stale states of DataNodes since a heartbeat msg may be missing "
           + "stale states of DataNodes since a heartbeat msg may be missing "
           + "due to temporary short-term failures. Reset stale interval to "
           + "due to temporary short-term failures. Reset stale interval to "
-          + minStaleInterval + ".");
+          + minStaleInterval);
       staleInterval = minStaleInterval;
       staleInterval = minStaleInterval;
     }
     }
     if (staleInterval > heartbeatExpireInterval) {
     if (staleInterval > heartbeatExpireInterval) {
       LOG.warn("The given interval for marking stale datanode = "
       LOG.warn("The given interval for marking stale datanode = "
           + staleInterval + ", which is larger than heartbeat expire interval "
           + staleInterval + ", which is larger than heartbeat expire interval "
-          + heartbeatExpireInterval + ".");
+          + heartbeatExpireInterval);
     }
     }
     return staleInterval;
     return staleInterval;
   }
   }
@@ -663,7 +663,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
           + DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY
           + DFSConfigKeys.DFS_NAMENODE_CHECK_STALE_DATANODE_KEY
           + " as false while setting "
           + " as false while setting "
           + DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY
           + DFSConfigKeys.DFS_NAMENODE_AVOID_STALE_DATANODE_FOR_WRITE_KEY
-          + " as true.");
+          + " as true");
     }
     }
     return avoidStaleDataNodesForWrite;
     return avoidStaleDataNodesForWrite;
   }
   }
@@ -858,7 +858,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if (fileINode == null) { // block does not belong to any file
     if (fileINode == null) { // block does not belong to any file
       return 0;
       return 0;
     }
     }
-    assert !fileINode.isDirectory() : "Block cannot belong to a directory.";
+    assert !fileINode.isDirectory() : "Block cannot belong to a directory";
     return fileINode.getReplication();
     return fileINode.getReplication();
   }
   }
 
 
@@ -998,11 +998,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       FSPermissionChecker pc = checkOwner(src);
       FSPermissionChecker pc = checkOwner(src);
       if (!pc.isSuper) {
       if (!pc.isSuper) {
         if (username != null && !pc.user.equals(username)) {
         if (username != null && !pc.user.equals(username)) {
-          throw new AccessControlException("Non-super user cannot change owner.");
+          throw new AccessControlException("Non-super user cannot change owner");
         }
         }
         if (group != null && !pc.containsGroup(group)) {
         if (group != null && !pc.containsGroup(group)) {
-          throw new AccessControlException("User does not belong to " + group
-              + " .");
+          throw new AccessControlException("User does not belong to " + group);
         }
         }
       }
       }
       dir.setOwner(src, username, group);
       dir.setOwner(src, username, group);
@@ -1189,7 +1188,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   public synchronized void setTimes(String src, long mtime, long atime) throws IOException {
   public synchronized void setTimes(String src, long mtime, long atime) throws IOException {
     if (!isAccessTimeSupported() && atime != -1) {
     if (!isAccessTimeSupported() && atime != -1) {
       throw new IOException("Access time for hdfs is not configured. " +
       throw new IOException("Access time for hdfs is not configured. " +
-                            " Please set dfs.access.time.precision configuration parameter.");
+                            " Please set dfs.access.time.precision configuration parameter");
     }
     }
     if (isInSafeMode()) {
     if (isInSafeMode()) {
       throw new SafeModeException("Cannot set accesstimes  for " + src, safeMode);
       throw new SafeModeException("Cannot set accesstimes  for " + src, safeMode);
@@ -1209,7 +1208,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                       "setTimes", src, null, stat);
                       "setTimes", src, null, stat);
       }
       }
     } else {
     } else {
-      throw new FileNotFoundException("File " + src + " does not exist.");
+      throw new FileNotFoundException("File " + src + " does not exist");
     }
     }
   }
   }
 
 
@@ -1263,12 +1262,12 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       
       
     if (oldRepl > replication) {  
     if (oldRepl > replication) {  
       // old replication > the new one; need to remove copies
       // old replication > the new one; need to remove copies
-      LOG.info("Reducing replication for file " + src 
+      LOG.info("Reducing replication for " + src 
                + ". New replication is " + replication);
                + ". New replication is " + replication);
       for(int idx = 0; idx < fileBlocks.length; idx++)
       for(int idx = 0; idx < fileBlocks.length; idx++)
         processOverReplicatedBlock(fileBlocks[idx], replication, null, null);
         processOverReplicatedBlock(fileBlocks[idx], replication, null, null);
     } else { // replication factor is increased
     } else { // replication factor is increased
-      LOG.info("Increasing replication for file " + src 
+      LOG.info("Increasing replication for " + src 
           + ". New replication is " + replication);
           + ". New replication is " + replication);
     }
     }
     return true;
     return true;
@@ -1354,7 +1353,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                               long blockSize
                                               long blockSize
                                               ) throws IOException {
                                               ) throws IOException {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: src=" + src
+      NameNode.stateChangeLog.debug("DIR* startFile: src=" + src
           + ", holder=" + holder
           + ", holder=" + holder
           + ", clientMachine=" + clientMachine
           + ", clientMachine=" + clientMachine
           + ", createParent=" + createParent
           + ", createParent=" + createParent
@@ -1364,15 +1363,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     }
     }
 
 
     if (isInSafeMode())
     if (isInSafeMode())
-      throw new SafeModeException("Cannot create file" + src, safeMode);
+      throw new SafeModeException("Cannot create " + src, safeMode);
     if (!DFSUtil.isValidName(src)) {
     if (!DFSUtil.isValidName(src)) {
-      throw new IOException("Invalid file name: " + src);
+      throw new IOException("Invalid name: " + src);
     }
     }
 
 
     // Verify that the destination does not exist as a directory already.
     // Verify that the destination does not exist as a directory already.
     boolean pathExists = dir.exists(src);
     boolean pathExists = dir.exists(src);
     if (pathExists && dir.isDir(src)) {
     if (pathExists && dir.isDir(src)) {
-      throw new IOException("Cannot create file "+ src + "; already exists as a directory.");
+      throw new IOException("Cannot create "+ src + "; already exists as a directory");
     }
     }
 
 
     if (isPermissionEnabled) {
     if (isPermissionEnabled) {
@@ -1399,7 +1398,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       }
       }
       if (append) {
       if (append) {
         if (myFile == null) {
         if (myFile == null) {
-          throw new FileNotFoundException("failed to append to non-existent file "
+          throw new FileNotFoundException("failed to append to non-existent "
               + src + " on client " + clientMachine);
               + src + " on client " + clientMachine);
         } else if (myFile.isDirectory()) {
         } else if (myFile.isDirectory()) {
           throw new IOException("failed to append to directory " + src 
           throw new IOException("failed to append to directory " + src 
@@ -1409,7 +1408,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         if (overwrite) {
         if (overwrite) {
           delete(src, true);
           delete(src, true);
         } else {
         } else {
-          throw new IOException("failed to create file " + src 
+          throw new IOException("failed to create " + src 
                                 +" on client " + clientMachine
                                 +" on client " + clientMachine
                                 +" either because the filename is invalid or the file exists");
                                 +" either because the filename is invalid or the file exists");
         }
         }
@@ -1448,17 +1447,16 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         INodeFileUnderConstruction newNode = dir.addFile(src, permissions,
         INodeFileUnderConstruction newNode = dir.addFile(src, permissions,
             replication, blockSize, holder, clientMachine, clientNode, genstamp);
             replication, blockSize, holder, clientMachine, clientNode, genstamp);
         if (newNode == null) {
         if (newNode == null) {
-          throw new IOException("DIR* NameSystem.startFile: " +
-                                "Unable to add file to namespace.");
+          throw new IOException("DIR* startFile: Unable to add to namespace");
         }
         }
         leaseManager.addLease(newNode.clientName, src);
         leaseManager.addLease(newNode.clientName, src);
         if (NameNode.stateChangeLog.isDebugEnabled()) {
         if (NameNode.stateChangeLog.isDebugEnabled()) {
-          NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: "
+          NameNode.stateChangeLog.debug("DIR* startFile: "
                                      +"add "+src+" to namespace for "+holder);
                                      +"add "+src+" to namespace for "+holder);
         }
         }
       }
       }
     } catch (IOException ie) {
     } catch (IOException ie) {
-      NameNode.stateChangeLog.warn("DIR* NameSystem.startFile: "
+      NameNode.stateChangeLog.warn("DIR* startFile: "
                                    +ie.getMessage());
                                    +ie.getMessage());
       throw ie;
       throw ie;
     }
     }
@@ -1482,7 +1480,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
           "Cannot recover the lease of " + src, safeMode);
           "Cannot recover the lease of " + src, safeMode);
     }
     }
     if (!DFSUtil.isValidName(src)) {
     if (!DFSUtil.isValidName(src)) {
-      throw new IOException("Invalid file name: " + src);
+      throw new IOException("Invalid name: " + src);
     }
     }
 
 
     INode inode = dir.getFileINode(src);
     INode inode = dir.getFileINode(src);
@@ -1521,7 +1519,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
           throw new AlreadyBeingCreatedException(
           throw new AlreadyBeingCreatedException(
                     "failed to create file " + src + " for " + holder +
                     "failed to create file " + src + " for " + holder +
                     " on client " + clientMachine + 
                     " on client " + clientMachine + 
-                    " because current leaseholder is trying to recreate file.");
+                    " because current leaseholder is trying to recreate file");
         }
         }
       }
       }
       //
       //
@@ -1532,7 +1530,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         throw new AlreadyBeingCreatedException(
         throw new AlreadyBeingCreatedException(
                     "failed to create file " + src + " for " + holder +
                     "failed to create file " + src + " for " + holder +
                     " on client " + clientMachine + 
                     " on client " + clientMachine + 
-                    " because pendingCreates is non-null but no leases found.");
+                    " because pendingCreates is non-null but no leases found");
       }
       }
       if (force) {
       if (force) {
         // close now: no need to wait for soft lease expiration and 
         // close now: no need to wait for soft lease expiration and 
@@ -1568,7 +1566,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       ) throws IOException {
       ) throws IOException {
     if (!allowBrokenAppend) {
     if (!allowBrokenAppend) {
       throw new IOException("Append is not supported. " +
       throw new IOException("Append is not supported. " +
-          "Please see the dfs.support.append configuration parameter.");
+          "Please see the dfs.support.append configuration parameter");
     }
     }
     startFileInternal(src, null, holder, clientMachine, false, true, 
     startFileInternal(src, null, holder, clientMachine, false, true, 
                       false, (short)maxReplication, (long)0);
                       false, (short)maxReplication, (long)0);
@@ -1631,7 +1629,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     }
     }
     if (lb != null) {
     if (lb != null) {
       if (NameNode.stateChangeLog.isDebugEnabled()) {
       if (NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug("DIR* NameSystem.appendFile: file "
+        NameNode.stateChangeLog.debug("DIR* appendFile: "
             +src+" for "+holder+" at "+clientMachine
             +src+" for "+holder+" at "+clientMachine
             +" block " + lb.getBlock()
             +" block " + lb.getBlock()
             +" block size " + lb.getBlock().getNumBytes());
             +" block size " + lb.getBlock().getNumBytes());
@@ -1675,7 +1673,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     DatanodeDescriptor clientNode = null;
     DatanodeDescriptor clientNode = null;
     Block newBlock = null;
     Block newBlock = null;
 
 
-    NameNode.stateChangeLog.debug("BLOCK* NameSystem.getAdditionalBlock: file "
+    NameNode.stateChangeLog.debug("BLOCK* getAdditionalBlock: "
                                   +src+" for "+clientName);
                                   +src+" for "+clientName);
 
 
     synchronized (this) {
     synchronized (this) {
@@ -1756,16 +1754,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     //
     //
     // Remove the block from the pending creates list
     // Remove the block from the pending creates list
     //
     //
-    NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
-                                  +b+"of file "+src);
+    NameNode.stateChangeLog.debug("BLOCK* abandonBlock: " + b + "of " + src);
     if (isInSafeMode()) {
     if (isInSafeMode()) {
-      throw new SafeModeException("Cannot abandon block " + b +
-                                  " for fle" + src, safeMode);
+      throw new SafeModeException("Cannot abandon " + b +
+                                  " for " + src, safeMode);
     }
     }
     INodeFileUnderConstruction file = checkLease(src, holder);
     INodeFileUnderConstruction file = checkLease(src, holder);
     dir.removeBlock(src, file, b);
     dir.removeBlock(src, file, b);
-    NameNode.stateChangeLog.debug("BLOCK* NameSystem.abandonBlock: "
-                                    + b
+    NameNode.stateChangeLog.debug("BLOCK* abandonBlock: " + b
                                     + " is removed from pendingCreates");
                                     + " is removed from pendingCreates");
     dir.persistBlocks(src, file);
     dir.persistBlocks(src, file);
     if (persistBlocks) {
     if (persistBlocks) {
@@ -1791,7 +1787,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                       " File does not exist. " +
                                       " File does not exist. " +
                                       (lease != null ? lease.toString() :
                                       (lease != null ? lease.toString() :
                                        "Holder " + holder + 
                                        "Holder " + holder + 
-                                       " does not have any open files."));
+                                       " does not have any open files"));
     }
     }
     if (!file.isUnderConstruction()) {
     if (!file.isUnderConstruction()) {
       Lease lease = leaseManager.getLease(holder);
       Lease lease = leaseManager.getLease(holder);
@@ -1799,7 +1795,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                       " File is not open for writing. " +
                                       " File is not open for writing. " +
                                       (lease != null ? lease.toString() :
                                       (lease != null ? lease.toString() :
                                        "Holder " + holder + 
                                        "Holder " + holder + 
-                                       " does not have any open files."));
+                                       " does not have any open files"));
     }
     }
     INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)file;
     INodeFileUnderConstruction pendingFile = (INodeFileUnderConstruction)file;
     if (holder != null && !pendingFile.getClientName().equals(holder)) {
     if (holder != null && !pendingFile.getClientName().equals(holder)) {
@@ -1829,15 +1825,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
 
   private synchronized CompleteFileStatus completeFileInternal(String src, 
   private synchronized CompleteFileStatus completeFileInternal(String src, 
                                                 String holder) throws IOException {
                                                 String holder) throws IOException {
-    NameNode.stateChangeLog.debug("DIR* NameSystem.completeFile: " + src + " for " + holder);
+    NameNode.stateChangeLog.debug("DIR* completeFile: " + src + " for " + holder);
     if (isInSafeMode())
     if (isInSafeMode())
-      throw new SafeModeException("Cannot complete file " + src, safeMode);
+      throw new SafeModeException("Cannot complete " + src, safeMode);
 
 
     INodeFileUnderConstruction pendingFile  = checkLease(src, holder);
     INodeFileUnderConstruction pendingFile  = checkLease(src, holder);
     Block[] fileBlocks =  dir.getFileBlocks(src);
     Block[] fileBlocks =  dir.getFileBlocks(src);
 
 
     if (fileBlocks == null ) {
     if (fileBlocks == null ) {
-      NameNode.stateChangeLog.warn("DIR* NameSystem.completeFile: "
+      NameNode.stateChangeLog.warn("DIR* completeFile: "
                                    + "failed to complete " + src
                                    + "failed to complete " + src
                                    + " because dir.getFileBlocks() is null,"
                                    + " because dir.getFileBlocks() is null,"
                                    + " pending from " + pendingFile.getClientMachine());
                                    + " pending from " + pendingFile.getClientMachine());
@@ -1850,7 +1846,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
 
     finalizeINodeFileUnderConstruction(src, pendingFile);
     finalizeINodeFileUnderConstruction(src, pendingFile);
 
 
-    NameNode.stateChangeLog.info("DIR* NameSystem.completeFile: file " + src
+    NameNode.stateChangeLog.info("DIR* completeFile: " + src
                                   + " is closed by " + holder);
                                   + " is closed by " + holder);
     return CompleteFileStatus.COMPLETE_SUCCESS;
     return CompleteFileStatus.COMPLETE_SUCCESS;
   }
   }
@@ -1891,7 +1887,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     }
     }
     b.setGenerationStamp(getGenerationStamp());
     b.setGenerationStamp(getGenerationStamp());
     b = dir.addBlock(src, inodes, b);
     b = dir.addBlock(src, inodes, b);
-    NameNode.stateChangeLog.info("BLOCK* NameSystem.allocateBlock: "
+    NameNode.stateChangeLog.info("BLOCK* allocateBlock: "
                                  +src+ ". "+b);
                                  +src+ ". "+b);
     return b;
     return b;
   }
   }
@@ -1946,7 +1942,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   void addToInvalidates(Block b, DatanodeInfo dn, boolean log) {
   void addToInvalidates(Block b, DatanodeInfo dn, boolean log) {
     addToInvalidatesNoLog(b, dn);
     addToInvalidatesNoLog(b, dn);
     if (log) {
     if (log) {
-      NameNode.stateChangeLog.info("BLOCK* NameSystem.addToInvalidates: "
+      NameNode.stateChangeLog.info("BLOCK* addToInvalidates: "
           + b.getBlockName() + " to " + dn.getName());
           + b.getBlockName() + " to " + dn.getName());
     }
     }
   }
   }
@@ -1994,7 +1990,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       datanodes.append(node.getName()).append(" ");
       datanodes.append(node.getName()).append(" ");
     }
     }
     if (datanodes.length() != 0) {
     if (datanodes.length() != 0) {
-      NameNode.stateChangeLog.info("BLOCK* NameSystem.addToInvalidates: "
+      NameNode.stateChangeLog.info("BLOCK* addToInvalidates: "
           + b.getBlockName() + " to " + datanodes.toString());
           + b.getBlockName() + " to " + datanodes.toString());
     }
     }
   }
   }
@@ -2005,7 +2001,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
   private synchronized void dumpRecentInvalidateSets(PrintWriter out) {
   private synchronized void dumpRecentInvalidateSets(PrintWriter out) {
     int size = recentInvalidateSets.values().size();
     int size = recentInvalidateSets.values().size();
     out.println("Metasave: Blocks " + pendingDeletionBlocksCount 
     out.println("Metasave: Blocks " + pendingDeletionBlocksCount 
-        + " waiting deletion from " + size + " datanodes.");
+        + " waiting deletion from " + size + " datanodes");
     if (size == 0) {
     if (size == 0) {
       return;
       return;
     }
     }
@@ -2037,15 +2033,15 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       // ignore the request for now. This could happen when BlockScanner
       // ignore the request for now. This could happen when BlockScanner
       // thread of Datanode reports bad block before Block reports are sent
       // thread of Datanode reports bad block before Block reports are sent
       // by the Datanode on startup
       // by the Datanode on startup
-      NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: " +
-                                   "block " + blk + " could not be marked " +
+      NameNode.stateChangeLog.info("BLOCK markBlockAsCorrupt: " +
+                                   blk + " could not be marked " +
                                    "as corrupt as it does not exists in " +
                                    "as corrupt as it does not exists in " +
                                    "blocksMap");
                                    "blocksMap");
     } else {
     } else {
       INodeFile inode = storedBlockInfo.getINode();
       INodeFile inode = storedBlockInfo.getINode();
       if (inode == null) {
       if (inode == null) {
-        NameNode.stateChangeLog.info("BLOCK NameSystem.markBlockAsCorrupt: " +
-                                     "block " + blk + " could not be marked " +
+        NameNode.stateChangeLog.info("BLOCK markBlockAsCorrupt: " +
+                                     blk + " could not be marked " +
                                      "as corrupt as it does not belong to " +
                                      "as corrupt as it does not belong to " +
                                      "any file");
                                      "any file");
         addToInvalidates(storedBlockInfo, node);
         addToInvalidates(storedBlockInfo, node);
@@ -2068,14 +2064,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    */
    */
   private synchronized void invalidateBlock(Block blk, DatanodeInfo dn)
   private synchronized void invalidateBlock(Block blk, DatanodeInfo dn)
     throws IOException {
     throws IOException {
-    NameNode.stateChangeLog.info("DIR* NameSystem.invalidateBlock: " 
+    NameNode.stateChangeLog.info("DIR* invalidateBlock: " 
                                  + blk + " on " 
                                  + blk + " on " 
                                  + dn.getName());
                                  + dn.getName());
     DatanodeDescriptor node = getDatanode(dn);
     DatanodeDescriptor node = getDatanode(dn);
     if (node == null) {
     if (node == null) {
-      throw new IOException("Cannot invalidate block " + blk +
+      throw new IOException("Cannot invalidate " + blk +
                             " because datanode " + dn.getName() +
                             " because datanode " + dn.getName() +
-                            " does not exist.");
+                            " does not exist");
     }
     }
 
 
     // Check how many copies we have of the block.  If we have at least one
     // Check how many copies we have of the block.  If we have at least one
@@ -2084,13 +2080,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if (count > 1) {
     if (count > 1) {
       addToInvalidates(blk, dn);
       addToInvalidates(blk, dn);
       removeStoredBlock(blk, node);
       removeStoredBlock(blk, node);
-      NameNode.stateChangeLog.debug("BLOCK* NameSystem.invalidateBlocks: "
+      NameNode.stateChangeLog.debug("BLOCK* invalidateBlocks: "
                                    + blk + " on " 
                                    + blk + " on " 
-                                   + dn.getName() + " listed for deletion.");
+                                   + dn.getName() + " listed for deletion");
     } else {
     } else {
-      NameNode.stateChangeLog.info("BLOCK* NameSystem.invalidateBlocks: "
+      NameNode.stateChangeLog.info("BLOCK* invalidateBlocks: "
                                    + blk + " on " 
                                    + blk + " on " 
-                                   + dn.getName() + " is the only copy and was not deleted.");
+                                   + dn.getName() + " is the only copy and was not deleted");
     }
     }
   }
   }
 
 
@@ -2120,7 +2116,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
 
   private synchronized boolean renameToInternal(String src, String dst
   private synchronized boolean renameToInternal(String src, String dst
       ) throws IOException {
       ) throws IOException {
-    NameNode.stateChangeLog.debug("DIR* NameSystem.renameTo: " + src + " to " + dst);
+    NameNode.stateChangeLog.debug("DIR* renameTo: " + src + " to " + dst);
     if (isInSafeMode())
     if (isInSafeMode())
       throw new SafeModeException("Cannot rename " + src, safeMode);
       throw new SafeModeException("Cannot rename " + src, safeMode);
     if (!DFSUtil.isValidName(dst)) {
     if (!DFSUtil.isValidName(dst)) {
@@ -2153,7 +2149,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         throw new IOException(src + " is non empty");
         throw new IOException(src + " is non empty");
       }
       }
       if (NameNode.stateChangeLog.isDebugEnabled()) {
       if (NameNode.stateChangeLog.isDebugEnabled()) {
-        NameNode.stateChangeLog.debug("DIR* NameSystem.delete: " + src);
+        NameNode.stateChangeLog.debug("DIR* delete: " + src);
       }
       }
       boolean status = deleteInternal(src, true);
       boolean status = deleteInternal(src, true);
       if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
       if (status && auditLog.isInfoEnabled() && isExternalInvocation()) {
@@ -2194,7 +2190,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     removeBlocks(collectedBlocks); // Incremental deletion of blocks
     removeBlocks(collectedBlocks); // Incremental deletion of blocks
     collectedBlocks.clear();
     collectedBlocks.clear();
     if (NameNode.stateChangeLog.isDebugEnabled()) {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("DIR* Namesystem.delete: " + src
+      NameNode.stateChangeLog.debug("DIR* delete: " + src
           + " is removed");
           + " is removed");
     }
     }
     return true;
     return true;
@@ -2263,7 +2259,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    */
    */
   private synchronized boolean mkdirsInternal(String src,
   private synchronized boolean mkdirsInternal(String src,
       PermissionStatus permissions) throws IOException {
       PermissionStatus permissions) throws IOException {
-    NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
+    NameNode.stateChangeLog.debug("DIR* mkdirs: " + src);
     if (isPermissionEnabled) {
     if (isPermissionEnabled) {
       checkTraverse(src);
       checkTraverse(src);
     }
     }
@@ -2324,11 +2320,11 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    */
    */
   void fsync(String src, String clientName) throws IOException {
   void fsync(String src, String clientName) throws IOException {
 
 
-    NameNode.stateChangeLog.info("BLOCK* NameSystem.fsync: file "
+    NameNode.stateChangeLog.info("BLOCK* fsync: "
                                   + src + " for " + clientName);
                                   + src + " for " + clientName);
     synchronized (this) {
     synchronized (this) {
       if (isInSafeMode()) {
       if (isInSafeMode()) {
-        throw new SafeModeException("Cannot fsync file " + src, safeMode);
+        throw new SafeModeException("Cannot fsync " + src, safeMode);
       }
       }
       INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
       INodeFileUnderConstruction pendingFile  = checkLease(src, clientName);
       dir.persistBlocks(src, pendingFile);
       dir.persistBlocks(src, pendingFile);
@@ -2367,16 +2363,16 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
 
     INodeFile iFile = dir.getFileINode(src);
     INodeFile iFile = dir.getFileINode(src);
     if (iFile == null) {
     if (iFile == null) {
-      final String message = "DIR* NameSystem.internalReleaseCreate: "
+      final String message = "DIR* internalReleaseCreate: "
         + "attempt to release a create lock on "
         + "attempt to release a create lock on "
-        + src + " file does not exist.";
+        + src + " file does not exist";
       NameNode.stateChangeLog.warn(message);
       NameNode.stateChangeLog.warn(message);
       throw new IOException(message);
       throw new IOException(message);
     }
     }
     if (!iFile.isUnderConstruction()) {
     if (!iFile.isUnderConstruction()) {
-      final String message = "DIR* NameSystem.internalReleaseCreate: "
+      final String message = "DIR* internalReleaseCreate: "
         + "attempt to release a create lock on "
         + "attempt to release a create lock on "
-        + src + " but file is already closed.";
+        + src + " but file is already closed";
       NameNode.stateChangeLog.warn(message);
       NameNode.stateChangeLog.warn(message);
       throw new IOException(message);
       throw new IOException(message);
     }
     }
@@ -2424,7 +2420,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
 
   private void finalizeINodeFileUnderConstruction(String src,
   private void finalizeINodeFileUnderConstruction(String src,
       INodeFileUnderConstruction pendingFile) throws IOException {
       INodeFileUnderConstruction pendingFile) throws IOException {
-    NameNode.stateChangeLog.info("Removing lease on  file " + src + 
+    NameNode.stateChangeLog.info("Removing lease on  " + src + 
                                  " from client " + pendingFile.clientName);
                                  " from client " + pendingFile.clientName);
     leaseManager.removeLease(pendingFile.clientName, src);
     leaseManager.removeLease(pendingFile.clientName, src);
 
 
@@ -2500,7 +2496,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
           descriptorsList.add(node);
           descriptorsList.add(node);
         } else {
         } else {
           LOG.error("commitBlockSynchronization included a target DN " +
           LOG.error("commitBlockSynchronization included a target DN " +
-            newtargets[i] + " which is not known to NN. Ignoring.");
+            newtargets[i] + " which is not known to NN. Ignoring");
         }
         }
       }
       }
       if (!descriptorsList.isEmpty()) {
       if (!descriptorsList.isEmpty()) {
@@ -2622,7 +2618,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     nodeReg.exportedKeys = getBlockKeys();
     nodeReg.exportedKeys = getBlockKeys();
       
       
     NameNode.stateChangeLog.info(
     NameNode.stateChangeLog.info(
-                                 "BLOCK* NameSystem.registerDatanode: "
+                                 "BLOCK* registerDatanode: "
                                  + "node registration from " + nodeReg.getName()
                                  + "node registration from " + nodeReg.getName()
                                  + " storage " + nodeReg.getStorageID());
                                  + " storage " + nodeReg.getStorageID());
 
 
@@ -2630,7 +2626,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     DatanodeDescriptor nodeN = host2DataNodeMap.getDatanodeByName(nodeReg.getName());
     DatanodeDescriptor nodeN = host2DataNodeMap.getDatanodeByName(nodeReg.getName());
       
       
     if (nodeN != null && nodeN != nodeS) {
     if (nodeN != null && nodeN != nodeS) {
-      NameNode.LOG.info("BLOCK* NameSystem.registerDatanode: "
+      NameNode.LOG.info("BLOCK* registerDatanode: "
                         + "node from name: " + nodeN.getName());
                         + "node from name: " + nodeN.getName());
       // nodeN previously served a different data storage, 
       // nodeN previously served a different data storage, 
       // which is not served by anybody anymore.
       // which is not served by anybody anymore.
@@ -2645,8 +2641,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         // The same datanode has been just restarted to serve the same data 
         // The same datanode has been just restarted to serve the same data 
         // storage. We do not need to remove old data blocks, the delta will
         // storage. We do not need to remove old data blocks, the delta will
         // be calculated on the next block report from the datanode
         // be calculated on the next block report from the datanode
-        NameNode.stateChangeLog.debug("BLOCK* NameSystem.registerDatanode: "
-                                      + "node restarted.");
+        NameNode.stateChangeLog.debug("BLOCK* registerDatanode: "
+                                      + "node restarted");
       } else {
       } else {
         // nodeS is found
         // nodeS is found
         /* The registering datanode is a replacement node for the existing 
         /* The registering datanode is a replacement node for the existing 
@@ -2657,7 +2653,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
           value in "VERSION" file under the data directory of the datanode,
           value in "VERSION" file under the data directory of the datanode,
           but this is might not work if VERSION file format has changed 
           but this is might not work if VERSION file format has changed 
        */        
        */        
-        NameNode.stateChangeLog.info( "BLOCK* NameSystem.registerDatanode: "
+        NameNode.stateChangeLog.info( "BLOCK* registerDatanode: "
                                       + "node " + nodeS.getName()
                                       + "node " + nodeS.getName()
                                       + " is replaced by " + nodeReg.getName() + 
                                       + " is replaced by " + nodeReg.getName() + 
                                       " with the same storageID " +
                                       " with the same storageID " +
@@ -2690,8 +2686,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       // it is either empty or was created by pre-storageID version of DFS
       // it is either empty or was created by pre-storageID version of DFS
       nodeReg.storageID = newStorageID();
       nodeReg.storageID = newStorageID();
       NameNode.stateChangeLog.debug(
       NameNode.stateChangeLog.debug(
-                                    "BLOCK* NameSystem.registerDatanode: "
-                                    + "new storageID " + nodeReg.getStorageID() + " assigned.");
+                                    "BLOCK* registerDatanode: "
+                                    + "new storageID " + nodeReg.getStorageID() + " assigned");
     }
     }
     // register new datanode
     // register new datanode
     DatanodeDescriptor nodeDescr 
     DatanodeDescriptor nodeDescr 
@@ -2937,7 +2933,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
           processPendingReplications();
           processPendingReplications();
           Thread.sleep(replicationRecheckInterval);
           Thread.sleep(replicationRecheckInterval);
         } catch (InterruptedException ie) {
         } catch (InterruptedException ie) {
-          LOG.warn("ReplicationMonitor thread received InterruptedException." + ie);
+          LOG.warn("ReplicationMonitor thread received InterruptedException" + ie);
           break;
           break;
         } catch (IOException ie) {
         } catch (IOException ie) {
           LOG.warn("ReplicationMonitor thread received exception. " + ie +  " " +
           LOG.warn("ReplicationMonitor thread received exception. " + ie +  " " +
@@ -3167,7 +3163,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
             break;
             break;
           neededReplicationsIterator = neededReplications.iterator();
           neededReplicationsIterator = neededReplications.iterator();
           assert neededReplicationsIterator.hasNext() : 
           assert neededReplicationsIterator.hasNext() : 
-                                  "neededReplications should not be empty.";
+                                  "neededReplications should not be empty";
         }
         }
 
 
         Block block = neededReplicationsIterator.next();
         Block block = neededReplicationsIterator.next();
@@ -3223,9 +3219,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         if(numEffectiveReplicas >= requiredReplication) {
         if(numEffectiveReplicas >= requiredReplication) {
           neededReplications.remove(block, priority); // remove from neededReplications
           neededReplications.remove(block, priority); // remove from neededReplications
           replIndex--;
           replIndex--;
-          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");
           return false;
           return false;
         }
         }
       }
       }
@@ -3260,9 +3255,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         if(numEffectiveReplicas >= requiredReplication) {
         if(numEffectiveReplicas >= requiredReplication) {
           neededReplications.remove(block, priority); // remove from neededReplications
           neededReplications.remove(block, priority); // remove from neededReplications
           replIndex--;
           replIndex--;
-          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");
           return false;
           return false;
         } 
         } 
 
 
@@ -3277,8 +3271,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         // The reason we use 'pending' is so we can retry
         // The reason we use 'pending' is so we can retry
         // replications that fail after an appropriate amount of time.
         // replications that fail after an appropriate amount of time.
         pendingReplications.increment(block, targets.length);
         pendingReplications.increment(block, targets.length);
-        NameNode.stateChangeLog.debug(
-            "BLOCK* block " + block
+        NameNode.stateChangeLog.debug("BLOCK* " + block
             + " is moved from neededReplications to pendingReplications");
             + " is moved from neededReplications to pendingReplications");
 
 
         // remove from neededReplications
         // remove from neededReplications
@@ -3295,10 +3288,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         targetList.append(' ');
         targetList.append(' ');
         targetList.append(targets[k].getName());
         targetList.append(targets[k].getName());
       }
       }
-      NameNode.stateChangeLog.info(
-                "BLOCK* ask "
-                + srcNode.getName() + " to replicate "
-                + block + " to " + targetList);
+      NameNode.stateChangeLog.info( "BLOCK* ask " + srcNode.getName() + 
+          " to replicate " + block + " to " + targetList);
       NameNode.stateChangeLog.debug(
       NameNode.stateChangeLog.debug(
                 "BLOCK* neededReplications = " + neededReplications.size()
                 "BLOCK* neededReplications = " + neededReplications.size()
                 + " pendingReplications = " + pendingReplications.size());
                 + " pendingReplications = " + pendingReplications.size());
@@ -3483,7 +3474,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if (nodeInfo != null) {
     if (nodeInfo != null) {
       removeDatanode(nodeInfo);
       removeDatanode(nodeInfo);
     } else {
     } else {
-      NameNode.stateChangeLog.warn("BLOCK* NameSystem.removeDatanode: "
+      NameNode.stateChangeLog.warn("BLOCK* removeDatanode: "
                                    + nodeID.getName() + " does not exist");
                                    + nodeID.getName() + " does not exist");
     }
     }
   }
   }
@@ -3516,8 +3507,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     nodeDescr.resetBlocks();
     nodeDescr.resetBlocks();
     removeFromInvalidates(nodeDescr.getStorageID());
     removeFromInvalidates(nodeDescr.getStorageID());
     NameNode.stateChangeLog.debug(
     NameNode.stateChangeLog.debug(
-                                  "BLOCK* NameSystem.unprotectedRemoveDatanode: "
-                                  + nodeDescr.getName() + " is out of service now.");
+                                  "BLOCK* unprotectedRemoveDatanode: "
+                                  + nodeDescr.getName() + " is out of service now");
   }
   }
     
     
   void unprotectedAddDatanode(DatanodeDescriptor nodeDescr) {
   void unprotectedAddDatanode(DatanodeDescriptor nodeDescr) {
@@ -3530,8 +3521,8 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     host2DataNodeMap.add(nodeDescr);
     host2DataNodeMap.add(nodeDescr);
       
       
     NameNode.stateChangeLog.debug(
     NameNode.stateChangeLog.debug(
-                                  "BLOCK* NameSystem.unprotectedAddDatanode: "
-                                  + "node " + nodeDescr.getName() + " is added to datanodeMap.");
+                                  "BLOCK* unprotectedAddDatanode: "
+                                  + "node " + nodeDescr.getName() + " is added to datanodeMap");
   }
   }
 
 
   /**
   /**
@@ -3543,9 +3534,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     String key = nodeID.getStorageID();
     String key = nodeID.getStorageID();
     host2DataNodeMap.remove(datanodeMap.remove(key));
     host2DataNodeMap.remove(datanodeMap.remove(key));
     NameNode.stateChangeLog.debug(
     NameNode.stateChangeLog.debug(
-                                  "BLOCK* NameSystem.wipeDatanode: "
+                                  "BLOCK* wipeDatanode: "
                                   + nodeID.getName() + " storage " + key 
                                   + nodeID.getName() + " storage " + key 
-                                  + " is removed from datanodeMap.");
+                                  + " is removed from datanodeMap");
   }
   }
 
 
   FSImage getFSImage() {
   FSImage getFSImage() {
@@ -3621,7 +3612,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                 nodeInfo = null;
                 nodeInfo = null;
               }
               }
               if (nodeInfo != null && isDatanodeDead(nodeInfo)) {
               if (nodeInfo != null && isDatanodeDead(nodeInfo)) {
-                NameNode.stateChangeLog.info("BLOCK* NameSystem.heartbeatCheck: "
+                NameNode.stateChangeLog.info("BLOCK* heartbeatCheck: "
                                              + "lost heartbeat from " + nodeInfo.getName());
                                              + "lost heartbeat from " + nodeInfo.getName());
                 removeDatanode(nodeInfo);
                 removeDatanode(nodeInfo);
               }
               }
@@ -3639,7 +3630,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    */
    */
   private Block rejectAddStoredBlock(Block block, DatanodeDescriptor node,
   private Block rejectAddStoredBlock(Block block, DatanodeDescriptor node,
       String msg) {
       String msg) {
-    NameNode.stateChangeLog.info("BLOCK* NameSystem.addStoredBlock: "
+    NameNode.stateChangeLog.info("BLOCK* addStoredBlock: "
         + "addStoredBlock request received for " + block + " on "
         + "addStoredBlock request received for " + block + " on "
         + node.getName() + " size " + block.getNumBytes()
         + node.getName() + " size " + block.getNumBytes()
         + " but was rejected: " + msg);
         + " but was rejected: " + msg);
@@ -3698,14 +3689,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       // Must be the last block of a file under construction,
       // Must be the last block of a file under construction,
       if (!underConstruction) {
       if (!underConstruction) {
         rejectAddStoredBlock(new Block(block), dataNode,
         rejectAddStoredBlock(new Block(block), dataNode,
-            "Reported as block being written but is a block of closed file.");
+            "Reported as block being written but is a block of closed file");
         continue;
         continue;
       }
       }
 
 
       if (!isLastBlock) {
       if (!isLastBlock) {
         rejectAddStoredBlock(new Block(block), dataNode,
         rejectAddStoredBlock(new Block(block), dataNode,
             "Reported as block being written but not the last block of "
             "Reported as block being written but not the last block of "
-                + "an under-construction file.");
+                + "an under-construction file");
         continue;
         continue;
       }
       }
 
 
@@ -3724,7 +3715,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                         ) throws IOException {
                                         ) throws IOException {
     long startTime = now();
     long startTime = now();
     if (NameNode.stateChangeLog.isDebugEnabled()) {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("BLOCK* NameSystem.processReport: "
+      NameNode.stateChangeLog.debug("BLOCK* processReport: "
                              + "from " + nodeID.getName()+" " + 
                              + "from " + nodeID.getName()+" " + 
                              newReport.getNumberOfBlocks()+" blocks");
                              newReport.getNumberOfBlocks()+" blocks");
     }
     }
@@ -3743,7 +3734,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     // To minimize startup time, we discard any second (or later) block reports
     // To minimize startup time, we discard any second (or later) block reports
     // that we receive while still in startup phase.
     // that we receive while still in startup phase.
     if (isInStartupSafeMode() && !node.firstBlockReport()) {
     if (isInStartupSafeMode() && !node.firstBlockReport()) {
-      NameNode.stateChangeLog.info("BLOCK* NameSystem.processReport: "
+      NameNode.stateChangeLog.info("BLOCK* processReport: "
           + "discarded non-initial block report from " + nodeID.getName()
           + "discarded non-initial block report from " + nodeID.getName()
           + " because namenode still in startup phase");
           + " because namenode still in startup phase");
       return;
       return;
@@ -3765,14 +3756,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       addStoredBlock(b, node, null);
       addStoredBlock(b, node, null);
     }
     }
     for (Block b : toInvalidate) {
     for (Block b : toInvalidate) {
-      NameNode.stateChangeLog.info("BLOCK* NameSystem.processReport: block " 
+      NameNode.stateChangeLog.info("BLOCK* processReport: " 
           + b + " on " + node.getName() + " size " + b.getNumBytes()
           + b + " on " + node.getName() + " size " + b.getNumBytes()
-          + " does not belong to any file.");
+          + " does not belong to any file");
       addToInvalidates(b, node);
       addToInvalidates(b, node);
     }
     }
     long endTime = now();
     long endTime = now();
     NameNode.getNameNodeMetrics().addBlockReport(endTime - startTime);
     NameNode.getNameNodeMetrics().addBlockReport(endTime - startTime);
-    NameNode.stateChangeLog.info("*BLOCK* NameSystem.processReport: from "
+    NameNode.stateChangeLog.info("*BLOCK* processReport: from "
         + nodeID.getName() + ", blocks: " + newReport.getNumberOfBlocks()
         + nodeID.getName() + ", blocks: " + newReport.getNumberOfBlocks()
         + ", processing time: " + (endTime - startTime) + " msecs");
         + ", processing time: " + (endTime - startTime) + " msecs");
     node.processedBlockReport();
     node.processedBlockReport();
@@ -3827,11 +3818,9 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       // for recovery purposes. They will get added to the node when
       // for recovery purposes. They will get added to the node when
       // commitBlockSynchronization runs
       // commitBlockSynchronization runs
       if (underConstruction && isLastBlock && (reportedOldGS || reportedNewGS)) {
       if (underConstruction && isLastBlock && (reportedOldGS || reportedNewGS)) {
-        NameNode.stateChangeLog.info(
-          "BLOCK* NameSystem.addStoredBlock: "
-          + "Targets updated: block " + block + " on " + node.getName() +
-          " is added as a target for block " + storedBlock + " with size " +
-          block.getNumBytes());
+        NameNode.stateChangeLog.info("BLOCK* addStoredBlock: Targets updated: "
+            + block + " on " + node.getName() + " is added as a target for " 
+            + storedBlock + " with size " + block.getNumBytes());
         ((INodeFileUnderConstruction)inode).addTarget(node);
         ((INodeFileUnderConstruction)inode).addTarget(node);
         return block;
         return block;
       }
       }
@@ -3857,7 +3846,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       if (last == null) {
       if (last == null) {
         // This should never happen, but better to handle it properly than to throw
         // This should never happen, but better to handle it properly than to throw
         // an NPE below.
         // an NPE below.
-        LOG.error("Null blocks for reported block=" + block + " stored=" + storedBlock +
+        LOG.error("Null blocks for reported =" + block + " stored=" + storedBlock +
           " inode=" + fileINode);
           " inode=" + fileINode);
         return block;
         return block;
       }
       }
@@ -3872,7 +3861,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         if (cursize == 0) {
         if (cursize == 0) {
           storedBlock.setNumBytes(block.getNumBytes());
           storedBlock.setNumBytes(block.getNumBytes());
         } else if (cursize != block.getNumBytes()) {
         } else if (cursize != block.getNumBytes()) {
-          LOG.warn("Inconsistent size for block " + block + 
+          LOG.warn("Inconsistent size for " + block + 
                    " reported from " + node.getName() + 
                    " reported from " + node.getName() + 
                    " current size is " + cursize +
                    " current size is " + cursize +
                    " reported size is " + block.getNumBytes());
                    " reported size is " + block.getNumBytes());
@@ -3912,7 +3901,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
               storedBlock.setNumBytes(block.getNumBytes());
               storedBlock.setNumBytes(block.getNumBytes());
             }
             }
           } catch (IOException e) {
           } catch (IOException e) {
-            LOG.warn("Error in deleting bad block " + block + e);
+            LOG.warn("Error in deleting bad " + block + e);
           }
           }
         }
         }
         
         
@@ -3946,14 +3935,13 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       // So, we log only when namenode is out of safemode.
       // So, we log only when namenode is out of safemode.
       //
       //
       if (!isInSafeMode()) {
       if (!isInSafeMode()) {
-        NameNode.stateChangeLog.info("BLOCK* NameSystem.addStoredBlock: "
+        NameNode.stateChangeLog.info("BLOCK* addStoredBlock: "
                                       +"blockMap updated: "+node.getName()+" is added to "+block+" size "+block.getNumBytes());
                                       +"blockMap updated: "+node.getName()+" is added to "+block+" size "+block.getNumBytes());
       }
       }
     } else {
     } else {
-      NameNode.stateChangeLog.warn("BLOCK* NameSystem.addStoredBlock: "
-                                   + "Redundant addStoredBlock request received for " 
-                                   + block + " on " + node.getName()
-                                   + " size " + block.getNumBytes());
+      NameNode.stateChangeLog.warn("BLOCK* addStoredBlock: " + 
+          "Redundant addStoredBlock request received for " + block + " on " + 
+          node.getName() + " size " + block.getNumBytes());
     }
     }
 
 
     // filter out containingNodes that are marked for decommission.
     // filter out containingNodes that are marked for decommission.
@@ -4033,7 +4021,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         invalidateBlock(blk, node);
         invalidateBlock(blk, node);
       } catch (IOException e) {
       } catch (IOException e) {
         NameNode.stateChangeLog.info("NameNode.invalidateCorruptReplicas " +
         NameNode.stateChangeLog.info("NameNode.invalidateCorruptReplicas " +
-                                      "error in deleting bad block " + blk +
+                                      "error in deleting bad " + blk +
                                       " on " + node + e);
                                       " on " + node + e);
         gotException = true;
         gotException = true;
       }
       }
@@ -4202,7 +4190,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       }
       }
       if (excessBlocks.add(b)) {
       if (excessBlocks.add(b)) {
         excessBlocksCount++;
         excessBlocksCount++;
-        NameNode.stateChangeLog.debug("BLOCK* NameSystem.chooseExcessReplicates: "
+        NameNode.stateChangeLog.debug("BLOCK* chooseExcessReplicates: "
                                       +"("+cur.getName()+", "+b
                                       +"("+cur.getName()+", "+b
                                       +") is added to excessReplicateMap");
                                       +") is added to excessReplicateMap");
       }
       }
@@ -4217,7 +4205,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       // upon giving instructions to the namenode.
       // upon giving instructions to the namenode.
       //
       //
       addToInvalidatesNoLog(b, cur);
       addToInvalidatesNoLog(b, cur);
-      NameNode.stateChangeLog.info("BLOCK* NameSystem.chooseExcessReplicates: "
+      NameNode.stateChangeLog.info("BLOCK* chooseExcessReplicates: "
                 +"("+cur.getName()+", "+b+") is added to recentInvalidateSets");
                 +"("+cur.getName()+", "+b+") is added to recentInvalidateSets");
     }
     }
   }
   }
@@ -4227,10 +4215,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
    * replication tasks, if the removed block is still valid.
    * replication tasks, if the removed block is still valid.
    */
    */
   synchronized void removeStoredBlock(Block block, DatanodeDescriptor node) {
   synchronized void removeStoredBlock(Block block, DatanodeDescriptor node) {
-    NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
+    NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
                                   +block + " from "+node.getName());
                                   +block + " from "+node.getName());
     if (!blocksMap.removeNode(block, node)) {
     if (!blocksMap.removeNode(block, node)) {
-      NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
+      NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
                                     +block+" has already been removed from node "+node);
                                     +block+" has already been removed from node "+node);
       return;
       return;
     }
     }
@@ -4255,7 +4243,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if (excessBlocks != null) {
     if (excessBlocks != null) {
       if (excessBlocks.remove(block)) {
       if (excessBlocks.remove(block)) {
         excessBlocksCount--;
         excessBlocksCount--;
-        NameNode.stateChangeLog.debug("BLOCK* NameSystem.removeStoredBlock: "
+        NameNode.stateChangeLog.debug("BLOCK* removeStoredBlock: "
             + block + " is removed from excessBlocks");
             + block + " is removed from excessBlocks");
         if (excessBlocks.size() == 0) {
         if (excessBlocks.size() == 0) {
           excessReplicateMap.remove(node.getStorageID());
           excessReplicateMap.remove(node.getStorageID());
@@ -4276,14 +4264,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
                                          ) throws IOException {
                                          ) throws IOException {
     DatanodeDescriptor node = getDatanode(nodeID);
     DatanodeDescriptor node = getDatanode(nodeID);
     if (node == null || !node.isAlive) {
     if (node == null || !node.isAlive) {
-      NameNode.stateChangeLog.warn("BLOCK* NameSystem.blockReceived: " + block
+      NameNode.stateChangeLog.warn("BLOCK* blockReceived: " + block
           + " is received from dead or unregistered node " + nodeID.getName());
           + " is received from dead or unregistered node " + nodeID.getName());
       throw new IOException(
       throw new IOException(
           "Got blockReceived message from unregistered or dead node " + block);
           "Got blockReceived message from unregistered or dead node " + block);
     }
     }
         
         
     if (NameNode.stateChangeLog.isDebugEnabled()) {
     if (NameNode.stateChangeLog.isDebugEnabled()) {
-      NameNode.stateChangeLog.debug("BLOCK* NameSystem.blockReceived: "
+      NameNode.stateChangeLog.debug("BLOCK* blockReceived: "
                                     +block+" is received from " + nodeID.getName());
                                     +block+" is received from " + nodeID.getName());
     }
     }
 
 
@@ -4298,7 +4286,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     if(delHint!=null && delHint.length()!=0) {
     if(delHint!=null && delHint.length()!=0) {
       delHintNode = datanodeMap.get(delHint);
       delHintNode = datanodeMap.get(delHint);
       if(delHintNode == null) {
       if(delHintNode == null) {
-        NameNode.stateChangeLog.warn("BLOCK* NameSystem.blockReceived: "
+        NameNode.stateChangeLog.warn("BLOCK* blockReceived: "
             + block
             + block
             + " is expected to be removed from an unrecorded node " 
             + " is expected to be removed from an unrecorded node " 
             + delHint);
             + delHint);
@@ -4483,10 +4471,10 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     checkSuperuserPrivilege();
     checkSuperuserPrivilege();
     if(!isInSafeMode()) {
     if(!isInSafeMode()) {
       throw new IOException("Safe mode should be turned ON " +
       throw new IOException("Safe mode should be turned ON " +
-                            "in order to create namespace image.");
+                            "in order to create namespace image");
     }
     }
     getFSImage().saveNamespace(true);
     getFSImage().saveNamespace(true);
-    LOG.info("New namespace image has been created.");
+    LOG.info("New namespace image has been created");
   }
   }
 
 
   /**
   /**
@@ -4867,7 +4855,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       return null;
       return null;
     if (!node.getName().equals(nodeID.getName())) {
     if (!node.getName().equals(nodeID.getName())) {
       e = new UnregisteredDatanodeException(nodeID, node);
       e = new UnregisteredDatanodeException(nodeID, node);
-      NameNode.stateChangeLog.fatal("BLOCK* NameSystem.getDatanode: "
+      NameNode.stateChangeLog.fatal("BLOCK* getDatanode: "
                                     + e.getLocalizedMessage());
                                     + e.getLocalizedMessage());
       throw e;
       throw e;
     }
     }
@@ -4994,7 +4982,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       this.blockSafe = -1;
       this.blockSafe = -1;
       this.reached = -1;
       this.reached = -1;
       enter();
       enter();
-      reportStatus("STATE* Safe mode is ON.", true);
+      reportStatus("STATE* Safe mode is ON", true);
     }
     }
       
       
     /**
     /**
@@ -5005,7 +4993,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       try {
       try {
         assert isConsistent() : " SafeMode: Inconsistent filesystem state: "
         assert isConsistent() : " SafeMode: Inconsistent filesystem state: "
           + "Total num of blocks, active blocks, or "
           + "Total num of blocks, active blocks, or "
-          + "total safe blocks don't match.";
+          + "total safe blocks don't match";
       } catch(IOException e) {
       } catch(IOException e) {
         System.err.print(StringUtils.stringifyException(e));
         System.err.print(StringUtils.stringifyException(e));
       }
       }
@@ -5050,11 +5038,11 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
 
 
       long timeInSafemode = now() - systemStart;
       long timeInSafemode = now() - systemStart;
       NameNode.stateChangeLog.info("STATE* Leaving safe mode after " 
       NameNode.stateChangeLog.info("STATE* Leaving safe mode after " 
-                                    + timeInSafemode/1000 + " secs.");
+                                    + timeInSafemode/1000 + " secs");
       NameNode.getNameNodeMetrics().setSafeModeTime(timeInSafemode);
       NameNode.getNameNodeMetrics().setSafeModeTime(timeInSafemode);
       
       
       if (reached >= 0) {
       if (reached >= 0) {
-        NameNode.stateChangeLog.info("STATE* Safe mode is OFF."); 
+        NameNode.stateChangeLog.info("STATE* Safe mode is OFF"); 
       }
       }
       reached = -1;
       reached = -1;
       safeMode = null;
       safeMode = null;
@@ -5075,7 +5063,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
       if (reached == 0)
       if (reached == 0)
         return false;
         return false;
       if (now() - reached < extension) {
       if (now() - reached < extension) {
-        reportStatus("STATE* Safe mode ON.", false);
+        reportStatus("STATE* Safe mode ON", false);
         return false;
         return false;
       }
       }
       return !needEnter();
       return !needEnter();
@@ -5104,7 +5092,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     private void checkMode() {
     private void checkMode() {
       if (needEnter()) {
       if (needEnter()) {
         enter();
         enter();
-        reportStatus("STATE* Safe mode ON.", false);
+        reportStatus("STATE* Safe mode ON", false);
         return;
         return;
       }
       }
       // the threshold is reached
       // the threshold is reached
@@ -5114,14 +5102,14 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
         return;
         return;
       }
       }
       if (reached > 0) {  // threshold has already been reached before
       if (reached > 0) {  // threshold has already been reached before
-        reportStatus("STATE* Safe mode ON.", false);
+        reportStatus("STATE* Safe mode ON", false);
         return;
         return;
       }
       }
       // start monitor
       // start monitor
       reached = now();
       reached = now();
       smmthread = new Daemon(new SafeModeMonitor());
       smmthread = new Daemon(new SafeModeMonitor());
       smmthread.start();
       smmthread.start();
-      reportStatus("STATE* Safe mode extension entered.", true);
+      reportStatus("STATE* Safe mode extension entered", true);
     }
     }
       
       
     /**
     /**
@@ -5174,7 +5162,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean, FSClusterSt
     String getTurnOffTip() {
     String getTurnOffTip() {
       String leaveMsg = "Safe mode will be turned off automatically";
       String leaveMsg = "Safe mode will be turned off automatically";
       if(reached < 0)
       if(reached < 0)
-        return "Safe mode is OFF.";
+        return "Safe mode is OFF";
       if(isManual()) {
       if(isManual()) {
         if(getDistributedUpgradeState())
         if(getDistributedUpgradeState())
           return leaveMsg + " upon completion of " + 
           return leaveMsg + " upon completion of " + 

+ 5 - 5
src/hdfs/org/apache/hadoop/hdfs/server/namenode/NameNode.java

@@ -677,7 +677,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
                              ) throws IOException {
                              ) throws IOException {
     String clientMachine = getClientMachine();
     String clientMachine = getClientMachine();
     if (stateChangeLog.isDebugEnabled()) {
     if (stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*DIR* NameNode.create: file "
+      stateChangeLog.debug("*DIR* NameNode.create: "
                          +src+" for "+clientName+" at "+clientMachine);
                          +src+" for "+clientName+" at "+clientMachine);
     }
     }
     if (!checkPathLength(src)) {
     if (!checkPathLength(src)) {
@@ -696,7 +696,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   public LocatedBlock append(String src, String clientName) throws IOException {
   public LocatedBlock append(String src, String clientName) throws IOException {
     String clientMachine = getClientMachine();
     String clientMachine = getClientMachine();
     if (stateChangeLog.isDebugEnabled()) {
     if (stateChangeLog.isDebugEnabled()) {
-      stateChangeLog.debug("*DIR* NameNode.append: file "
+      stateChangeLog.debug("*DIR* NameNode.append: "
           +src+" for "+clientName+" at "+clientMachine);
           +src+" for "+clientName+" at "+clientMachine);
     }
     }
     LocatedBlock info = namesystem.appendFile(src, clientName, clientMachine);
     LocatedBlock info = namesystem.appendFile(src, clientName, clientMachine);
@@ -750,7 +750,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
       }
       }
     }
     }
 
 
-    stateChangeLog.debug("*BLOCK* NameNode.addBlock: file "
+    stateChangeLog.debug("*BLOCK* NameNode.addBlock: "
                          +src+" for "+clientName);
                          +src+" for "+clientName);
     LocatedBlock locatedBlock = namesystem.getAdditionalBlock(
     LocatedBlock locatedBlock = namesystem.getAdditionalBlock(
       src, clientName, excludedNodesSet);
       src, clientName, excludedNodesSet);
@@ -765,7 +765,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
   public void abandonBlock(Block b, String src, String holder
   public void abandonBlock(Block b, String src, String holder
       ) throws IOException {
       ) throws IOException {
     stateChangeLog.debug("*BLOCK* NameNode.abandonBlock: "
     stateChangeLog.debug("*BLOCK* NameNode.abandonBlock: "
-                         +b+" of file "+src);
+                         +b+" of "+src);
     if (!namesystem.abandonBlock(b, src, holder)) {
     if (!namesystem.abandonBlock(b, src, holder)) {
       throw new IOException("Cannot abandon block during write to " + src);
       throw new IOException("Cannot abandon block during write to " + src);
     }
     }
@@ -780,7 +780,7 @@ public class NameNode implements ClientProtocol, DatanodeProtocol,
     } else if (returnCode == CompleteFileStatus.COMPLETE_SUCCESS) {
     } else if (returnCode == CompleteFileStatus.COMPLETE_SUCCESS) {
       return true;
       return true;
     } else {
     } else {
-      throw new IOException("Could not complete write to file " + src + " by " + clientName);
+      throw new IOException("Could not complete write to " + src + " by " + clientName);
     }
     }
   }
   }