Return-Path: Delivered-To: apmail-cassandra-user-archive@www.apache.org Received: (qmail 7063 invoked from network); 15 Feb 2011 18:16:01 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 15 Feb 2011 18:16:01 -0000 Received: (qmail 85657 invoked by uid 500); 15 Feb 2011 18:15:59 -0000 Delivered-To: apmail-cassandra-user-archive@cassandra.apache.org Received: (qmail 85462 invoked by uid 500); 15 Feb 2011 18:15:56 -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 85452 invoked by uid 99); 15 Feb 2011 18:15:55 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 15 Feb 2011 18:15:55 +0000 X-ASF-Spam-Status: No, hits=1.5 required=5.0 tests=HTML_MESSAGE,RCVD_IN_DNSWL_LOW,SPF_PASS X-Spam-Check-By: apache.org Received-SPF: pass (nike.apache.org: domain of sylvain@datastax.com designates 209.85.160.172 as permitted sender) Received: from [209.85.160.172] (HELO mail-gy0-f172.google.com) (209.85.160.172) by apache.org (qpsmtpd/0.29) with ESMTP; Tue, 15 Feb 2011 18:15:48 +0000 Received: by gyd12 with SMTP id 12so241280gyd.31 for ; Tue, 15 Feb 2011 10:15:27 -0800 (PST) MIME-Version: 1.0 Received: by 10.236.110.2 with SMTP id t2mr4120347yhg.34.1297793726085; Tue, 15 Feb 2011 10:15:26 -0800 (PST) Received: by 10.147.41.10 with HTTP; Tue, 15 Feb 2011 10:15:26 -0800 (PST) X-Originating-IP: [88.183.33.171] In-Reply-To: References: <4D5AA409.8070207@trioptima.com> Date: Tue, 15 Feb 2011 19:15:26 +0100 Message-ID: Subject: Re: Possible EOFException regression in 0.7.1 From: Sylvain Lebresne To: user@cassandra.apache.org Content-Type: multipart/alternative; boundary=0023547c9a1d08fa7a049c562693 X-Virus-Checked: Checked by ClamAV on apache.org --0023547c9a1d08fa7a049c562693 Content-Type: text/plain; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable On Tue, Feb 15, 2011 at 7:10 PM, ruslan usifov wro= te: > It will be great if patch appear very quick > patch attached here: https://issues.apache.org/jira/browse/CASSANDRA-2165 Hoping this is quick enough. > > 2011/2/15 Jonathan Ellis > > I can reproduce with your script. Thanks! >> >> 2011/2/15 Jonas Borgstr=F6m : >> > Hi all, >> > >> > While testing the new 0.7.1 release I got the following exception: >> > >> > ERROR [ReadStage:11] 2011-02-15 16:39:18,105 >> > DebuggableThreadPoolExecutor.java (line 103) Error in ThreadPoolExecut= or >> > java.io.IOError: java.io.EOFException >> > at >> > >> org.apache.cassandra.db.columniterator.SSTableNamesIterator.(SSTab= leNamesIterator.java:75) >> > at >> > >> org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator= (NamesQueryFilter.java:59) >> > at >> > >> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(Quer= yFilter.java:80) >> > at >> > >> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamil= yStore.java:1274) >> > at >> > >> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilySt= ore.java:1166) >> > at >> > >> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilySt= ore.java:1095) >> > at org.apache.cassandra.db.Table.getRow(Table.java:384) >> > at >> > >> org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadC= ommand.java:60) >> > at >> > >> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(= StorageProxy.java:473) >> > at >> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) >> > at >> > >> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.jav= a:1110) >> > at >> > >> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.ja= va:603) >> > at java.lang.Thread.run(Thread.java:636) >> > Caused by: java.io.EOFException >> > at java.io.DataInputStream.readInt(DataInputStream.java:392) >> > at >> > >> org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilter= Serializer.java:48) >> > at >> > >> org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilter= Serializer.java:30) >> > at >> > >> org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexHel= per.java:108) >> > at >> > >> org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTable= NamesIterator.java:106) >> > at >> > >> org.apache.cassandra.db.columniterator.SSTableNamesIterator.(SSTab= leNamesIterator.java:71) >> > ... 12 more >> > >> > I'm able reliably reproduce this using the following one node cluster: >> > - apache-cassandra-0.7.1-bin.tar.gz >> > - Fedora 14 >> > - java version "1.6.0_20". >> > OpenJDK 64-Bit Server VM (build 19.0-b09, mixed mode) >> > - Default cassandra.yaml >> > - cassandra-env.sh: MAX_HEAP_SIZE=3D"1G"; HEAP_NEWSIZE=3D"200M" >> > >> > cassandra-cli initialization: >> > - create keyspace foo; >> > - use foo; >> > - create column family datasets; >> > >> > $ python dataset_check.py (attached) >> > Inserting row 0 of 10 >> > Inserting row 1 of 10 >> > Inserting row 2 of 10 >> > Inserting row 3 of 10 >> > Inserting row 4 of 10 >> > Inserting row 5 of 10 >> > Inserting row 6 of 10 >> > Inserting row 7 of 10 >> > Inserting row 8 of 10 >> > Inserting row 9 of 10 >> > Attempting to fetch key 0 >> > Traceback (most recent call last): >> > ... >> > pycassa.pool.MaximumRetryException: Retried 6 times >> > >> > After this I have 6 EOFExceptions in system.log. >> > Running "get datasets[0]['name'];" using cassandra-cli also triggers t= he >> > same exception. >> > I've not been able to reproduce this with cassandra 0.7.0. >> > >> > Regards, >> > Jonas >> > >> > >> > >> >> >> >> -- >> Jonathan Ellis >> Project Chair, Apache Cassandra >> co-founder of DataStax, the source for professional Cassandra support >> http://www.datastax.com >> > > --0023547c9a1d08fa7a049c562693 Content-Type: text/html; charset=ISO-8859-1 Content-Transfer-Encoding: quoted-printable
On Tue, Feb 15, 2011 at 7:10 PM, ruslan usifov <= span dir=3D"ltr"><ruslan.usif= ov@gmail.com> wrote:
It will be great if patch appear very quick

=
patch attached here:=A0https://issues.apache.org/jira/browse/CASSANDRA-2165<= /div>

Hoping this is quick enough.
=A0

2011/2/15 Jonathan = Ellis <jbellis@gmail.com>

I can reproduce with your script. =A0Thanks!

2011/2/15 Jonas Borgstr=F6m <jonas.borgstrom@trioptima.com>:
> Hi all,
>
> While testing the new 0.7.1 release I got the following exception:
>
> ERROR [ReadStage:11] 2011-02-15 16:39:18,105
> DebuggableThreadPoolExecutor.java (line 103) Error in ThreadPoolExecut= or
> java.io.IOError: java.io.EOFException
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init&g= t;(SSTableNamesIterator.java:75)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterat= or(NamesQueryFilter.java:59)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(Qu= eryFilter.java:80)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFam= ilyStore.java:1274)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily= Store.java:1166)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamily= Store.java:1095)
> =A0 =A0 =A0 =A0at org.apache.cassandra.db.Table.getRow(Table.java:384)=
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesRea= dCommand.java:60)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThro= w(StorageProxy.java:473)
> =A0 =A0 =A0 =A0at org.apache.cassandra.utils.WrappedRunnable.run(Wrapp= edRunnable.java:30)
> =A0 =A0 =A0 =A0at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.j= ava:1110)
> =A0 =A0 =A0 =A0at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.= java:603)
> =A0 =A0 =A0 =A0at java.lang.Thread.run(Thread.java:636)
> Caused by: java.io.EOFException
> =A0 =A0 =A0 =A0at java.io.DataInputStream.readInt(DataInputStream.java= :392)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilt= erSerializer.java:48)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilt= erSerializer.java:30)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexH= elper.java:108)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTab= leNamesIterator.java:106)
> =A0 =A0 =A0 =A0at
> org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init&g= t;(SSTableNamesIterator.java:71)
> =A0 =A0 =A0 =A0... 12 more
>
> I'm able reliably reproduce this using the following one node clus= ter:
> - apache-cassandra-0.7.1-bin.tar.gz
> - Fedora 14
> - java version "1.6.0_20".
> =A0OpenJDK 64-Bit Server VM (build 19.0-b09, mixed mode)
> - Default cassandra.yaml
> - cassandra-env.sh: MAX_HEAP_SIZE=3D"1G"; HEAP_NEWSIZE=3D&qu= ot;200M"
>
> cassandra-cli initialization:
> - create keyspace foo;
> - use foo;
> - create column family datasets;
>
> $ python dataset_check.py (attached)
> Inserting row 0 of 10
> Inserting row 1 of 10
> Inserting row 2 of 10
> Inserting row 3 of 10
> Inserting row 4 of 10
> Inserting row 5 of 10
> Inserting row 6 of 10
> Inserting row 7 of 10
> Inserting row 8 of 10
> Inserting row 9 of 10
> Attempting to fetch key 0
> Traceback (most recent call last):
> ...
> pycassa.pool.MaximumRetryException: Retried 6 times
>
> After this I have 6 EOFExceptions in system.log.
> Running "get datasets[0]['name'];" using cassandra-c= li also triggers the
> same exception.
> I've not been able to reproduce this with cassandra 0.7.0.
>
> Regards,
> Jonas
>
>
>



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


--0023547c9a1d08fa7a049c562693--