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
This commit is contained in:
Jitendra Nath Pandey 2011-07-13 01:11:01 +00:00
parent faf8747e3e
commit 2c5dd549e3
18 changed files with 57 additions and 85 deletions

View File

@ -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

View File

@ -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);
}
}

View File

@ -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);
}
}
}

View File

@ -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

View File

@ -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);
}
}

View File

@ -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);

View File

@ -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;
}

View File

@ -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

View File

@ -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) {

View File

@ -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

View File

@ -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;
}

View File

@ -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);
}
}
}

View File

@ -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

View File

@ -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);
}
}
}

View File

@ -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);
}
}

View File

@ -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

View File

@ -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);
}
}

View File

@ -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);
}