cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tylerho...@apache.org
Subject [1/3] git commit: Log unclean client disconnect excs at DEBUG
Date Thu, 18 Sep 2014 17:37:16 GMT
Repository: cassandra
Updated Branches:
  refs/heads/trunk b18478d05 -> a810c477d


Log unclean client disconnect excs at DEBUG

Patch by Graham Sanderson; reviewed by Tyler Hobbs for CASSANDRA-7849


Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo
Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/cbc705d4
Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/cbc705d4
Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/cbc705d4

Branch: refs/heads/trunk
Commit: cbc705d419d0db54d5a94e181e0f50916eb57dbc
Parents: 549f035
Author: Graham Sanderson <graham@vast.com>
Authored: Thu Sep 18 12:24:29 2014 -0500
Committer: Tyler Hobbs <tyler@datastax.com>
Committed: Thu Sep 18 12:26:08 2014 -0500

----------------------------------------------------------------------
 CHANGES.txt                                     |  2 +
 .../org/apache/cassandra/transport/Message.java | 73 +++++++++++++++++++-
 .../transport/messages/ErrorMessage.java        | 19 +++--
 3 files changed, 87 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/cassandra/blob/cbc705d4/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 01d32e7..c9d507a 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -1,4 +1,6 @@
 2.0.11:
+ * Log exceptions related to unclean native protocol client disconnects
+   at DEBUG or INFO (CASSANDRA-7849)
  * Allow permissions cache to be set via JMX (CASSANDRA-7698)
  * Include schema_triggers CF in readable system resources (CASSANDRA-7967)
  * Fix RowIndexEntry to report correct serializedSize (CASSANDRA-7948)

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cbc705d4/src/java/org/apache/cassandra/transport/Message.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/Message.java b/src/java/org/apache/cassandra/transport/Message.java
index bbe7e38..eb38558 100644
--- a/src/java/org/apache/cassandra/transport/Message.java
+++ b/src/java/org/apache/cassandra/transport/Message.java
@@ -17,9 +17,13 @@
  */
 package org.apache.cassandra.transport;
 
+import java.io.IOException;
 import java.util.EnumSet;
+import java.util.Set;
 import java.util.UUID;
 
+import com.google.common.base.Predicate;
+import com.google.common.collect.ImmutableSet;
 import org.jboss.netty.buffer.ChannelBuffer;
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.*;
@@ -38,6 +42,17 @@ public abstract class Message
 {
     protected static final Logger logger = LoggerFactory.getLogger(Message.class);
 
+    /**
+     * When we encounter an unexpected IOException we look for these {@link Throwable#getMessage()
messages}
+     * (because we have no better way to distinguish) and log them at DEBUG rather than INFO,
since they
+     * are generally caused by unclean client disconnects rather than an actual problem.
+     */
+    private static final Set<String> ioExceptionsAtDebugLevel = ImmutableSet.<String>builder().
+            add("Connection reset by peer").
+            add("Broken pipe").
+            add("Connection timed out").
+            build();
+
     public interface Codec<M extends Message> extends CBCodec<M> {}
 
     public enum Direction
@@ -315,7 +330,7 @@ public abstract class Message
             catch (Throwable ex)
             {
                 // Don't let the exception propagate to exceptionCaught() if we can help
it so that we can assign the right streamID.
-                ctx.getChannel().write(ErrorMessage.fromException(ex).setStreamId(request.getStreamId()));
+                ctx.getChannel().write(ErrorMessage.fromException(ex, new UnexpectedChannelExceptionHandler(ctx.getChannel(),
true)).setStreamId(request.getStreamId()));
             }
         }
 
@@ -325,7 +340,7 @@ public abstract class Message
         {
             if (ctx.getChannel().isOpen())
             {
-                ChannelFuture future = ctx.getChannel().write(ErrorMessage.fromException(e.getCause()));
+                ChannelFuture future = ctx.getChannel().write(ErrorMessage.fromException(e.getCause(),
new UnexpectedChannelExceptionHandler(ctx.getChannel(), false)));
                 // On protocol exception, close the channel as soon as the message have been
sent
                 if (e.getCause() instanceof ProtocolException)
                 {
@@ -338,4 +353,58 @@ public abstract class Message
             }
         }
     }
+
+    /**
+     * Include the channel info in the logged information for unexpected errors, and (if
{@link #alwaysLogAtError} is
+     * false then choose the log level based on the type of exception (some are clearly client
issues and shouldn't be
+     * logged at server ERROR level)
+     */
+    static final class UnexpectedChannelExceptionHandler implements Predicate<Throwable>
+    {
+        private final Channel channel;
+        private final boolean alwaysLogAtError;
+
+        UnexpectedChannelExceptionHandler(Channel channel, boolean alwaysLogAtError)
+        {
+            this.channel = channel;
+            this.alwaysLogAtError = alwaysLogAtError;
+        }
+
+        @Override
+        public boolean apply(Throwable exception)
+        {
+            String message;
+            try
+            {
+                message = "Unexpected exception during request; channel = " + channel;
+            }
+            catch (Exception ignore)
+            {
+                // We don't want to make things worse if String.valueOf() throws an exception
+                message = "Unexpected exception during request; channel = <unprintable>";
+            }
+
+            if (!alwaysLogAtError && exception instanceof IOException)
+            {
+                if (ioExceptionsAtDebugLevel.contains(exception.getMessage()))
+                {
+                    // Likely unclean client disconnects
+                    logger.debug(message, exception);
+                }
+                else
+                {
+                    // Generally unhandled IO exceptions are network issues, not actual ERRORS
+                    logger.info(message, exception);
+                }
+            }
+            else
+            {
+                // Anything else is probably a bug in server of client binary protocol handling
+                logger.error(message, exception);
+            }
+
+            // We handled the exception.
+            return true;
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/cassandra/blob/cbc705d4/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
----------------------------------------------------------------------
diff --git a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
index 3ca5801..4d60a1f 100644
--- a/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
+++ b/src/java/org/apache/cassandra/transport/messages/ErrorMessage.java
@@ -17,10 +17,8 @@
  */
 package org.apache.cassandra.transport.messages;
 
-import java.nio.ByteBuffer;
-
+import com.google.common.base.Predicate;
 import org.jboss.netty.buffer.ChannelBuffer;
-import org.jboss.netty.buffer.ChannelBuffers;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -31,7 +29,6 @@ import org.apache.cassandra.transport.CBUtil;
 import org.apache.cassandra.transport.Message;
 import org.apache.cassandra.transport.ProtocolException;
 import org.apache.cassandra.transport.ServerError;
-import org.apache.cassandra.utils.ByteBufferUtil;
 import org.apache.cassandra.utils.MD5Digest;
 
 /**
@@ -208,6 +205,16 @@ public class ErrorMessage extends Message.Response
 
     public static ErrorMessage fromException(Throwable e)
     {
+        return fromException(e, null);
+    }
+
+    /**
+     * @param e the exception
+     * @param unexpectedExceptionHandler a callback for handling unexpected exceptions. If
null, or if this
+     *                                   returns false, the error is logged at ERROR level
via sl4fj
+     */
+    public static ErrorMessage fromException(Throwable e, Predicate<Throwable> unexpectedExceptionHandler)
+    {
         int streamId = 0;
         if (e instanceof WrappedException)
         {
@@ -219,7 +226,9 @@ public class ErrorMessage extends Message.Response
             return new ErrorMessage((TransportException)e, streamId);
 
         // Unexpected exception
-        logger.error("Unexpected exception during request", e);
+        if (unexpectedExceptionHandler == null || !unexpectedExceptionHandler.apply(e))
+            logger.error("Unexpected exception during request", e);
+
         return new ErrorMessage(new ServerError(e), streamId);
     }
 


Mime
View raw message