You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@cassandra.apache.org by Jim Ancona <ji...@anconafamily.com> on 2011/09/06 21:58:49 UTC

Re: Cassandra client loses connectivity to cluster

Since we finally fixed this issue, I thought I'd document the
solution, with the hope that it makes it easier for others who might
run into it.

During the time this issue was occurring Anthony Ikeda reported a very
similar issue, although without the strange pattern of occurrences we
saw: http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Trying-to-find-the-problem-with-a-broken-pipe-td6645526.html

It turns out that our problem was the same as Anthony's: exceeding
Thrift's maximum frame size, as set by
thrift_framed_transport_size_in_mb in cassandra.yaml. This problem was
extremely hard to troubleshoot, for the following reasons:

* TFramedTransport responds to an oversized frame by throwing a
TTransportException, which is a generic exception thrown for various
types of network or protocol errors. Because such errors are common,
many servers (TSimpleServer, TThreadPoolServer, and Cassandra's
CustomTThreadPoolServer) swallow TTransportException without a log
message. I've filed https://issues.apache.org/jira/browse/THRIFT-1323
and https://issues.apache.org/jira/browse/CASSANDRA-3142 to address
the lack of logging. (We finally found the issue by adding logging
code to our production Cassandra deploy. The fact that we could do
that it a big win for open source.)
* After the TTransportException occurs, the server closes the
underlying socket. To the client (Hector in our case), this appears as
broken socket, most likely caused by a network problem or failed
server node. Hector responds by marking the server node down and
retrying the too-large request on another node, where it also fails.
This process repeated leads to the entire cluster being marked down
(see https://github.com/rantav/hector/issues/212).
* Ideally, sending an oversized frame should trigger a recognizable
error on the client, so that the client knows that it has made a error
and avoids compounding the mistake by retrying. Thrift's framed
transport is pretty simple and I assume there isn't a good way for the
server to communicate the error to the client. As a second-best
solution, I've logged a bug against Thrift
(https://issues.apache.org/jira/browse/THRIFT-1324) saying that
TFramedTransport should enforce the configured frame size limit on
writes. At least that way people can avoid the issue by configuring a
client frame size to match their servers'. If that is implemented then
clients like Hector will be able to detect the frame too large case
and return an error instead of retrying it.

In addition to the issues above, some other things made this issue
difficult to solve:
* The pattern of occurrences (only occurring at a certain time of day,
on a single server at a time, only on weekdays, etc.) was something of
a distraction.
* Even after finding out that Anthony's similar problem was caused by
an oversized frame, I was convinced that we could not be generating an
operation large enough to exceed the configured value (15 mb).

It turns out that I was almost right: out of our hundreds of thousands
of customers, exactly one was working with data that large, and that
customer was doing so not because of anomalous behavior on their part,
but because of a bug in our system. So the fact that it was a single
customer explained the regular occurrences, and the bug explained the
unexpectedly large data size. Of course in this case "almost right"
wasn't good enough, my BOTE calculation failed to take the bug into
account. Plus, as I tweeted immediately after I figured out what was
going on, "Lesson: when you have millions of users it becomes easier
to say things about averages, but harder to do the same for extremes."

Jim

On Wed, Jun 29, 2011 at 5:42 PM, Jim Ancona <ji...@anconafamily.com> wrote:
> In reviewing client logs as part of our Cassandra testing, I noticed
> several Hector "All host pools marked down" exceptions in the logs.
> Further investigation showed a consistent pattern of
> "java.net.SocketException: Broken pipe" and "java.net.SocketException:
> Connection reset" messages. These errors occur for all 36 hosts in the
> cluster over a period of seconds, as Hector tries to find a working
> host to connect to. Failing to find a host results in the "All host
> pools marked down" messages. These messages recur for a period ranging
> from several seconds up to almost 15 minutes, clustering around two to
> three minutes. Then connectivity returns and when Hector tries to
> reconnect it succeeds.
>
> The clients are instances of a JBoss 5 web application. We use Hector
> 0.7.0-29 (plus a patch that was pulled in advance of -30) The
> Cassandra cluster has 72 nodes split between two datacenters. It's
> running 0.7.5 plus a couple of bug fixes pulled in advance of 0.7.6.
> The keyspace uses NetworkTopologyStrategy and RF=6 (3 in each
> datacenter). The clients are reading and writing at LOCAL_QUORUM to
> the 36 nodes in their own data center. Right now the second datacenter
> is for failover only, so there are no clients actually writing there.
>
> There's nothing else obvious in the JBoss logs at around the same
> time, e.g. other application errors, GC events. The Cassandra
> system.log files at INFO level shows nothing out of the ordinary. I
> have a capture of one of the incidents at DEBUG level where again I
> see nothing abnormal looking, but there's so much data that it would
> be easy to miss something.
>
> Other observations:
> * It only happens on weekdays (Our weekends are much lower load)
> * It has occurred every weekday for the last month except for Monday
> May 30, the Memorial Day holiday in the US.
> * Most days it occurs only once, but six times it has occurred twice,
> never more often than that.
> * It generally happens in the late afternoon, but there have been
> occurrences earlier in the afternoon and twice in the late morning.
> Earliest occurrence is 11:19 am, latest is 18:11 pm. Our peak loads
> are between 10:00 and 14:00, so most occurrences do *not* correspond
> with peak load times.
> * It only happens on a single client JBoss instance at a time.
> * Generally, it affects a different host each day, but the same host
> was affected on consecutive days once.
> * Out of 40 clients, one has been affected three times, seven have
> been affected twice, 11 have been affected once and 21 have not been
> affected.
> * The cluster is lightly loaded.
>
> Given that the problem affects a single client machine at a time and
> that machine loses the ability to connect to the entire cluster, It
> seems unlikely that the problem is on the C* server side. Even a
> network problem seems hard to explain, given that the clients are on
> the same subnet, I would expect all of them to fail if it were a
> network issue.
>
> I'm hoping that perhaps someone has seen a similar issue or can
> suggest things to try.
>
> Thanks in advance for any help!
>
> Jim
>

Re: Cassandra client loses connectivity to cluster

Posted by Jonathan Ellis <jb...@gmail.com>.
Thanks for the followup, Jim!

We'll review https://issues.apache.org/jira/browse/CASSANDRA-3142 shortly.

On Tue, Sep 6, 2011 at 2:58 PM, Jim Ancona <ji...@anconafamily.com> wrote:
> Since we finally fixed this issue, I thought I'd document the
> solution, with the hope that it makes it easier for others who might
> run into it.
>
> During the time this issue was occurring Anthony Ikeda reported a very
> similar issue, although without the strange pattern of occurrences we
> saw: http://cassandra-user-incubator-apache-org.3065146.n2.nabble.com/Trying-to-find-the-problem-with-a-broken-pipe-td6645526.html
>
> It turns out that our problem was the same as Anthony's: exceeding
> Thrift's maximum frame size, as set by
> thrift_framed_transport_size_in_mb in cassandra.yaml. This problem was
> extremely hard to troubleshoot, for the following reasons:
>
> * TFramedTransport responds to an oversized frame by throwing a
> TTransportException, which is a generic exception thrown for various
> types of network or protocol errors. Because such errors are common,
> many servers (TSimpleServer, TThreadPoolServer, and Cassandra's
> CustomTThreadPoolServer) swallow TTransportException without a log
> message. I've filed https://issues.apache.org/jira/browse/THRIFT-1323
> and https://issues.apache.org/jira/browse/CASSANDRA-3142 to address
> the lack of logging. (We finally found the issue by adding logging
> code to our production Cassandra deploy. The fact that we could do
> that it a big win for open source.)
> * After the TTransportException occurs, the server closes the
> underlying socket. To the client (Hector in our case), this appears as
> broken socket, most likely caused by a network problem or failed
> server node. Hector responds by marking the server node down and
> retrying the too-large request on another node, where it also fails.
> This process repeated leads to the entire cluster being marked down
> (see https://github.com/rantav/hector/issues/212).
> * Ideally, sending an oversized frame should trigger a recognizable
> error on the client, so that the client knows that it has made a error
> and avoids compounding the mistake by retrying. Thrift's framed
> transport is pretty simple and I assume there isn't a good way for the
> server to communicate the error to the client. As a second-best
> solution, I've logged a bug against Thrift
> (https://issues.apache.org/jira/browse/THRIFT-1324) saying that
> TFramedTransport should enforce the configured frame size limit on
> writes. At least that way people can avoid the issue by configuring a
> client frame size to match their servers'. If that is implemented then
> clients like Hector will be able to detect the frame too large case
> and return an error instead of retrying it.
>
> In addition to the issues above, some other things made this issue
> difficult to solve:
> * The pattern of occurrences (only occurring at a certain time of day,
> on a single server at a time, only on weekdays, etc.) was something of
> a distraction.
> * Even after finding out that Anthony's similar problem was caused by
> an oversized frame, I was convinced that we could not be generating an
> operation large enough to exceed the configured value (15 mb).
>
> It turns out that I was almost right: out of our hundreds of thousands
> of customers, exactly one was working with data that large, and that
> customer was doing so not because of anomalous behavior on their part,
> but because of a bug in our system. So the fact that it was a single
> customer explained the regular occurrences, and the bug explained the
> unexpectedly large data size. Of course in this case "almost right"
> wasn't good enough, my BOTE calculation failed to take the bug into
> account. Plus, as I tweeted immediately after I figured out what was
> going on, "Lesson: when you have millions of users it becomes easier
> to say things about averages, but harder to do the same for extremes."
>
> Jim
>
> On Wed, Jun 29, 2011 at 5:42 PM, Jim Ancona <ji...@anconafamily.com> wrote:
>> In reviewing client logs as part of our Cassandra testing, I noticed
>> several Hector "All host pools marked down" exceptions in the logs.
>> Further investigation showed a consistent pattern of
>> "java.net.SocketException: Broken pipe" and "java.net.SocketException:
>> Connection reset" messages. These errors occur for all 36 hosts in the
>> cluster over a period of seconds, as Hector tries to find a working
>> host to connect to. Failing to find a host results in the "All host
>> pools marked down" messages. These messages recur for a period ranging
>> from several seconds up to almost 15 minutes, clustering around two to
>> three minutes. Then connectivity returns and when Hector tries to
>> reconnect it succeeds.
>>
>> The clients are instances of a JBoss 5 web application. We use Hector
>> 0.7.0-29 (plus a patch that was pulled in advance of -30) The
>> Cassandra cluster has 72 nodes split between two datacenters. It's
>> running 0.7.5 plus a couple of bug fixes pulled in advance of 0.7.6.
>> The keyspace uses NetworkTopologyStrategy and RF=6 (3 in each
>> datacenter). The clients are reading and writing at LOCAL_QUORUM to
>> the 36 nodes in their own data center. Right now the second datacenter
>> is for failover only, so there are no clients actually writing there.
>>
>> There's nothing else obvious in the JBoss logs at around the same
>> time, e.g. other application errors, GC events. The Cassandra
>> system.log files at INFO level shows nothing out of the ordinary. I
>> have a capture of one of the incidents at DEBUG level where again I
>> see nothing abnormal looking, but there's so much data that it would
>> be easy to miss something.
>>
>> Other observations:
>> * It only happens on weekdays (Our weekends are much lower load)
>> * It has occurred every weekday for the last month except for Monday
>> May 30, the Memorial Day holiday in the US.
>> * Most days it occurs only once, but six times it has occurred twice,
>> never more often than that.
>> * It generally happens in the late afternoon, but there have been
>> occurrences earlier in the afternoon and twice in the late morning.
>> Earliest occurrence is 11:19 am, latest is 18:11 pm. Our peak loads
>> are between 10:00 and 14:00, so most occurrences do *not* correspond
>> with peak load times.
>> * It only happens on a single client JBoss instance at a time.
>> * Generally, it affects a different host each day, but the same host
>> was affected on consecutive days once.
>> * Out of 40 clients, one has been affected three times, seven have
>> been affected twice, 11 have been affected once and 21 have not been
>> affected.
>> * The cluster is lightly loaded.
>>
>> Given that the problem affects a single client machine at a time and
>> that machine loses the ability to connect to the entire cluster, It
>> seems unlikely that the problem is on the C* server side. Even a
>> network problem seems hard to explain, given that the clients are on
>> the same subnet, I would expect all of them to fail if it were a
>> network issue.
>>
>> I'm hoping that perhaps someone has seen a similar issue or can
>> suggest things to try.
>>
>> Thanks in advance for any help!
>>
>> Jim
>>
>



-- 
Jonathan Ellis
Project Chair, Apache Cassandra
co-founder of DataStax, the source for professional Cassandra support
http://www.datastax.com