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:
parent
faf8747e3e
commit
2c5dd549e3
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue