Skip to content

Commit

Permalink
[FLINK-17501][qs] Improve logging in AbstractServerHandler#channelRea…
Browse files Browse the repository at this point in the history
…d(ChannelHandlerContext, Object)

Log errors as soon as possible, i.e., before serializing and sending an
error response to the client. This mitigates the problem that errors
might be masked by other errors that are caused by the error handling
logic.

This closes apache#11984.
  • Loading branch information
GJL committed May 6, 2020
1 parent fdd128a commit c2540e4
Showing 1 changed file with 7 additions and 2 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -55,6 +55,8 @@ public abstract class AbstractServerHandler<REQ extends MessageBody, RESP extend

private static final Logger LOG = LoggerFactory.getLogger(AbstractServerHandler.class);

private static final long UNKNOWN_REQUEST_ID = -1;

/** The owning server of this handler. */
private final AbstractServerBase<REQ, RESP> server;

Expand Down Expand Up @@ -101,7 +103,7 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception {
@Override
public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception {
REQ request = null;
long requestId = -1L;
long requestId = UNKNOWN_REQUEST_ID;

try {
final ByteBuf buf = (ByteBuf) msg;
Expand Down Expand Up @@ -135,6 +137,10 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
ctx.writeAndFlush(failure);
}
} catch (Throwable t) {
LOG.error("Error while handling request with ID [{}]",
requestId == UNKNOWN_REQUEST_ID ? "unknown" : requestId,
t);

final String stringifiedCause = ExceptionUtils.stringifyException(t);

String errMsg;
Expand All @@ -148,7 +154,6 @@ public void channelRead(ChannelHandlerContext ctx, Object msg) throws Exception
err = MessageSerializer.serializeServerFailure(ctx.alloc(), new RuntimeException(errMsg));
}

LOG.debug(errMsg);
ctx.writeAndFlush(err);

} finally {
Expand Down

0 comments on commit c2540e4

Please sign in to comment.