It does look a lot like 1932. Make sure everything is really running 0.7.2, 0.7.0 can't read data files created by 0.7.1+.
If the versions are ok, take a snapshot, then compact, and see if the problem still occurs with all files on the [compacted] new format. 2011/2/16 Jonas Borgström <jonas.borgst...@trioptima.com>: > On 02/15/2011 07:15 PM, Sylvain Lebresne wrote: >> On Tue, Feb 15, 2011 at 7:10 PM, ruslan usifov <ruslan.usi...@gmail.com >> <mailto:ruslan.usi...@gmail.com>> wrote: >> >> It will be great if patch appear very quick >> >> >> patch attached here: https://issues.apache.org/jira/browse/CASSANDRA-2165 > > Hi again and thanks for quickly fixing this. I can no longer reproduce > this using my script on a clean 0.7.2 install. > > However, after upgrading the cluster where I originally found this issue > I get the following exception when access some of the rows inserted > during my testing of 0.7.0 and 0.7.1. > > [default@foo] get datasets[5]['name'] as utf8; > => (column=6e616d65, value=key:5, timestamp=1297342560881994) > [default@foo] get datasets[4]; > => Lots of data > [default@foo] get datasets[4]['name'] as utf8; > null > > And in system.log: > ------------------ > ERROR [ReadStage:59] 2011-02-16 14:55:01,737 > AbstractCassandraDaemon.java (line 114) Fatal exception in thread > Thread[ReadStage:59,5,main] > java.lang.RuntimeException: java.lang.NegativeArraySizeException > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:34) > 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:662) > Caused by: java.lang.NegativeArraySizeException > at > org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:49) > at > org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:30) > at > org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexHelper.java:108) > at > org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:106) > at > org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNamesIterator.java:71) > at > org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:59) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:80) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1275) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1167) > at > org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1095) > at org.apache.cassandra.db.Table.getRow(Table.java:384) > at > org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.java:60) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:473) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) > ... 3 more > > get_range also fails with a similar exception: > ---------------------------------------------- > ERROR [ReadStage:43] 2011-02-16 14:59:08,662 > AbstractCassandraDaemon.java (line 114) Fatal exception in thread > Thread[ReadStage:43,5,main] > java.lang.RuntimeException: java.lang.NegativeArraySizeException > at > org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:60) > at > org.apache.cassandra.net.MessageDeliveryTask.run(MessageDeliveryTask.java:72) > 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:619) > Caused by: java.lang.NegativeArraySizeException > at > org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:49) > at > org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:30) > at > org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexHelper.java:108) > at > org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:106) > at > org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNamesIterator.java:91) > at > org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:64) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:87) > at > org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:184) > at > org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:144) > at > org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:136) > at > org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:39) > at > org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:284) > at > org.apache.commons.collections.iterators.CollatingIterator.least(CollatingIterator.java:326) > at > org.apache.commons.collections.iterators.CollatingIterator.next(CollatingIterator.java:230) > at > org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.java:68) > at > com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:136) > at > com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:131) > at org.apache.cassandra.db.RowIterator.hasNext(RowIterator.java:49) > at > org.apache.cassandra.db.ColumnFamilyStore.getRangeSlice(ColumnFamilyStore.java:1354) > at > org.apache.cassandra.service.RangeSliceVerbHandler.doVerb(RangeSliceVerbHandler.java:49) > ... 4 more > > The column family was created with 0.7.0 and data has been inserted with > 0.7.0, 0.7.1 and 0.7.2. There seems to be sstable files of version e and > f. There's probably around 1000-2000 keys and each key has up to 1 > million columns. > > Does this mean that the EOFException bug could have caused corrupted > bloom filters after all or is this perhaps a separate issue, perhaps: > https://issues.apache.org/jira/browse/CASSANDRA-1932 ? > > Regards, > Jonas > >> Hoping this is quick enough. >> >> >> >> 2011/2/15 Jonathan Ellis <jbel...@gmail.com <mailto:jbel...@gmail.com>> >> >> I can reproduce with your script. Thanks! >> >> 2011/2/15 Jonas Borgström <jonas.borgst...@trioptima.com >> <mailto:jonas.borgst...@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 >> ThreadPoolExecutor >> > java.io.IOError: java.io.EOFException >> > at >> > >> >> org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNamesIterator.java:75) >> > at >> > >> >> org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(NamesQueryFilter.java:59) >> > at >> > >> >> org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFilter.java:80) >> > at >> > >> >> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1274) >> > at >> > >> >> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1166) >> > at >> > >> >> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1095) >> > at org.apache.cassandra.db.Table.getRow(Table.java:384) >> > at >> > >> >> org.apache.cassandra.db.SliceByNamesReadCommand.getRow(SliceByNamesReadCommand.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.java:1110) >> > at >> > >> >> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java: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(BloomFilterSerializer.java:48) >> > at >> > >> >> org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSerializer.java:30) >> > at >> > >> >> org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexHelper.java:108) >> > at >> > >> >> org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableNamesIterator.java:106) >> > at >> > >> >> org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNamesIterator.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="1G"; HEAP_NEWSIZE="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 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.com >> >> >> > > -- Jonathan Ellis Project Chair, Apache Cassandra co-founder of DataStax, the source for professional Cassandra support http://www.datastax.com