Przeglądaj źródła

HDFS-1977. Stop using StringUtils.stringifyException(). Contributed by Bharath Mundlapudi.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1145834 13f79535-47bb-0310-9956-ffa450edef68
Jitendra Nath Pandey 14 lat temu
rodzic
commit
2c5dd549e3

+ 3 - 0
hdfs/CHANGES.txt

@@ -546,6 +546,9 @@ Trunk (unreleased changes)
     HDFS-2134. Move DecommissionManager to the blockmanagement package.
     (szetszwo)
 
+    HDFS-1977. Stop using StringUtils.stringifyException(). 
+    (Bharath Mundlapudi via jitendra)
+
   OPTIMIZATIONS
 
     HDFS-1458. Improve checkpoint performance by avoiding unnecessary image

+ 3 - 6
hdfs/src/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -101,7 +101,6 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
-import org.apache.hadoop.util.StringUtils;
 
 /********************************************************
  * DFSClient can connect to a Hadoop Filesystem and 
@@ -489,8 +488,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     try {
       return namenode.getPreferredBlockSize(f);
     } catch (IOException ie) {
-      LOG.warn("Problem getting block size: " + 
-          StringUtils.stringifyException(ie));
+      LOG.warn("Problem getting block size", ie);
       throw ie;
     }
   }
@@ -1578,9 +1576,8 @@ public class DFSClient implements FSConstants, java.io.Closeable {
     try {
       reportBadBlocks(lblocks);
     } catch (IOException ie) {
-      LOG.info("Found corruption while reading " + file 
-               + ".  Error repairing corrupt blocks.  Bad blocks remain. " 
-               + StringUtils.stringifyException(ie));
+      LOG.info("Found corruption while reading " + file
+          + ".  Error repairing corrupt blocks.  Bad blocks remain.", ie);
     }
   }
 

+ 4 - 6
hdfs/src/java/org/apache/hadoop/hdfs/DFSInputStream.java

@@ -47,7 +47,6 @@ import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.util.StringUtils;
 
 /****************************************************************
  * DFSInputStream provides bytes from a named file.  It handles 
@@ -499,7 +498,7 @@ public class DFSInputStream extends FSInputStream {
         if (!retryCurrentNode) {
           DFSClient.LOG.warn("Exception while reading from "
               + getCurrentBlock() + " of " + src + " from "
-              + currentNode + ": " + StringUtils.stringifyException(e));
+              + currentNode, e);
         }
         ioe = e;
       }
@@ -557,7 +556,7 @@ public class DFSInputStream extends FSInputStream {
           throw ce;            
         } catch (IOException e) {
           if (retries == 1) {
-            DFSClient.LOG.warn("DFS Read: " + StringUtils.stringifyException(e));
+            DFSClient.LOG.warn("DFS Read", e);
           }
           blockEnd = -1;
           if (currentNode != null) { addToDeadNodes(currentNode); }
@@ -931,9 +930,8 @@ public class DFSInputStream extends FSInputStream {
         } catch (IOException e) {//make following read to retry
           if(DFSClient.LOG.isDebugEnabled()) {
             DFSClient.LOG.debug("Exception while seek to " + targetPos
-                + " from " + getCurrentBlock() + " of " + src
-                + " from " + currentNode + ": "
-                + StringUtils.stringifyException(e));
+                + " from " + getCurrentBlock() + " of " + src + " from "
+                + currentNode, e);
           }
         }
       }

+ 4 - 8
hdfs/src/java/org/apache/hadoop/hdfs/DFSOutputStream.java

@@ -75,8 +75,6 @@ import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.StringUtils;
-
 
 
 /****************************************************************
@@ -548,8 +546,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
             Thread.sleep(artificialSlowdown); 
           }
         } catch (Throwable e) {
-          DFSClient.LOG.warn("DataStreamer Exception: " + 
-              StringUtils.stringifyException(e));
+          DFSClient.LOG.warn("DataStreamer Exception", e);
           if (e instanceof IOException) {
             setLastException((IOException)e);
           }
@@ -698,9 +695,8 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
               synchronized (dataQueue) {
                 dataQueue.notifyAll();
               }
-              DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception " + 
-                  " for block " + block +
-                  StringUtils.stringifyException(e));
+              DFSClient.LOG.warn("DFSOutputStream ResponseProcessor exception "
+                  + " for block " + block, e);
               responderClosed = true;
             }
           }
@@ -1101,7 +1097,7 @@ class DFSOutputStream extends FSOutputSummer implements Syncable {
                 throw e;
               } else {
                 --retries;
-                DFSClient.LOG.info(StringUtils.stringifyException(e));
+                DFSClient.LOG.info("Exception while adding a block", e);
                 if (System.currentTimeMillis() - localstart > 5000) {
                   DFSClient.LOG.info("Waiting for replication for "
                       + (System.currentTimeMillis() - localstart) / 1000

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

@@ -729,7 +729,7 @@ public class Balancer {
             blocksToReceive -= getBlockList();
             continue;
           } catch (IOException e) {
-            LOG.warn(StringUtils.stringifyException(e));
+            LOG.warn("Exception while getting block list", e);
             return;
           }
         } 
@@ -1553,7 +1553,7 @@ public class Balancer {
     try {
       System.exit(ToolRunner.run(null, new Cli(), args));
     } catch (Throwable e) {
-      LOG.error(StringUtils.stringifyException(e));
+      LOG.error("Exiting balancer due an exception", e);
       System.exit(-1);
     }
   }

+ 1 - 2
hdfs/src/java/org/apache/hadoop/hdfs/server/balancer/NameNodeConnector.java

@@ -52,7 +52,6 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * The class provides utilities for {@link Balancer} to access a NameNode
@@ -222,7 +221,7 @@ class NameNodeConnector {
         try {
           blockTokenSecretManager.setKeys(namenode.getBlockKeys());
         } catch (Exception e) {
-          LOG.error(StringUtils.stringifyException(e));
+          LOG.error("Failed to set keys", e);
         }
         try {
           Thread.sleep(keyUpdaterInterval);

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

@@ -49,7 +49,6 @@ import org.apache.hadoop.hdfs.server.common.GenerationStamp;
 import org.apache.hadoop.hdfs.server.datanode.FSDataset.FSVolume;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * Performs two types of scanning:
@@ -434,9 +433,8 @@ class BlockPoolSliceScanner {
           return;
         }
 
-        LOG.warn((second ? "Second " : "First ") + 
-                 "Verification failed for " + block + ". Exception : " +
-                 StringUtils.stringifyException(e));
+        LOG.warn((second ? "Second " : "First ") + "Verification failed for "
+            + block, e);
         
         if (second) {
           totalScanErrors++;
@@ -512,8 +510,7 @@ class BlockPoolSliceScanner {
         logReader[1] = log.getPreviousFileReader();
       }
     } catch (IOException e) {
-      LOG.warn("Could not read previous verification times : " +
-               StringUtils.stringifyException(e));
+      LOG.warn("Could not read previous verification times", e);
     }
     
     try {
@@ -645,8 +642,7 @@ class BlockPoolSliceScanner {
         }
       }
     } catch (RuntimeException e) {
-      LOG.warn("RuntimeException during BlockPoolScanner.scan() : " +
-               StringUtils.stringifyException(e));
+      LOG.warn("RuntimeException during BlockPoolScanner.scan()", e);
       throw e;
     } finally {
       cleanUp();
@@ -910,8 +906,7 @@ class BlockPoolSliceScanner {
         try {
           readNext();
         } catch (IOException e) {
-          LOG.info("Could not reade next line in LogHandler : " +
-                   StringUtils.stringifyException(e));
+          LOG.info("Could not read next line in LogHandler", e);
         }
         return curLine;
       }

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

@@ -49,7 +49,6 @@ import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.StringUtils;
 
 /** A class that receives a block and writes to its own disk, meanwhile
  * may copies it to another site. If a throttler is provided,
@@ -276,9 +275,8 @@ class BlockReceiver implements Closeable, FSConstants {
    */
   private void handleMirrorOutError(IOException ioe) throws IOException {
     String bpid = block.getBlockPoolId();
-    LOG.info(datanode.getDNRegistrationForBP(bpid) + ":Exception writing block " +
-             block + " to mirror " + mirrorAddr + "\n" +
-             StringUtils.stringifyException(ioe));
+    LOG.info(datanode.getDNRegistrationForBP(bpid)
+        + ":Exception writing block " + block + " to mirror " + mirrorAddr, ioe);
     if (Thread.interrupted()) { // shut down if the thread is interrupted
       throw ioe;
     } else { // encounter an error while writing to mirror

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

@@ -38,7 +38,6 @@ import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.net.SocketOutputStream;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.StringUtils;
 
 /**
  * Reads a block from the disk and sends it to a recipient.
@@ -328,9 +327,8 @@ class BlockSender implements java.io.Closeable, FSConstants {
       try {
         checksumIn.readFully(buf, checksumOff, checksumLen);
       } catch (IOException e) {
-        LOG.warn(" Could not read or failed to veirfy checksum for data" +
-                 " at offset " + offset + " for block " + block + " got : "
-                 + StringUtils.stringifyException(e));
+        LOG.warn(" Could not read or failed to veirfy checksum for data"
+            + " at offset " + offset + " for block " + block, e);
         IOUtils.closeStream(checksumIn);
         checksumIn = null;
         if (corruptChecksumOk) {

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

@@ -861,8 +861,8 @@ public class DataNode extends Configured
         /* One common reason is that NameNode could be in safe mode.
          * Should we keep on retrying in that case?
          */
-        LOG.warn("Failed to report bad block " + block + " to namenode : " +
-                 " Exception : " + StringUtils.stringifyException(e));
+        LOG.warn("Failed to report bad block " + block + " to namenode : "
+            + " Exception", e);
       }
       
     }
@@ -1111,12 +1111,11 @@ public class DataNode extends Configured
           if (UnregisteredNodeException.class.getName().equals(reClass) ||
               DisallowedDatanodeException.class.getName().equals(reClass) ||
               IncorrectVersionException.class.getName().equals(reClass)) {
-            LOG.warn("blockpool " + blockPoolId + " is shutting down: " + 
-                StringUtils.stringifyException(re));
+            LOG.warn("blockpool " + blockPoolId + " is shutting down", re);
             shouldServiceRun = false;
             return;
           }
-          LOG.warn(StringUtils.stringifyException(re));
+          LOG.warn("RemoteException in offerService", re);
           try {
             long sleepTime = Math.min(1000, heartBeatInterval);
             Thread.sleep(sleepTime);
@@ -1124,7 +1123,7 @@ public class DataNode extends Configured
             Thread.currentThread().interrupt();
           }
         } catch (IOException e) {
-          LOG.warn(StringUtils.stringifyException(e));
+          LOG.warn("IOException in offerService", e);
         }
       } // while (shouldRun && shouldServiceRun)
     } // offerService
@@ -1241,18 +1240,18 @@ public class DataNode extends Configured
             startDistributedUpgradeIfNeeded();
             offerService();
           } catch (Exception ex) {
-            LOG.error("Exception: " + StringUtils.stringifyException(ex));
+            LOG.error("Exception in BPOfferService", ex);
             if (shouldRun && shouldServiceRun) {
               try {
                 Thread.sleep(5000);
               } catch (InterruptedException ie) {
-                LOG.warn("Received exception: ", ie);
+                LOG.warn("Received exception", ie);
               }
             }
           }
         }
       } catch (Throwable ex) {
-        LOG.warn("Unexpected exception ", ex);
+        LOG.warn("Unexpected exception", ex);
       } finally {
         LOG.warn(bpRegistration + " ending block pool service for: " 
             + blockPoolId);
@@ -1737,8 +1736,7 @@ public class DataNode extends Configured
       try {
         nn.errorReport(bpos.bpRegistration, dpError, errMsgr);
       } catch(IOException e) {
-        LOG.warn("Error reporting disk failure to NameNode: " + 
-            StringUtils.stringifyException(e));
+        LOG.warn("Error reporting disk failure to NameNode", e);
       }
     }
     
@@ -2007,8 +2005,9 @@ public class DataNode extends Configured
           }
         }
       } catch (IOException ie) {
-        LOG.warn(bpReg + ":Failed to transfer " + b + " to " + targets[0].getName()
-            + " got " + StringUtils.stringifyException(ie));
+        LOG.warn(
+            bpReg + ":Failed to transfer " + b + " to " + targets[0].getName()
+                + " got ", ie);
         // check if there are any disk problem
         checkDiskError();
         
@@ -2279,7 +2278,7 @@ public class DataNode extends Configured
       if (datanode != null)
         datanode.join();
     } catch (Throwable e) {
-      LOG.error(StringUtils.stringifyException(e));
+      LOG.error("Exception in secureMain", e);
       System.exit(-1);
     } finally {
       // We need to add System.exit here because either shutdown was called or

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

@@ -173,8 +173,8 @@ public class DataStorage extends Storage {
         }
       } catch (IOException ioe) {
         sd.unlock();
-        LOG.warn("Ignoring storage directory "+ dataDir
-        		+ " due to an exception: " + StringUtils.stringifyException(ioe));
+        LOG.warn("Ignoring storage directory " + dataDir
+            + " due to an exception", ioe);
         //continue with other good dirs
         continue;
       }

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

@@ -62,7 +62,6 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
-import org.apache.hadoop.util.StringUtils;
 
 import com.google.protobuf.ByteString;
 
@@ -268,10 +267,8 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
       /* What exactly should we do here?
        * Earlier version shutdown() datanode if there is disk error.
        */
-      LOG.warn(dnR +  ":Got exception while serving " + 
-          block + " to " +
-                remoteAddress + ":\n" + 
-                StringUtils.stringifyException(ioe) );
+      LOG.warn(dnR + ":Got exception while serving " + block + " to "
+          + remoteAddress, ioe);
       throw ioe;
     } finally {
       IOUtils.closeStream(blockSender);
@@ -424,8 +421,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
           } else {
             LOG.info(datanode + ":Exception transfering block " +
                      block + " to mirror " + mirrorNode +
-                     ". continuing without the mirror.\n" +
-                     StringUtils.stringifyException(e));
+                     ". continuing without the mirror.", e);
           }
         }
       }

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

@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.server.balancer.Balancer;
 import org.apache.hadoop.hdfs.util.DataTransferThrottler;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.util.Daemon;
-import org.apache.hadoop.util.StringUtils;
 
 
 /**
@@ -157,7 +156,7 @@ class DataXceiverServer implements Runnable, FSConstants {
       ss.close();
     } catch (IOException ie) {
       LOG.warn(datanode.getMachineName()
-          + ":DataXceiverServer: Close exception due to: ", ie);
+          + " :DataXceiverServer: close exception", ie);
     }
   }
   
@@ -167,8 +166,7 @@ class DataXceiverServer implements Runnable, FSConstants {
     try {
       this.ss.close();
     } catch (IOException ie) {
-      LOG.warn(datanode.getMachineName() + ":DataXceiverServer.kill(): "
-                              + StringUtils.stringifyException(ie));
+      LOG.warn(datanode.getMachineName() + ":DataXceiverServer.kill(): ", ie);
     }
 
     // close all the sockets that were accepted earlier

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

@@ -62,7 +62,6 @@ import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlo
 import org.apache.hadoop.metrics2.util.MBeans;
 import org.apache.hadoop.util.DataChecksum;
 import org.apache.hadoop.util.DiskChecker;
-import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.util.DiskChecker.DiskOutOfSpaceException;
 import org.apache.hadoop.util.ReflectionUtils;
@@ -2377,8 +2376,7 @@ public class FSDataset implements FSConstants, FSDatasetInterface {
       try {
         datanode.reportBadBlocks(new ExtendedBlock(bpid, corruptBlock));  
       } catch (IOException e) {
-        DataNode.LOG.warn("Failed to repot bad block " + corruptBlock
-            + "Exception:" + StringUtils.stringifyException(e));
+        DataNode.LOG.warn("Failed to repot bad block " + corruptBlock, e);
       }
     }
   }

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

@@ -24,7 +24,6 @@ import org.apache.hadoop.hdfs.server.common.UpgradeObject;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol;
 import org.apache.hadoop.hdfs.server.protocol.NamespaceInfo;
 import org.apache.hadoop.hdfs.server.protocol.UpgradeCommand;
-import org.apache.hadoop.util.StringUtils;
 import java.io.IOException;
 import java.net.SocketTimeoutException;
 
@@ -110,7 +109,7 @@ public abstract class UpgradeObjectDatanode extends UpgradeObject implements Run
       try {
         doUpgrade();
       } catch(Exception e) {
-        DataNode.LOG.error(StringUtils.stringifyException(e));
+        DataNode.LOG.error("Exception in doUpgrade", e);
       }
       break;
     }
@@ -129,7 +128,7 @@ public abstract class UpgradeObjectDatanode extends UpgradeObject implements Run
       if(upgradeManager != null)
         upgradeManager.completeUpgrade();
     } catch(IOException e) {
-      DataNode.LOG.error(StringUtils.stringifyException(e));
+      DataNode.LOG.error("Exception in completeUpgrade", e);
     }
   }
 

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

@@ -3216,7 +3216,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
             lastBlockKeyUpdate = now;
           }
         } catch (Exception e) {
-          FSNamesystem.LOG.error(StringUtils.stringifyException(e));
+          FSNamesystem.LOG.error("Exception while checking heartbeat", e);
         }
         try {
           Thread.sleep(5000);  // 5 seconds
@@ -4294,7 +4294,7 @@ public class FSNamesystem implements FSConstants, FSNamesystemMBean,
         try {
           needUpgrade = startDistributedUpgradeIfNeeded();
         } catch(IOException e) {
-          FSNamesystem.LOG.error(StringUtils.stringifyException(e));
+          FSNamesystem.LOG.error("IOException in startDistributedUpgradeIfNeeded", e);
         }
         if(needUpgrade) {
           // switch to manual safe mode

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

@@ -691,7 +691,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
     try {
       if (httpServer != null) httpServer.stop();
     } catch (Exception e) {
-      LOG.error(StringUtils.stringifyException(e));
+      LOG.error("Exception while stopping httpserver", e);
     }
     if(namesystem != null) namesystem.close();
     if(emptier != null) emptier.interrupt();
@@ -1702,7 +1702,7 @@ public class NameNode implements NamenodeProtocols, FSConstants {
       if (namenode != null)
         namenode.join();
     } catch (Throwable e) {
-      LOG.error(StringUtils.stringifyException(e));
+      LOG.error("Exception in namenode join", e);
       System.exit(-1);
     }
   }

+ 4 - 6
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/SecondaryNameNode.java

@@ -254,7 +254,7 @@ public class SecondaryNameNode implements Runnable {
     try {
       if (checkpointImage != null) checkpointImage.close();
     } catch(IOException e) {
-      LOG.warn(StringUtils.stringifyException(e));
+      LOG.warn("Exception while closing CheckpointStorage", e);
     }
   }
 
@@ -264,7 +264,7 @@ public class SecondaryNameNode implements Runnable {
       try { 
         ugi = UserGroupInformation.getLoginUser();
       } catch (IOException e) {
-        LOG.error(StringUtils.stringifyException(e));
+        LOG.error("Exception while getting login user", e);
         e.printStackTrace();
         Runtime.getRuntime().exit(-1);
       }
@@ -316,13 +316,11 @@ public class SecondaryNameNode implements Runnable {
           lastCheckpointTime = now;
         }
       } catch (IOException e) {
-        LOG.error("Exception in doCheckpoint: ");
-        LOG.error(StringUtils.stringifyException(e));
+        LOG.error("Exception in doCheckpoint", e);
         e.printStackTrace();
         checkpointImage.getStorage().imageDigest = null;
       } catch (Throwable e) {
-        LOG.error("Throwable Exception in doCheckpoint: ");
-        LOG.error(StringUtils.stringifyException(e));
+        LOG.error("Throwable Exception in doCheckpoint", e);
         e.printStackTrace();
         Runtime.getRuntime().exit(-1);
       }