1717 */
1818package org .apache .cassandra .transport ;
1919
20+ import java .io .IOException ;
2021import java .util .EnumSet ;
22+ import java .util .Set ;
2123import java .util .UUID ;
2224
25+ import com .google .common .base .Predicate ;
26+ import com .google .common .collect .ImmutableSet ;
2327import org .jboss .netty .buffer .ChannelBuffer ;
2428import org .jboss .netty .buffer .ChannelBuffers ;
2529import 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}
0 commit comments