Skip to content

Commit cbc705d

Browse files
Graham Sandersonthobbs
authored andcommitted
Log unclean client disconnect excs at DEBUG
Patch by Graham Sanderson; reviewed by Tyler Hobbs for CASSANDRA-7849
1 parent 549f035 commit cbc705d

File tree

3 files changed

+87
-7
lines changed

3 files changed

+87
-7
lines changed

CHANGES.txt

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,6 @@
11
2.0.11:
2+
* Log exceptions related to unclean native protocol client disconnects
3+
at DEBUG or INFO (CASSANDRA-7849)
24
* Allow permissions cache to be set via JMX (CASSANDRA-7698)
35
* Include schema_triggers CF in readable system resources (CASSANDRA-7967)
46
* Fix RowIndexEntry to report correct serializedSize (CASSANDRA-7948)

src/java/org/apache/cassandra/transport/Message.java

Lines changed: 71 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,13 @@
1717
*/
1818
package org.apache.cassandra.transport;
1919

20+
import java.io.IOException;
2021
import java.util.EnumSet;
22+
import java.util.Set;
2123
import java.util.UUID;
2224

25+
import com.google.common.base.Predicate;
26+
import com.google.common.collect.ImmutableSet;
2327
import org.jboss.netty.buffer.ChannelBuffer;
2428
import org.jboss.netty.buffer.ChannelBuffers;
2529
import org.jboss.netty.channel.*;
@@ -38,6 +42,17 @@ public abstract class Message
3842
{
3943
protected static final Logger logger = LoggerFactory.getLogger(Message.class);
4044

45+
/**
46+
* When we encounter an unexpected IOException we look for these {@link Throwable#getMessage() messages}
47+
* (because we have no better way to distinguish) and log them at DEBUG rather than INFO, since they
48+
* are generally caused by unclean client disconnects rather than an actual problem.
49+
*/
50+
private static final Set<String> ioExceptionsAtDebugLevel = ImmutableSet.<String>builder().
51+
add("Connection reset by peer").
52+
add("Broken pipe").
53+
add("Connection timed out").
54+
build();
55+
4156
public interface Codec<M extends Message> extends CBCodec<M> {}
4257

4358
public enum Direction
@@ -315,7 +330,7 @@ public void messageReceived(ChannelHandlerContext ctx, MessageEvent e)
315330
catch (Throwable ex)
316331
{
317332
// Don't let the exception propagate to exceptionCaught() if we can help it so that we can assign the right streamID.
318-
ctx.getChannel().write(ErrorMessage.fromException(ex).setStreamId(request.getStreamId()));
333+
ctx.getChannel().write(ErrorMessage.fromException(ex, new UnexpectedChannelExceptionHandler(ctx.getChannel(), true)).setStreamId(request.getStreamId()));
319334
}
320335
}
321336

@@ -325,7 +340,7 @@ public void exceptionCaught(final ChannelHandlerContext ctx, ExceptionEvent e)
325340
{
326341
if (ctx.getChannel().isOpen())
327342
{
328-
ChannelFuture future = ctx.getChannel().write(ErrorMessage.fromException(e.getCause()));
343+
ChannelFuture future = ctx.getChannel().write(ErrorMessage.fromException(e.getCause(), new UnexpectedChannelExceptionHandler(ctx.getChannel(), false)));
329344
// On protocol exception, close the channel as soon as the message have been sent
330345
if (e.getCause() instanceof ProtocolException)
331346
{
@@ -338,4 +353,58 @@ public void operationComplete(ChannelFuture future) {
338353
}
339354
}
340355
}
356+
357+
/**
358+
* Include the channel info in the logged information for unexpected errors, and (if {@link #alwaysLogAtError} is
359+
* false then choose the log level based on the type of exception (some are clearly client issues and shouldn't be
360+
* logged at server ERROR level)
361+
*/
362+
static final class UnexpectedChannelExceptionHandler implements Predicate<Throwable>
363+
{
364+
private final Channel channel;
365+
private final boolean alwaysLogAtError;
366+
367+
UnexpectedChannelExceptionHandler(Channel channel, boolean alwaysLogAtError)
368+
{
369+
this.channel = channel;
370+
this.alwaysLogAtError = alwaysLogAtError;
371+
}
372+
373+
@Override
374+
public boolean apply(Throwable exception)
375+
{
376+
String message;
377+
try
378+
{
379+
message = "Unexpected exception during request; channel = " + channel;
380+
}
381+
catch (Exception ignore)
382+
{
383+
// We don't want to make things worse if String.valueOf() throws an exception
384+
message = "Unexpected exception during request; channel = <unprintable>";
385+
}
386+
387+
if (!alwaysLogAtError && exception instanceof IOException)
388+
{
389+
if (ioExceptionsAtDebugLevel.contains(exception.getMessage()))
390+
{
391+
// Likely unclean client disconnects
392+
logger.debug(message, exception);
393+
}
394+
else
395+
{
396+
// Generally unhandled IO exceptions are network issues, not actual ERRORS
397+
logger.info(message, exception);
398+
}
399+
}
400+
else
401+
{
402+
// Anything else is probably a bug in server of client binary protocol handling
403+
logger.error(message, exception);
404+
}
405+
406+
// We handled the exception.
407+
return true;
408+
}
409+
}
341410
}

src/java/org/apache/cassandra/transport/messages/ErrorMessage.java

Lines changed: 14 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -17,10 +17,8 @@
1717
*/
1818
package org.apache.cassandra.transport.messages;
1919

20-
import java.nio.ByteBuffer;
21-
20+
import com.google.common.base.Predicate;
2221
import org.jboss.netty.buffer.ChannelBuffer;
23-
import org.jboss.netty.buffer.ChannelBuffers;
2422
import org.slf4j.Logger;
2523
import org.slf4j.LoggerFactory;
2624

@@ -31,7 +29,6 @@
3129
import org.apache.cassandra.transport.Message;
3230
import org.apache.cassandra.transport.ProtocolException;
3331
import org.apache.cassandra.transport.ServerError;
34-
import org.apache.cassandra.utils.ByteBufferUtil;
3532
import org.apache.cassandra.utils.MD5Digest;
3633

3734
/**
@@ -207,6 +204,16 @@ private ErrorMessage(TransportException error, int streamId)
207204
}
208205

209206
public static ErrorMessage fromException(Throwable e)
207+
{
208+
return fromException(e, null);
209+
}
210+
211+
/**
212+
* @param e the exception
213+
* @param unexpectedExceptionHandler a callback for handling unexpected exceptions. If null, or if this
214+
* returns false, the error is logged at ERROR level via sl4fj
215+
*/
216+
public static ErrorMessage fromException(Throwable e, Predicate<Throwable> unexpectedExceptionHandler)
210217
{
211218
int streamId = 0;
212219
if (e instanceof WrappedException)
@@ -219,7 +226,9 @@ public static ErrorMessage fromException(Throwable e)
219226
return new ErrorMessage((TransportException)e, streamId);
220227

221228
// Unexpected exception
222-
logger.error("Unexpected exception during request", e);
229+
if (unexpectedExceptionHandler == null || !unexpectedExceptionHandler.apply(e))
230+
logger.error("Unexpected exception during request", e);
231+
223232
return new ErrorMessage(new ServerError(e), streamId);
224233
}
225234

0 commit comments

Comments
 (0)