[ 
https://issues.apache.org/jira/browse/CASSANDRA-2675?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sylvain Lebresne updated CASSANDRA-2675:
----------------------------------------

    Attachment: 0002-Avoid-modifying-super-column-in-memtable-being-flush.patch
                0001-Don-t-remove-columns-from-super-columns-in-memtable.patch

I was able to reproduce, thanks for the java version.

I think the problem is that reads can remove subcolumns from a super-column 
that happens to be in a memtable being flushed. If a subcolumn become gc-able 
after when the super column count size was written on disk and the time the 
subcolumn itself is written we won't write it and will end up with short super 
columns (hence the EOFException). Note that this should not happen with a 
reasonable gc_grace value (one such that nothing that gets flushed will be 
gcable).

First attached patch fixes this by making reads copy the super-column before 
modifying it (0.7 patch).

I think there is a related second bug, in that when we reduce super columns (in 
QueryFilter), if we merge multiple super column with the same name, we'll 
"merge" them in the first super column. That is, we may end up adding 
subcolumns to a super column that is in an in-memory memtable. Most of the time 
this will be harmless, except some useless data duplication. But if that 
happens for a super column (in a memtable) being flushed and, as above, between 
the write of the number of column and the actual column writes, we may end up 
with too long super column. With could result in unreachable columns (i.e, data 
loss effectively) and quite probably some weird corruption during a compaction.

Second patch fixes this second problem.

I haven't been able to reproduce with the 2 attached patches and the thing is 
running since more than an hour.


> java.io.IOError: java.io.EOFException with version 0.7.6 
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-2675
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2675
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 0.7.6
>         Environment: Reproduced on single Cassandra node (CentOS 5.5)
> Reproduced on single Cassandra node (Windows Server 2008)
>            Reporter: rene kochen
>            Assignee: Sylvain Lebresne
>             Fix For: 0.7.7
>
>         Attachments: 
> 0001-Don-t-remove-columns-from-super-columns-in-memtable.patch, 
> 0002-Avoid-modifying-super-column-in-memtable-being-flush.patch, 
> CassandraIssue.zip, CassandraIssueJava.zip
>
>
> I use the following data-model
> column_metadata: []
> name: Customers
> column_type: Super
> gc_grace_seconds: 60
> I have a super-column-family with a single row.
> Within this row I have a single super-column.
> Within this super-column, I concurrently create, read and delete columns.
> I have three threads:
> - Do in a loop: add a column to the super-column.
> - Do in a loop: delete a random column from the super-column.
> - Do in a loop: read the super-column (with all columns).
> After running the above threads concurrently, I always receive one of the 
> following errors:
> ERROR 17:09:57,036 Fatal exception in thread Thread[ReadStage:81,5,main]
> java.io.IOError: java.io.EOFException
>         at 
> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap.java:252)
>         at 
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:268)
>         at 
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:227)
>         at java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(Unknown 
> Source)
>         at java.util.concurrent.ConcurrentSkipListMap.<init>(Unknown Source)
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:379)
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:362)
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:322)
>         at 
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:79)
>         at 
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:40)
>         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.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108)
>         at 
> org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:283)
>         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:69)
>         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.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:116)
>         at 
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:130)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1390)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1267)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1195)
>         at org.apache.cassandra.db.Table.getRow(Table.java:324)
>         at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:63)
>         at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:451)
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>         at java.lang.Thread.run(Unknown Source)
> Caused by: java.io.EOFException
>         at java.io.RandomAccessFile.readByte(Unknown Source)
>         at 
> org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.java:324)
>         at 
> org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil.java:335)
>         at 
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:71)
>         at 
> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap.java:248)
>         ... 30 more
> java.io.IOError: 
> org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: invalid 
> column name length 0
>         at 
> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap.java:252)
>         at 
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:268)
>         at 
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:227)
>         at java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(Unknown 
> Source)
>         at java.util.concurrent.ConcurrentSkipListMap.<init>(Unknown Source)
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:379)
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:362)
>         at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:322)
>         at 
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:79)
>         at 
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:40)
>         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.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108)
>         at 
> org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:283)
>         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:69)
>         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.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:116)
>         at 
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:130)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1385)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1262)
>         at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1190)
>         at org.apache.cassandra.db.Table.getRow(Table.java:324)
>         at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:63)
>         at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:451)
>         at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>         at java.lang.Thread.run(Unknown Source)
> Caused by: org.apache.cassandra.db.ColumnSerializer$CorruptColumnException: 
> invalid column name length 0
>         at 
> org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:73)
>         at 
> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap.java:248)
>         ... 30 more 
> ERROR 11:02:19,824 Fatal exception in thread Thread[ReadStage:3404,5,main]
> java.io.IOError: java.io.IOException: mmap segment underflow; remaining is 
> 660267 but 758592100 requested
>       at 
> org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap.java:252)
>       at 
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:268)
>       at 
> org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:227)
>       at java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(Unknown 
> Source)
>       at java.util.concurrent.ConcurrentSkipListMap.<init>(Unknown Source)
>       at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:379)
>       at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:362)
>       at 
> org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:322)
>       at 
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:79)
>       at 
> org.apache.cassandra.db.columniterator.SimpleSliceReader.computeNext(SimpleSliceReader.java:40)
>       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.columniterator.SSTableSliceIterator.hasNext(SSTableSliceIterator.java:108)
>       at 
> org.apache.commons.collections.iterators.CollatingIterator.set(CollatingIterator.java:283)
>       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:69)
>       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.filter.SliceQueryFilter.collectReducedColumns(SliceQueryFilter.java:116)
>       at 
> org.apache.cassandra.db.filter.QueryFilter.collectCollatedColumns(QueryFilter.java:130)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilyStore.java:1390)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1267)
>       at 
> org.apache.cassandra.db.ColumnFamilyStore.getColumnFamily(ColumnFamilyStore.java:1195)
>       at org.apache.cassandra.db.Table.getRow(Table.java:324)
>       at 
> org.apache.cassandra.db.SliceFromReadCommand.getRow(SliceFromReadCommand.java:63)
>       at 
> org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(StorageProxy.java:451)
>       at 
> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>       at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(Unknown 
> Source)
>       at java.util.concurrent.ThreadPoolExecutor$Worker.run(Unknown Source)
>       at java.lang.Thread.run(Unknown Source)

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to