HADOOP-10106. Incorrect thread name in RPC log messages. Contributed by Ming Ma.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1551369 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jing Zhao 2013-12-16 21:57:47 +00:00
parent 5a1b33507b
commit 763f073f41
2 changed files with 24 additions and 22 deletions

View File

@ -470,6 +470,8 @@ Release 2.4.0 - UNRELEASED
HADOOP-10058. TestMetricsSystemImpl#testInitFirstVerifyStopInvokedImmediately HADOOP-10058. TestMetricsSystemImpl#testInitFirstVerifyStopInvokedImmediately
fails on trunk (Chen He via jeagles) fails on trunk (Chen He via jeagles)
HADOOP-10106. Incorrect thread name in RPC log messages. (Ming Ma via jing9)
Release 2.3.0 - UNRELEASED Release 2.3.0 - UNRELEASED
INCOMPATIBLE CHANGES INCOMPATIBLE CHANGES

View File

@ -551,14 +551,14 @@ private class Reader extends Thread {
@Override @Override
public void run() { public void run() {
LOG.info("Starting " + getName()); LOG.info("Starting " + Thread.currentThread().getName());
try { try {
doRunLoop(); doRunLoop();
} finally { } finally {
try { try {
readSelector.close(); readSelector.close();
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.error("Error closing read selector in " + this.getName(), ioe); LOG.error("Error closing read selector in " + Thread.currentThread().getName(), ioe);
} }
} }
} }
@ -589,7 +589,7 @@ private synchronized void doRunLoop() {
} }
} catch (InterruptedException e) { } catch (InterruptedException e) {
if (running) { // unexpected -- log it if (running) { // unexpected -- log it
LOG.info(getName() + " unexpectedly interrupted", e); LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
} }
} catch (IOException ex) { } catch (IOException ex) {
LOG.error("Error in Reader", ex); LOG.error("Error in Reader", ex);
@ -620,7 +620,7 @@ void shutdown() {
@Override @Override
public void run() { public void run() {
LOG.info(getName() + ": starting"); LOG.info(Thread.currentThread().getName() + ": starting");
SERVER.set(Server.this); SERVER.set(Server.this);
connectionManager.startIdleScan(); connectionManager.startIdleScan();
while (running) { while (running) {
@ -652,7 +652,7 @@ public void run() {
closeCurrentConnection(key, e); closeCurrentConnection(key, e);
} }
} }
LOG.info("Stopping " + this.getName()); LOG.info("Stopping " + Thread.currentThread().getName());
synchronized (this) { synchronized (this) {
try { try {
@ -710,14 +710,14 @@ void doRead(SelectionKey key) throws InterruptedException {
try { try {
count = c.readAndProcess(); count = c.readAndProcess();
} catch (InterruptedException ieo) { } catch (InterruptedException ieo) {
LOG.info(getName() + ": readAndProcess caught InterruptedException", ieo); LOG.info(Thread.currentThread().getName() + ": readAndProcess caught InterruptedException", ieo);
throw ieo; throw ieo;
} catch (Exception e) { } catch (Exception e) {
// a WrappedRpcServerException is an exception that has been sent // a WrappedRpcServerException is an exception that has been sent
// to the client, so the stacktrace is unnecessary; any other // to the client, so the stacktrace is unnecessary; any other
// exceptions are unexpected internal server errors and thus the // exceptions are unexpected internal server errors and thus the
// stacktrace should be logged // stacktrace should be logged
LOG.info(getName() + ": readAndProcess from client " + LOG.info(Thread.currentThread().getName() + ": readAndProcess from client " +
c.getHostAddress() + " threw exception [" + e + "]", c.getHostAddress() + " threw exception [" + e + "]",
(e instanceof WrappedRpcServerException) ? null : e); (e instanceof WrappedRpcServerException) ? null : e);
count = -1; //so that the (count < 0) block is executed count = -1; //so that the (count < 0) block is executed
@ -740,7 +740,7 @@ synchronized void doStop() {
try { try {
acceptChannel.socket().close(); acceptChannel.socket().close();
} catch (IOException e) { } catch (IOException e) {
LOG.info(getName() + ":Exception in closing listener socket. " + e); LOG.info(Thread.currentThread().getName() + ":Exception in closing listener socket. " + e);
} }
} }
for (Reader r : readers) { for (Reader r : readers) {
@ -773,16 +773,16 @@ private class Responder extends Thread {
@Override @Override
public void run() { public void run() {
LOG.info(getName() + ": starting"); LOG.info(Thread.currentThread().getName() + ": starting");
SERVER.set(Server.this); SERVER.set(Server.this);
try { try {
doRunLoop(); doRunLoop();
} finally { } finally {
LOG.info("Stopping " + this.getName()); LOG.info("Stopping " + Thread.currentThread().getName());
try { try {
writeSelector.close(); writeSelector.close();
} catch (IOException ioe) { } catch (IOException ioe) {
LOG.error("Couldn't close write selector in " + this.getName(), ioe); LOG.error("Couldn't close write selector in " + Thread.currentThread().getName(), ioe);
} }
} }
} }
@ -803,7 +803,7 @@ private void doRunLoop() {
doAsyncWrite(key); doAsyncWrite(key);
} }
} catch (IOException e) { } catch (IOException e) {
LOG.info(getName() + ": doAsyncWrite threw exception " + e); LOG.info(Thread.currentThread().getName() + ": doAsyncWrite threw exception " + e);
} }
} }
long now = Time.now(); long now = Time.now();
@ -918,7 +918,7 @@ private boolean processResponse(LinkedList<Call> responseQueue,
call = responseQueue.removeFirst(); call = responseQueue.removeFirst();
SocketChannel channel = call.connection.channel; SocketChannel channel = call.connection.channel;
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": responding to " + call); LOG.debug(Thread.currentThread().getName() + ": responding to " + call);
} }
// //
// Send as much data as we can in the non-blocking fashion // Send as much data as we can in the non-blocking fashion
@ -937,7 +937,7 @@ private boolean processResponse(LinkedList<Call> responseQueue,
done = false; // more calls pending to be sent. done = false; // more calls pending to be sent.
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": responding to " + call LOG.debug(Thread.currentThread().getName() + ": responding to " + call
+ " Wrote " + numBytes + " bytes."); + " Wrote " + numBytes + " bytes.");
} }
} else { } else {
@ -965,7 +965,7 @@ private boolean processResponse(LinkedList<Call> responseQueue,
} }
} }
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": responding to " + call LOG.debug(Thread.currentThread().getName() + ": responding to " + call
+ " Wrote partial " + numBytes + " bytes."); + " Wrote partial " + numBytes + " bytes.");
} }
} }
@ -973,7 +973,7 @@ private boolean processResponse(LinkedList<Call> responseQueue,
} }
} finally { } finally {
if (error && call != null) { if (error && call != null) {
LOG.warn(getName()+", call " + call + ": output error"); LOG.warn(Thread.currentThread().getName()+", call " + call + ": output error");
done = true; // error. no more data for this channel. done = true; // error. no more data for this channel.
closeConnection(call.connection); closeConnection(call.connection);
} }
@ -2011,7 +2011,7 @@ public Handler(int instanceNumber) {
@Override @Override
public void run() { public void run() {
LOG.debug(getName() + ": starting"); LOG.debug(Thread.currentThread().getName() + ": starting");
SERVER.set(Server.this); SERVER.set(Server.this);
ByteArrayOutputStream buf = ByteArrayOutputStream buf =
new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE); new ByteArrayOutputStream(INITIAL_RESP_BUF_SIZE);
@ -2019,7 +2019,7 @@ public void run() {
try { try {
final Call call = callQueue.take(); // pop the queue; maybe blocked here final Call call = callQueue.take(); // pop the queue; maybe blocked here
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
LOG.debug(getName() + ": " + call + " for RpcKind " + call.rpcKind); LOG.debug(Thread.currentThread().getName() + ": " + call + " for RpcKind " + call.rpcKind);
} }
String errorClass = null; String errorClass = null;
String error = null; String error = null;
@ -2052,7 +2052,7 @@ public Writable run() throws Exception {
if (e instanceof UndeclaredThrowableException) { if (e instanceof UndeclaredThrowableException) {
e = e.getCause(); e = e.getCause();
} }
String logMsg = getName() + ", call " + call + ": error: " + e; String logMsg = Thread.currentThread().getName() + ", call " + call + ": error: " + e;
if (e instanceof RuntimeException || e instanceof Error) { if (e instanceof RuntimeException || e instanceof Error) {
// These exception types indicate something is probably wrong // These exception types indicate something is probably wrong
// on the server side, as opposed to just a normal exceptional // on the server side, as opposed to just a normal exceptional
@ -2101,13 +2101,13 @@ public Writable run() throws Exception {
} }
} catch (InterruptedException e) { } catch (InterruptedException e) {
if (running) { // unexpected -- log it if (running) { // unexpected -- log it
LOG.info(getName() + " unexpectedly interrupted", e); LOG.info(Thread.currentThread().getName() + " unexpectedly interrupted", e);
} }
} catch (Exception e) { } catch (Exception e) {
LOG.info(getName() + " caught an exception", e); LOG.info(Thread.currentThread().getName() + " caught an exception", e);
} }
} }
LOG.debug(getName() + ": exiting"); LOG.debug(Thread.currentThread().getName() + ": exiting");
} }
} }