[ https://issues.apache.org/jira/browse/CASSANDRA-2084?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12989054#comment-12989054 ]
Stu Hood edited comment on CASSANDRA-2084 at 2/1/11 6:00 AM: ------------------------------------------------------------- EDIT: Just double checked, apparently version 'f' was in the 0.7 branch, but did not make it into 0.7.0: apologies. I'll take a close look at this tomorrow. -It looks like those SSTables were created with a pre-release version of Cassandra 0.7 (version 'e', vs the release version 'f'). Mind you, that is a usecase that we would like to support, but it's important information to include in a bug report.- -This error occurs suspiciously close to the bloom filter reading code, which changed between e and f. I'll CC kingryan to have him take a look tomorrow.- Keeping a list of uncompactable SSTables is an excellent idea: opened CASSANDRA-2087. Also opened CASSANDRA-2088 for the compaction cleanup problem. Thanks for the report! was (Author: stuhood): It looks like those SSTables were created with a pre-release version of Cassandra 0.7 (version 'e', vs the release version 'f'). Mind you, that is a usecase that we would like to support, but it's important information to include in a bug report. This error occurs suspiciously close to the bloom filter reading code, which changed between e and f. I'll CC kingryan to have him take a look tomorrow. Keeping a list of uncompactable SSTables is an excellent idea: opened CASSANDRA-2087. Also opened CASSANDRA-2088 for the compaction cleanup problem. Thanks for the report! > Corrupt sstables cause compaction to fail again, and again and again, ... > ------------------------------------------------------------------------- > > Key: CASSANDRA-2084 > URL: https://issues.apache.org/jira/browse/CASSANDRA-2084 > Project: Cassandra > Issue Type: Bug > Components: Core > Affects Versions: 0.7.0 > Environment: Ubuntu 10.10 > Cassandra 0.7.0 (4 Nodes) > Java: > - java version "1.6.0_22" > - Java(TM) SE Runtime Environment (build 1.6.0_22-b04) > - Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode) > Reporter: Dan Hendry > > I have been having some serious data corruption issues in my cluster. I > suspect some deeper more serious Cassandra bug but I dont know what or where > it is and I have not found a way to reproduce the issues I have been having. > This ticket is for a behaviour I have observed where cassandra starts > compacting a set of sstables, fails, does not clean up the tmp files, then > start compacting the exact same set of sstables again. (See logs below). > After awhile, the node runs out of disk space and crashes. At the very least, > cassandra should clean up temp files after a failed compaction. Better yet, > it should stop trying to compact that file and log what file the error > occurred for. The list of corrupt sstables does not even have to be > persistent, just an in memory list which gets wiped out on a restart. > Here is a sample log, the same 4 sstables are being compacted then failing > then being compacted again. > INFO [CompactionExecutor:1] 2011-01-31 13:08:26,434 CompactionManager.java > (line 272) Compacting > [org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-562-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-692-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-773-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-940-Data.db')] > INFO [HintedHandoff:1] 2011-01-31 13:08:28,878 HintedHandOffManager.java > (line 226) Could not complete hinted handoff to /192.168.4.16 > INFO [HintedHandoff:1] 2011-01-31 13:08:28,879 ColumnFamilyStore.java (line > 648) switching in a fresh Memtable for HintsColumnFamily at > CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1296500864696.log', > position=104140211) > INFO [HintedHandoff:1] 2011-01-31 13:08:28,879 ColumnFamilyStore.java (line > 952) Enqueuing flush of Memtable-HintsColumnFamily@1652350488(1155546 bytes, > 20839 operations) > INFO [FlushWriter:1] 2011-01-31 13:08:28,879 Memtable.java (line 155) > Writing Memtable-HintsColumnFamily@1652350488(1155546 bytes, 20839 operations) > INFO [FlushWriter:1] 2011-01-31 13:08:29,199 Memtable.java (line 162) > Completed flushing > /var/lib/cassandra/data/system/HintsColumnFamily-e-9-Data.db (1075487 bytes) > INFO [GossipStage:1] 2011-01-31 13:08:45,508 Gossiper.java (line 569) > InetAddress /192.168.4.16 is now UP > INFO [COMMIT-LOG-WRITER] 2011-01-31 13:08:59,736 CommitLogSegment.java (line > 50) Creating new commitlog segment > /var/lib/cassandra/commitlog/CommitLog-1296500939735.log > INFO [MutationStage:8] 2011-01-31 13:09:15,868 ColumnFamilyStore.java (line > 648) switching in a fresh Memtable for UserSearch at > CommitLogContext(file='/var/lib/cassandra/commitlog/CommitLog-1296500939735.log', > position=56028937) > INFO [MutationStage:8] 2011-01-31 13:09:15,868 ColumnFamilyStore.java (line > 952) Enqueuing flush of Memtable-UserSearch@1186863256(174163962 bytes, > 2097155 operations) > INFO [FlushWriter:1] 2011-01-31 13:09:15,868 Memtable.java (line 155) > Writing Memtable-UserSearch@1186863256(174163962 bytes, 2097155 operations) > ERROR [CompactionExecutor:1] 2011-01-31 13:09:22,462 > AbstractCassandraDaemon.java (line 91) Fatal exception in thread > Thread[CompactionExecutor:1,1,main] > java.io.IOError: java.io.EOFException: attempted to skip 776104308 bytes but > only skipped 8469212 > at > org.apache.cassandra.io.sstable.SSTableIdentityIterator.<init>(SSTableIdentityIterator.java:78) > at > org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:178) > at > org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:143) > at > org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:135) > at > org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:38) > 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.commons.collections.iterators.FilterIterator.setNextObject(FilterIterator.java:183) > at > org.apache.commons.collections.iterators.FilterIterator.hasNext(FilterIterator.java:94) > at > org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:323) > at > org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:122) > at > org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:92) > 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) > at java.lang.Thread.run(Thread.java:662) > Caused by: java.io.EOFException: attempted to skip 776104308 bytes but only > skipped 8469212 > at > org.apache.cassandra.io.sstable.IndexHelper.skipBloomFilter(IndexHelper.java:52) > at > org.apache.cassandra.io.sstable.SSTableIdentityIterator.<init>(SSTableIdentityIterator.java:69) > ... 20 more > INFO [CompactionExecutor:1] 2011-01-31 13:09:22,463 CompactionManager.java > (line 272) Compacting > [org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-562-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-692-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-773-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-940-Data.db')] > INFO [FlushWriter:1] 2011-01-31 13:09:29,010 Memtable.java (line 162) > Completed flushing > /var/lib/cassandra/data/kikmetrics/UserSearch-e-1264-Data.db (184687455 bytes) > INFO [COMMIT-LOG-WRITER] 2011-01-31 13:09:38,221 CommitLogSegment.java (line > 50) Creating new commitlog segment > /var/lib/cassandra/commitlog/CommitLog-1296500978221.log > INFO [COMMIT-LOG-WRITER] 2011-01-31 13:10:15,781 CommitLogSegment.java (line > 50) Creating new commitlog segment > /var/lib/cassandra/commitlog/CommitLog-1296501015781.log > ERROR [CompactionExecutor:1] 2011-01-31 13:10:29,139 > AbstractCassandraDaemon.java (line 91) Fatal exception in thread > Thread[CompactionExecutor:1,1,main] > java.io.IOError: java.io.EOFException: attempted to skip 776104308 bytes but > only skipped 8469212 > at > org.apache.cassandra.io.sstable.SSTableIdentityIterator.<init>(SSTableIdentityIterator.java:78) > at > org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:178) > at > org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTableScanner.java:143) > at > org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:135) > at > org.apache.cassandra.io.sstable.SSTableScanner.next(SSTableScanner.java:38) > 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.commons.collections.iterators.FilterIterator.setNextObject(FilterIterator.java:183) > at > org.apache.commons.collections.iterators.FilterIterator.hasNext(FilterIterator.java:94) > at > org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.java:323) > at > org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:122) > at > org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:92) > 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) > at java.lang.Thread.run(Thread.java:662) > Caused by: java.io.EOFException: attempted to skip 776104308 bytes but only > skipped 8469212 > at > org.apache.cassandra.io.sstable.IndexHelper.skipBloomFilter(IndexHelper.java:52) > at > org.apache.cassandra.io.sstable.SSTableIdentityIterator.<init>(SSTableIdentityIterator.java:69) > ... 20 more > INFO [CompactionExecutor:1] 2011-01-31 13:10:29,148 CompactionManager.java > (line 272) Compacting > [org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-562-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-692-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-773-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/DeviceEventsByDevice-e-940-Data.db')] -- This message is automatically generated by JIRA. - For more information on JIRA, see: http://www.atlassian.com/software/jira