cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Terry Cumaranatunge (Created) (JIRA)" <j...@apache.org>
Subject [jira] [Created] (CASSANDRA-3468) SStable data corruption in 1.0.x
Date Mon, 07 Nov 2011 21:24:51 GMT
SStable data corruption in 1.0.x
--------------------------------

                 Key: CASSANDRA-3468
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3468
             Project: Cassandra
          Issue Type: Bug
          Components: Core
    Affects Versions: 1.0.0
         Environment: RHEL 6 running Cassandra 1.0.x.
            Reporter: Terry Cumaranatunge


We have noticed several instances of sstable corruptions in 1.0.x. This has occurred in 1.0.0-rcx
and 1.0.0 and 1.0.1. It has happened on multiple nodes and multiple hosts with different disks,
so this is the reason the software is suspected at this time. The file system used is XFS,
but no resets or any type of failure scenarios have been run to create the problem. We were
basically running under load and every so often, we see that the sstable gets corrupted and
compaction stops on that node.

I will attach the relevant sstable files if it lets me do that when I create this ticket.

ERROR [CompactionExecutor:23] 2011-10-27 11:14:09,309 PrecompactedRow.java (line 119) Skipping
row DecoratedKey(128013852116656632841539411062933532114, 37303730303138313533) in /var/lib/cassandra/data/MSA/participants-h-8688-Data.db
java.io.EOFException
        at java.io.RandomAccessFile.readFully(RandomAccessFile.java:399)
        at java.io.RandomAccessFile.readFully(RandomAccessFile.java:377)
        at org.apache.cassandra.utils.BytesReadTracker.readFully(BytesReadTracker.java:95)
        at org.apache.cassandra.utils.ByteBufferUtil.read(ByteBufferUtil.java:388)
        at org.apache.cassandra.utils.ByteBufferUtil.readWithLength(ByteBufferUtil.java:350)
        at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:96)
        at org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:36)
        at org.apache.cassandra.db.ColumnFamilySerializer.deserializeColumns(ColumnFamilySerializer.java:143)
        at org.apache.cassandra.io.sstable.SSTableIdentityIterator.getColumnFamilyWithColumns(SSTableIdentityIterator.java:231)
        at org.apache.cassandra.db.compaction.PrecompactedRow.merge(PrecompactedRow.java:115)
        at org.apache.cassandra.db.compaction.PrecompactedRow.<init>(PrecompactedRow.java:102)
        at org.apache.cassandra.db.compaction.CompactionController.getCompactedRow(CompactionController.java:127)
        at org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:102)
        at org.apache.cassandra.db.compaction.CompactionIterable$Reducer.getReduced(CompactionIterable.java:87)
        at org.apache.cassandra.utils.MergeIterator$ManyToOne.consume(MergeIterator.java:116)
        at org.apache.cassandra.utils.MergeIterator$ManyToOne.computeNext(MergeIterator.java:99)
        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
        at com.google.common.collect.Iterators$7.computeNext(Iterators.java:614)
        at com.google.common.collect.AbstractIterator.tryToComputeNext(AbstractIterator.java:140)
        at com.google.common.collect.AbstractIterator.hasNext(AbstractIterator.java:135)
        at org.apache.cassandra.db.compaction.CompactionTask.execute(CompactionTask.java:179)
        at org.apache.cassandra.db.compaction.LeveledCompactionTask.execute(LeveledCompactionTask.java:47)
        at org.apache.cassandra.db.compaction.CompactionManager$1.call(CompactionManager.java:131)
        at org.apache.cassandra.db.compaction.CompactionManager$1.call(CompactionManager.java:114)
        at java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:303)
        at java.util.concurrent.FutureTask.run(FutureTask.java:138)
        at java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.java:886)
        at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:908)


This was Sylvain's analysis:

I don't have much better news. Basically it seems the 2 last MB of the file are complete garbage
(which also explain the mmap error btw). And given where the corruption actually starts, it
suggests that it's either a very low level bug in our file writer code that start writting
bad data at some point for some reason, or it's corruption not related to Cassandra. But given
that, a Cassandra bug sounds fairly unlikely.

You said that you saw that corruption more than once. Could you be more precise? In particular,
did you get it on different hosts? Also, what file system are you using?

If you do happen to have another instance of a corrupted sstable (ideally from some other
host) that you can share, please don't hesitate. I could try to look if I find something common
between the two.


--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Mime
View raw message