Return-Path: Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: (qmail 23989 invoked from network); 31 Aug 2010 22:56:03 -0000 Received: from unknown (HELO mail.apache.org) (140.211.11.3) by 140.211.11.9 with SMTP; 31 Aug 2010 22:56:03 -0000 Received: (qmail 68151 invoked by uid 500); 31 Aug 2010 22:56:01 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 68100 invoked by uid 500); 31 Aug 2010 22:56:01 -0000 Mailing-List: contact user-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: user@cassandra.apache.org Delivered-To: mailing list user@cassandra.apache.org Received: (qmail 68092 invoked by uid 99); 31 Aug 2010 22:56:00 -0000 Received: from athena.apache.org (HELO athena.apache.org) (140.211.11.136) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 31 Aug 2010 22:56:00 +0000 X-ASF-Spam-Status: No, hits=2.9 required=10.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_NONE,SPF_NEUTRAL X-Spam-Check-By: apache.org Received-SPF: neutral (athena.apache.org: local policy) Received: from [64.12.206.41] (HELO imr-ma03.mx.aol.com) (64.12.206.41) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 31 Aug 2010 22:55:55 +0000 Received: from AOLDTCMEH01.ad.office.aol.com (aoldtcmeh01.office.aol.com [10.180.121.20]) by imr-ma03.mx.aol.com (8.14.1/8.14.1) with ESMTP id o7VMtKDW005043 for ; Tue, 31 Aug 2010 18:55:20 -0400 Received: from AOLMTCMEI03.ad.office.aol.com ([10.178.3.20]) by AOLDTCMEH01.ad.office.aol.com with Microsoft SMTPSVC(6.0.3790.4675); Tue, 31 Aug 2010 18:54:53 -0400 Received: from brackenm.local ([10.178.3.10]) by AOLMTCMEI03.ad.office.aol.com over TLS secured channel with Microsoft SMTPSVC(6.0.3790.4675); Tue, 31 Aug 2010 18:54:53 -0400 Message-ID: <4C7D883C.10102@corp.aol.com> Date: Tue, 31 Aug 2010 18:54:52 -0400 From: Carl Bruecken User-Agent: Mozilla/5.0 (Macintosh; U; Intel Mac OS X 10.5; en-US; rv:1.9.2.8) Gecko/20100802 Thunderbird/3.1.2 MIME-Version: 1.0 To: user@cassandra.apache.org Subject: Re: TTransportException intermittently in 0.7 References: <4C7D41A1.2060700@corp.aol.com> In-Reply-To: Content-Type: multipart/alternative; boundary="------------000704080802020305040403" X-OriginalArrivalTime: 31 Aug 2010 22:54:53.0803 (UTC) FILETIME=[8688E3B0:01CB495F] This is a multi-part message in MIME format. --------------000704080802020305040403 Content-Type: text/plain; charset=ISO-8859-1; format=flowed Content-Transfer-Encoding: 7bit I've made some progress on narrowing this down and am able to reproduce easily. I am using pelops as a client and I configured the policy in pelops to only establish 1 connection to a cassandra node. I'm able to step through the pelops code line by line and see the resulting thrift transport logging in cassandra. Seems that flushing the transport causes the unwanted TTransportConnection in the server and subsequent closing of the connection. The connection should stay open after flushing. When there are many connection established the behaviour seems intermittent and many operations succeed. Here are the details 1) The trigger from the client side is when the framed transport is flushed. conn.getAPI().batch_mutate(convertedBatch, cLevel); // Flush connection conn.flush(); 2) In CustomTThreadPoolServer.java in Cassandra I modified the code to log TTransportExceptions. catch (TTransportException ttx) { LOGGER.error("Transport exception", ttx); } catch (TException tx) { LOGGER.error("Thrift error occurred during processing of message.", tx); } catch (Exception x) { LOGGER.error("Error occurred during processing of message.", x); } 3) Here is the exception that is ignored in cassandra. Flushing the transport causes the server to believe the client has closed the connection. org.apache.thrift.transport.TTransportException: Cannot read. Remote side has closed. Tried to read 4 bytes, but only got 0 bytes. at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86) at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:369) at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:295) at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:202) at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2487) at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:167) at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908) at java.lang.Thread.run(Thread.java:637) 4) The next batch mutate to this connection caused the exception in the client WARN [main] 2010-08-31 18:40:06,749 Operand.java (line 72) Operation failed as result of network exception. Connection must be destroyed. See cause for details... org.apache.thrift.transport.TTransportException: java.net.SocketException: Connection reset at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:129) at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84) at org.apache.thrift.transport.TFramedTransport.readFrame(TFramedTransport.java:129) at org.apache.thrift.transport.TFramedTransport.read(TFramedTransport.java:101) at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84) at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:369) at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:295) at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:202) at org.apache.cassandra.thrift.Cassandra$Client.recv_batch_mutate(Cassandra.java:905) at org.apache.cassandra.thrift.Cassandra$Client.batch_mutate(Cassandra.java:889) at org.scale7.cassandra.pelops.Mutator$1.execute(Mutator.java:42) at org.scale7.cassandra.pelops.Mutator$1.execute(Mutator.java:38) at org.scale7.cassandra.pelops.Operand.tryOperation(Operand.java:53) at org.scale7.cassandra.pelops.Mutator.execute(Mutator.java:49) at com.aol.data.c7.App.doWork(App.java:41) at com.aol.data.c7.App.main(App.java:77) Caused by: java.net.SocketException: Connection reset at java.net.SocketInputStream.read(SocketInputStream.java:168) at java.io.BufferedInputStream.fill(BufferedInputStream.java:218) at java.io.BufferedInputStream.read1(BufferedInputStream.java:258) at java.io.BufferedInputStream.read(BufferedInputStream.java:317) at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127) ... 15 more On 8/31/10 4:04 PM, Jonathan Ellis wrote: > > No, I don't know that anyone has reproduced that. TTransportException > always means "something went wrong on the thrift side" in my > experience, it shouldn't be cassandra-version specific. > > On Tue, Aug 31, 2010 at 12:53 PM, Carl Bruecken > wrote: > > > > Are there any estimates as to when a fix for this will be checked into > > trunk? > > > > Coincidentally, has anyone tracked down the issue? > > > > I'm experiencing same issue with nightly build from a week ago. > > > > Thank You > > > > > > -- > Jonathan Ellis > Project Chair, Apache Cassandra > co-founder of Riptano, the source for professional Cassandra support > http://riptano.com > --------------000704080802020305040403 Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: 7bit I've made some progress on narrowing this down and am able to reproduce easily.   I am using pelops as a client and I configured the policy in pelops to only establish 1 connection to a cassandra node.  I'm able to step through the pelops code line by line and see the resulting thrift transport logging in cassandra.   Seems that flushing the transport causes the unwanted TTransportConnection in the server and subsequent closing of the connection.   The connection should stay open after flushing.   When there are many connection established the behaviour seems intermittent and many operations succeed.



Here are the details

1) The trigger from the client side is when the framed transport is flushed.
               conn.getAPI().batch_mutate(convertedBatch, cLevel);
                // Flush connection
                conn.flush();

2) In CustomTThreadPoolServer.java in Cassandra I modified the code to log TTransportExceptions.

        catch (TTransportException ttx) {
        LOGGER.error("Transport exception", ttx);
    } catch (TException tx) {
        LOGGER.error("Thrift error occurred during processing of message.", tx);
    } catch (Exception x) {
        LOGGER.error("Error occurred during processing of message.", x);
    }


3) Here is the exception that is ignored in cassandra.   Flushing the transport causes the server to believe the client has closed the connection.

org.apache.thrift.transport.TTransportException: Cannot read. Remote side has closed. Tried to read 4 bytes, but only got 0 bytes.
    at org.apache.thrift.transport.TTransport.readAll(TTransport.java:86)
    at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:369)
    at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:295)
    at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:202)
    at org.apache.cassandra.thrift.Cassandra$Processor.process(Cassandra.java:2487)
    at org.apache.cassandra.thrift.CustomTThreadPoolServer$WorkerProcess.run(CustomTThreadPoolServer.java:167)
    at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
    at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)
    at java.lang.Thread.run(Thread.java:637)

4) The next batch mutate to this connection caused the exception in the client

 WARN [main] 2010-08-31 18:40:06,749 Operand.java (line 72) Operation failed as result of network exception. Connection must be destroyed.  See cause for details...
org.apache.thrift.transport.TTransportException: java.net.SocketException: Connection reset
    at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:129)
    at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84)
    at org.apache.thrift.transport.TFramedTransport.readFrame(TFramedTransport.java:129)
    at org.apache.thrift.transport.TFramedTransport.read(TFramedTransport.java:101)
    at org.apache.thrift.transport.TTransport.readAll(TTransport.java:84)
    at org.apache.thrift.protocol.TBinaryProtocol.readAll(TBinaryProtocol.java:369)
    at org.apache.thrift.protocol.TBinaryProtocol.readI32(TBinaryProtocol.java:295)
    at org.apache.thrift.protocol.TBinaryProtocol.readMessageBegin(TBinaryProtocol.java:202)
    at org.apache.cassandra.thrift.Cassandra$Client.recv_batch_mutate(Cassandra.java:905)
    at org.apache.cassandra.thrift.Cassandra$Client.batch_mutate(Cassandra.java:889)
    at org.scale7.cassandra.pelops.Mutator$1.execute(Mutator.java:42)
    at org.scale7.cassandra.pelops.Mutator$1.execute(Mutator.java:38)
    at org.scale7.cassandra.pelops.Operand.tryOperation(Operand.java:53)
    at org.scale7.cassandra.pelops.Mutator.execute(Mutator.java:49)
    at com.aol.data.c7.App.doWork(App.java:41)
    at com.aol.data.c7.App.main(App.java:77)
Caused by: java.net.SocketException: Connection reset
    at java.net.SocketInputStream.read(SocketInputStream.java:168)
    at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
    at java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
    at java.io.BufferedInputStream.read(BufferedInputStream.java:317)
    at org.apache.thrift.transport.TIOStreamTransport.read(TIOStreamTransport.java:127)
    ... 15 more




On 8/31/10 4:04 PM, Jonathan Ellis wrote:
Re: TTransportException intermittently in 0.7

No, I don't know that anyone has reproduced that.  TTransportException
always means "something went wrong on the thrift side" in my
experience, it shouldn't be cassandra-version specific.

On Tue, Aug 31, 2010 at 12:53 PM, Carl Bruecken
<carl.bruecken@corp.aol.com> wrote:
>
>  Are there any estimates as to when a fix for this will be checked into
> trunk?
>
> Coincidentally, has anyone tracked down the issue?
>
>  I'm experiencing same issue with nightly build from a week ago.
>
> Thank You
>



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


--------------000704080802020305040403--