HBASE-4640 Catch ClosedChannelException and document it
git-svn-id: https://svn.apache.org/repos/asf/hbase/trunk@1245730 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
5b542b7921
commit
6fa8f6a042
|
@ -898,6 +898,15 @@ ERROR org.apache.hadoop.hbase.regionserver.HRegionServer: ZooKeeper session expi
|
||||||
symlink them into place and the message should go away.
|
symlink them into place and the message should go away.
|
||||||
</para>
|
</para>
|
||||||
</section>
|
</section>
|
||||||
|
<section xml:id="trouble.rs.runtime.client_went_away">
|
||||||
|
<title>Server handler X on 60020 caught: java.nio.channels.ClosedChannelException</title>
|
||||||
|
<para>
|
||||||
|
If you see this type of message it means that the region server was trying to read/send data from/to a client but
|
||||||
|
it already went away. Typical causes for this are if the client was killed (you see a storm of messages like this when a MapReduce
|
||||||
|
job is killed or fails) or if the client receives a SocketTimeoutException. It's harmless, but you should consider digging in
|
||||||
|
a bit more if you aren't doing something to trigger them.
|
||||||
|
</para>
|
||||||
|
</section>
|
||||||
|
|
||||||
</section>
|
</section>
|
||||||
<section xml:id="trouble.rs.shutdown">
|
<section xml:id="trouble.rs.shutdown">
|
||||||
|
|
|
@ -1378,6 +1378,11 @@ public abstract class HBaseServer implements RpcServer {
|
||||||
// rethrow if no handler
|
// rethrow if no handler
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
|
} catch (ClosedChannelException cce) {
|
||||||
|
LOG.warn(getName() + " caught a ClosedChannelException, " +
|
||||||
|
"this means that the server was processing a " +
|
||||||
|
"request but the client went away. The error message was: " +
|
||||||
|
cce.getMessage());
|
||||||
} catch (Exception e) {
|
} catch (Exception e) {
|
||||||
LOG.warn(getName() + " caught: " +
|
LOG.warn(getName() + " caught: " +
|
||||||
StringUtils.stringifyException(e));
|
StringUtils.stringifyException(e));
|
||||||
|
|
Loading…
Reference in New Issue