HDFS-1568. Improve the log messages in DataXceiver. Contributed by Joey Echeverria
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1138098 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
92aba1eeca
commit
3f190b3e1a
@ -516,6 +516,9 @@ Trunk (unreleased changes)
|
|||||||
|
|
||||||
HDFS-420. Fuse-dfs should cache fs handles. (Brian Bockelman and eli)
|
HDFS-420. Fuse-dfs should cache fs handles. (Brian Bockelman and eli)
|
||||||
|
|
||||||
|
HDFS-1568. Improve the log messages in DataXceiver. (Joey Echeverria via
|
||||||
|
szetszwo)
|
||||||
|
|
||||||
OPTIMIZATIONS
|
OPTIMIZATIONS
|
||||||
|
|
||||||
HDFS-1458. Improve checkpoint performance by avoiding unnecessary image
|
HDFS-1458. Improve checkpoint performance by avoiding unnecessary image
|
||||||
|
@ -129,6 +129,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
|
|
||||||
DataInputStream in=null;
|
DataInputStream in=null;
|
||||||
int opsProcessed = 0;
|
int opsProcessed = 0;
|
||||||
|
Op op = null;
|
||||||
try {
|
try {
|
||||||
in = new DataInputStream(
|
in = new DataInputStream(
|
||||||
new BufferedInputStream(NetUtils.getInputStream(s),
|
new BufferedInputStream(NetUtils.getInputStream(s),
|
||||||
@ -139,7 +140,6 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
// This optimistic behaviour allows the other end to reuse connections.
|
// This optimistic behaviour allows the other end to reuse connections.
|
||||||
// Setting keepalive timeout to 0 disable this behavior.
|
// Setting keepalive timeout to 0 disable this behavior.
|
||||||
do {
|
do {
|
||||||
Op op;
|
|
||||||
try {
|
try {
|
||||||
if (opsProcessed != 0) {
|
if (opsProcessed != 0) {
|
||||||
assert socketKeepaliveTimeout > 0;
|
assert socketKeepaliveTimeout > 0;
|
||||||
@ -180,8 +180,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
++opsProcessed;
|
++opsProcessed;
|
||||||
} while (!s.isClosed() && socketKeepaliveTimeout > 0);
|
} while (!s.isClosed() && socketKeepaliveTimeout > 0);
|
||||||
} catch (Throwable t) {
|
} catch (Throwable t) {
|
||||||
LOG.error(datanode.getMachineName() + ":DataXceiver, at " +
|
LOG.error(datanode.getMachineName() + ":DataXceiver error processing " +
|
||||||
s.toString(), t);
|
((op == null) ? "unknown" : op.name()) + " operation " +
|
||||||
|
" src: " + remoteAddress +
|
||||||
|
" dest: " + localAddress, t);
|
||||||
} finally {
|
} finally {
|
||||||
if (LOG.isDebugEnabled()) {
|
if (LOG.isDebugEnabled()) {
|
||||||
LOG.debug(datanode.getMachineName() + ":Number of active connections is: "
|
LOG.debug(datanode.getMachineName() + ":Number of active connections is: "
|
||||||
@ -218,7 +220,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
"%d", "HDFS_READ", clientName, "%d",
|
"%d", "HDFS_READ", clientName, "%d",
|
||||||
dnR.getStorageID(), block, "%d")
|
dnR.getStorageID(), block, "%d")
|
||||||
: dnR + " Served block " + block + " to " +
|
: dnR + " Served block " + block + " to " +
|
||||||
s.getInetAddress();
|
remoteAddress;
|
||||||
|
|
||||||
updateCurrentThreadName("Sending block " + block);
|
updateCurrentThreadName("Sending block " + block);
|
||||||
try {
|
try {
|
||||||
@ -226,6 +228,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
blockSender = new BlockSender(block, startOffset, length,
|
blockSender = new BlockSender(block, startOffset, length,
|
||||||
true, true, false, datanode, clientTraceFmt);
|
true, true, false, datanode, clientTraceFmt);
|
||||||
} catch(IOException e) {
|
} catch(IOException e) {
|
||||||
|
LOG.info("opReadBlock " + block + " received exception " + e);
|
||||||
sendResponse(s, ERROR, datanode.socketWriteTimeout);
|
sendResponse(s, ERROR, datanode.socketWriteTimeout);
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
@ -256,6 +259,10 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
datanode.metrics.incrBytesRead((int) read);
|
datanode.metrics.incrBytesRead((int) read);
|
||||||
datanode.metrics.incrBlocksRead();
|
datanode.metrics.incrBlocksRead();
|
||||||
} catch ( SocketException ignored ) {
|
} catch ( SocketException ignored ) {
|
||||||
|
if (LOG.isTraceEnabled()) {
|
||||||
|
LOG.trace(dnR + ":Ignoring exception while serving " + block + " to " +
|
||||||
|
remoteAddress, ignored);
|
||||||
|
}
|
||||||
// Its ok for remote side to close the connection anytime.
|
// Its ok for remote side to close the connection anytime.
|
||||||
datanode.metrics.incrBlocksRead();
|
datanode.metrics.incrBlocksRead();
|
||||||
IOUtils.closeStream(out);
|
IOUtils.closeStream(out);
|
||||||
@ -265,7 +272,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
*/
|
*/
|
||||||
LOG.warn(dnR + ":Got exception while serving " +
|
LOG.warn(dnR + ":Got exception while serving " +
|
||||||
block + " to " +
|
block + " to " +
|
||||||
s.getInetAddress() + ":\n" +
|
remoteAddress + ":\n" +
|
||||||
StringUtils.stringifyException(ioe) );
|
StringUtils.stringifyException(ioe) );
|
||||||
throw ioe;
|
throw ioe;
|
||||||
} finally {
|
} finally {
|
||||||
@ -412,6 +419,8 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
IOUtils.closeSocket(mirrorSock);
|
IOUtils.closeSocket(mirrorSock);
|
||||||
mirrorSock = null;
|
mirrorSock = null;
|
||||||
if (isClient) {
|
if (isClient) {
|
||||||
|
LOG.error(datanode + ":Exception transfering block " +
|
||||||
|
block + " to mirror " + mirrorNode + ": " + e);
|
||||||
throw e;
|
throw e;
|
||||||
} else {
|
} else {
|
||||||
LOG.info(datanode + ":Exception transfering block " +
|
LOG.info(datanode + ":Exception transfering block " +
|
||||||
@ -473,7 +482,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
|
|
||||||
|
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
LOG.info("writeBlock " + block + " received exception " + ioe);
|
LOG.info("opWriteBlock " + block + " received exception " + ioe);
|
||||||
throw ioe;
|
throw ioe;
|
||||||
} finally {
|
} finally {
|
||||||
// close all opened streams
|
// close all opened streams
|
||||||
@ -619,6 +628,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
LOG.info("Copied block " + block + " to " + s.getRemoteSocketAddress());
|
LOG.info("Copied block " + block + " to " + s.getRemoteSocketAddress());
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
isOpSuccess = false;
|
isOpSuccess = false;
|
||||||
|
LOG.info("opCopyBlock " + block + " received exception " + ioe);
|
||||||
throw ioe;
|
throw ioe;
|
||||||
} finally {
|
} finally {
|
||||||
dataXceiverServer.balanceThrottler.release();
|
dataXceiverServer.balanceThrottler.release();
|
||||||
@ -724,6 +734,7 @@ class DataXceiver extends Receiver implements Runnable, FSConstants {
|
|||||||
|
|
||||||
} catch (IOException ioe) {
|
} catch (IOException ioe) {
|
||||||
opStatus = ERROR;
|
opStatus = ERROR;
|
||||||
|
LOG.info("opReplaceBlock " + block + " received exception " + ioe);
|
||||||
throw ioe;
|
throw ioe;
|
||||||
} finally {
|
} finally {
|
||||||
// receive the last byte that indicates the proxy released its thread resource
|
// receive the last byte that indicates the proxy released its thread resource
|
||||||
|
Loading…
x
Reference in New Issue
Block a user