Dan, Just wondered if you had a chance to try 0.7.1 or 0.7.2 out with your context/data.
Jeremy On Feb 11, 2011, at 12:17 PM, Dan Hendry wrote: > Here is my chronology: > > - I turned on my current cluster in early December, starting with > about 0.7 RC2 or thereabouts. I ran into a number of problems but was able > to get things more or less ironed out (upgrading to new versions pretty > quickly). Once 0.7.0 was released, I had no problems for about 2 weeks. In > that time (using 0.7.0), I ran frequent compactions but never a repair (not > really needed with my workload: read repair works fine and I use expiring > columns) > > - Sometime mid-January (~21) I hit my first instance of sstable > corruption. I was able to fix things by deleting EVERYTHING related to the > problematic CF on the problematic node (sstables, hints column family, saved > caches, the works). I then ran a repair which failed > (http://www.mail-archive.com/user@cassandra.apache.org/msg09154.html) > possibly due to CASSANDRA-1992 or some such. I was able to delete sstables, > run cleanup and compaction, truncate unimportant CFs and get my service > operating again > > - A week or two ago I created a new CF which turned out to be the > most problematic in terms of SSTable corruption. Since creating this CF via > CLI (again, all nodes running 0.7.0), there have been no repairs run and no > ring changes (moving, bootstrapping or decommissioning). Unless I > misunderstand CASSANDRA-1992, I don't think it is related. I have been > seeing steadily increasing rates of Cassandra exceptions on multiple CFs, > reads failing, compaction failing, hinted handoff failing, inability to > export sstables2json, etc. > > > > Now I cant even understand how any of the RC bugs I hit back in the day are > still around given I was running 0.7.0 for weeks and ran multiple full > compactions without issue. Regardless, they could not have affected then new > CF. I have seen no improvement for the one node I put on the 0.7 branch (r > 1069600). I am seeing the same behaviour for multiple failed compactions (as > per the bug I filed: https://issues.apache.org/jira/browse/CASSANDRA-2084) > and am still investigating the read exceptions. Jonathan suggested a > completely clean 0.7.1 install which would be a lot of work given I would > have to migrate data at an application level but I would be willing to do so > if I had any confidence it would help. > > > > Here is an example of one SSTable on the node running 0.7. Notice that a > bunch of seemingly fine sstables (no problems reading from it that I could > determine but my tests were not rigorous and this happened overnight), which > had to either be created or compacted at least once (given -f-) on 0.7 > (with, for example, the CASSANDRA-1992 and CASSANDRA-1999 fixes) gets > compacted to a new SSTable that is completely f'ed. > > > > INFO [CompactionExecutor:1] 2011-02-10 20:25:42,200 CompactionManager.java > (line 373) Compacting > [org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data > /kikmetrics/ProcessingBuckets-f-4311-Data.db'),org.apache.cassandra.io.sstab > le.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/ProcessingBuckets- > f-4312-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/li > b/cassandra/data/kikmetrics/ProcessingBuckets-f-4313-Data.db'),org.apache.ca > ssandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/Pr > ocessingBuckets-f-4314-Data.db')] > > ... > > INFO [CompactionExecutor:1] 2011-02-10 20:25:49,908 CompactionIterator.java > (line 135) Compacting large row 555345525f4556454e547c5345434f4e445f3130 > (393599737 bytes) incrementally > > ... > > INFO [CompactionExecutor:1] 2011-02-10 20:26:42,989 CompactionManager.java > (line 458) Compacted to > /var/lib/cassandra/data/kikmetrics/ProcessingBuckets-tmp-f-4315-Data.db. > 487,085,029 to 477,567,892 (~98% of original) bytes for 12 keys. Time: > 60,788ms. > > ... > > ... this sstable does not appear in the logs unitll: > > ... > > INFO [CompactionExecutor:1] 2011-02-10 22:51:30,613 CompactionManager.java > (line 373) Compacting > [org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data > /kikmetrics/ProcessingBuckets-e-4295-Data.db'),org.apache.cassandra.io.sstab > le.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/ProcessingBuckets- > f-4315-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/li > b/cassandra/data/kikmetrics/ProcessingBuckets-f-4320-Data.db'),org.apache.ca > ssandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/Pr > ocessingBuckets-f-4327-Data.db')] > > ... > > ... Note the 'File :' output is a minor change I made (obviously) > > ... > > ERROR [CompactionExecutor:1] 2011-02-10 22:51:49,029 > AbstractCassandraDaemon.java (line 114) Fatal exception in thread > Thread[CompactionExecutor:1,1,main] > > java.io.IOError: java.io.IOException: File > :/var/lib/cassandra/data/kikmetrics/ProcessingBuckets-f-4315-Data.db > > at > org.apache.cassandra.io.sstable.SSTableIdentityIterator.<init>(SSTableIdenti > tyIterator.java:85) > > at > org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTa > bleScanner.java:179) > > at > org.apache.cassandra.io.sstable.SSTableScanner$KeyScanningIterator.next(SSTa > bleScanner.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(CollatingIter > ator.java:284) > > at > org.apache.commons.collections.iterators.CollatingIterator.least(CollatingIt > erator.java:326) > > at > org.apache.commons.collections.iterators.CollatingIterator.next(CollatingIte > rator.java:230) > > at > org.apache.cassandra.utils.ReducingIterator.computeNext(ReducingIterator.jav > a: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(Filter > Iterator.java:183) > > at > org.apache.commons.collections.iterators.FilterIterator.hasNext(FilterIterat > or.java:94) > > at > org.apache.cassandra.db.CompactionManager.doCompaction(CompactionManager.jav > a:427) > > at > org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:123) > > at > org.apache.cassandra.db.CompactionManager$1.call(CompactionManager.java:93) > > 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.ja > va:886) > > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > > at java.lang.Thread.run(Thread.java:662) > > Caused by: java.io.IOException: File > :/var/lib/cassandra/data/kikmetrics/ProcessingBuckets-f-4315-Data.db > > ... 21 more > > Caused by: java.io.EOFException: attempted to skip 67113066 bytes but only > skipped 27852647 > > at > org.apache.cassandra.io.sstable.IndexHelper.skipBloomFilter(IndexHelper.java > :51) > > at > org.apache.cassandra.io.sstable.SSTableIdentityIterator.<init>(SSTableIdenti > tyIterator.java:69) > > ... 20 more > > INFO [CompactionExecutor:1] 2011-02-10 22:51:49,030 CompactionManager.java > (line 373) Compacting > [org.apache.cassandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data > /kikmetrics/ProcessingBuckets-e-4295-Data.db'),org.apache.cassandra.io.sstab > le.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/ProcessingBuckets- > f-4315-Data.db'),org.apache.cassandra.io.sstable.SSTableReader(path='/var/li > b/cassandra/data/kikmetrics/ProcessingBuckets-f-4320-Data.db'),org.apache.ca > ssandra.io.sstable.SSTableReader(path='/var/lib/cassandra/data/kikmetrics/Pr > ocessingBuckets-f-4327-Data.db')] > > ... etc ... > > > > > > > > Now when trying to export ProcessingBuckets-f-4315-Data.db using > sstable2json, I get the following: > > > > Exception in thread "main" java.io.IOError: java.io.EOFException > > at > org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap. > java:246) > > at > org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:262) > > at > org.apache.cassandra.io.util.ColumnIterator.next(ColumnSortedMap.java:223) > > at > java.util.concurrent.ConcurrentSkipListMap.buildFromSorted(ConcurrentSkipLis > tMap.java:1493) > > at > java.util.concurrent.ConcurrentSkipListMap.<init>(ConcurrentSkipListMap.java > :1443) > > at > org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:3 > 66) > > at > org.apache.cassandra.db.SuperColumnSerializer.deserialize(SuperColumn.java:3 > 14) > > at > org.apache.cassandra.db.columniterator.IndexedSliceReader$IndexedBlockFetche > r.getNextBlock(IndexedSliceReader.java:181) > > at > org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(Indexe > dSliceReader.java:120) > > at > org.apache.cassandra.db.columniterator.IndexedSliceReader.computeNext(Indexe > dSliceReader.java:48) > > 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(SSTableS > liceIterator.java:108) > > at > org.apache.cassandra.tools.SSTableExport.serializeRow(SSTableExport.java:178 > ) > > at > org.apache.cassandra.tools.SSTableExport.export(SSTableExport.java:353) > > at > org.apache.cassandra.tools.SSTableExport.export(SSTableExport.java:375) > > at > org.apache.cassandra.tools.SSTableExport.export(SSTableExport.java:388) > > at > org.apache.cassandra.tools.SSTableExport.main(SSTableExport.java:446) > > Caused by: java.io.EOFException > > at > java.io.RandomAccessFile.readByte(RandomAccessFile.java:591) > > at > org.apache.cassandra.utils.ByteBufferUtil.readShortLength(ByteBufferUtil.jav > a:287) > > at > org.apache.cassandra.utils.ByteBufferUtil.readWithShortLength(ByteBufferUtil > .java:298) > > at > org.apache.cassandra.db.ColumnSerializer.deserialize(ColumnSerializer.java:6 > 6) > > at > org.apache.cassandra.io.util.ColumnIterator.deserializeNext(ColumnSortedMap. > java:242) > > ... 17 more > > > > :( > > > > Dan > > > > From: Aaron Morton [mailto:aa...@thelastpickle.com] > Sent: February-10-11 20:16 > To: dev@cassandra.apache.org > Subject: Re: RE: SEVERE Data Corruption Problems > > > > Looks like the bloom filter for the row is corrupted, does it happen for all > reads or just for reads on one row ? After the upgrade to 0.7 (assuming an > 0.7 nightly build) did you run anything like nodetool repair ? > > > > Have you tried asking on the #cassandra IRC room to see if their are any > comitters around ? > > Aaron > > > On 11 Feb, 2011,at 01:18 PM, Dan Hendry <dan.hendry.j...@gmail.com> wrote: > > Upgraded one node to 0.7. Its logging exceptions like mad (thousands per > minute). All like below (which is fairly new to me): > > ERROR [ReadStage:721] 2011-02-10 18:13:56,190 AbstractCassandraDaemon.java > (line 114) Fatal exception in thread Threa > d[ReadStage:721,5,main] > java.io.IOError: java.io.EOFException > at > org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNa > mesIterator.java:75) > at > org.apache.cassandra.db.filter.NamesQueryFilter.getSSTableColumnIterator(Nam > esQueryFilter.java:59) > at > org.apache.cassandra.db.filter.QueryFilter.getSSTableColumnIterator(QueryFil > ter.java:80) > at > org.apache.cassandra.db.ColumnFamilyStore.getTopLevelColumns(ColumnFamilySto > re.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(SliceByNamesReadComma > nd.java:60) > at > org.apache.cassandra.service.StorageProxy$LocalReadRunnable.runMayThrow(Stor > ageProxy.java:473) > at > org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30) > at > java.util.concurrent.ThreadPoolExecutor$Worker.runTask(ThreadPoolExecutor.ja > va:886) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:9 > 08) > at java.lang.Thread.run(Thread.java:662) > Caused by: java.io.EOFException > at java.io.DataInputStream.readInt(DataInputStream.java:375) > at > org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSeri > alizer.java:48) > at > org.apache.cassandra.utils.BloomFilterSerializer.deserialize(BloomFilterSeri > alizer.java:30) > at > org.apache.cassandra.io.sstable.IndexHelper.defreezeBloomFilter(IndexHelper. > java:108) > at > org.apache.cassandra.db.columniterator.SSTableNamesIterator.read(SSTableName > sIterator.java:106) > at > org.apache.cassandra.db.columniterator.SSTableNamesIterator.<init>(SSTableNa > mesIterator.java:71) > ... 12 more > > Dan > > > -----Original Message----- > From: Jonathan Ellis [mailto:jbel...@gmail.com] > Sent: February-09-11 18:14 > To: dev > Subject: Re: SEVERE Data Corruption Problems > > Hi Dan, > > it would be very useful to test with 0.7 branch instead of 0.7.0 so at > least you're not chasing known and fixed bugs like CASSANDRA-1992. > > As you say, there's a lot of people who aren't seeing this, so it > would also be useful if you can provide some kind of test harness > where you can say "point this at a cluster and within a few hours > > On Wed, Feb 9, 2011 at 4:31 PM, Dan Hendry <dan.hendry.j...@gmail.com> > wrote: >> I have been having SEVERE data corruption issues with SSTables in my >> cluster, for one CF it was happening almost daily (I have since shut down >> the service using that CF as it was too much work to manage the Cassandra >> errors). At this point, I can't see how it is anything but a Cassandra bug >> yet it's somewhat strange and very scary that I am the only one who seems > to >> be having such serious issues. Most of my data is indexed in two ways so I >> have been able to write a validator which goes through and back fills >> missing data but it's kind of defeating the whole point of Cassandra. The >> only way I have found to deal with issues when they crop up to prevent > nodes >> crashing from repeated failed compactions is delete the SSTable. My > cluster >> is running a slightly modified 0.7.0 version which logs what files errors >> for so that I can stop the node and delete them. >> >> >> >> The problem: >> >> - Reads, compactions and hinted handoff fail with various >> exceptions (samples shown at the end of this email) which seem to indicate >> sstable corruption. >> >> - I have seen failed reads/compactions/hinted handoff on 4 out of > 4 >> nodes (RF=2) for 3 different super column families and 1 standard column >> family (4 out of 11) and just now, the Hints system CF. (if it matters the >> ring has not changed since one CF which has been giving me trouble was >> created). I have check SMART disk info and run various diagnostics and > there >> does not seem to be any hardware issues, plus what are the chances of all >> four nodes having the same hardware problems at the same time when for all >> other purposes, they appear fine? >> >> - I have added logging which outputs what sstable are causing >> exceptions to be thrown. The corrupt sstables have been both freshly > flushed >> memtables and the output of compaction (ie, 4 sstables which all seem to > be >> fine get compacted to 1 which is then corrupt). It seems that the majority >> of corrupt sstables are post-compacted (vs post-memtable flush). >> >> - The one CF which was giving me the most problems was heavily >> written to (1000-1500 writes/second continually across the cluster). For >> that cf, was having to deleting 4-6 sstables a day across the cluster (and >> the number was going up, even the number of problems for remaining CFs is >> going up). The other CFs which have had corrupt sstables are also quite >> heavily written to (generally a few hundred writes a second across the >> cluster). >> >> - Most of the time (5/6 attempts) when this problem occurs, >> sstable2json also fails. I have however, had one case where I was able to >> export the sstable to json, then re-import it at which point I was no > longer >> seeing exceptions. >> >> - The cluster has been running for a little over 2 months now, >> problem seems to have sprung up in the last 3-4 weeks and seems to be >> steadily getting worse. >> >> >> >> Ultimately, I think I am hitting some subtle race condition somewhere. I >> have been starting to dig into the Cassandra code but I barely know where > to >> start looking. I realize I have not provided nearly enough information to >> easily debug the problem but PLEASE keep your eyes open for possibly racy > or >> buggy code which could cause these sorts of problems. I am willing to >> provided full Cassandra logs and a corrupt SSTable on an individual basis: >> please email me and let me know. >> >> >> >> Here is possibly relevant information and my theories on a possible root >> cause. Again, I know little about the Cassandra code base and have only >> moderate java experience so these theories may be way off base. >> >> - Strictly speaking, I probably don't have enough memory for my >> workload. I see stop the world gc occurring ~30/day/node, often causing >> Cassandra to hang for 30+ seconds (according to the gc logs). Could there > be >> some java bug where a full gc in the middle of writing or flushing >> (compaction/memtable flush) or doing some other disk based activity causes >> some sort of data corruption? >> >> - Writes are usually done at ConsistencyLevel ONE with additional >> client side retry logic. Given that I often see consecutive nodes in the >> ring down, could there be some edge condition where dying at just the > right >> time causes parts of mutations/messages to be lost? >> >> - All of the CFs which have been causing me problems have large >> rows which are compacted incrementally. Could there be some problem with > the >> incremental compaction logic? >> >> - My cluster has a fairly heavy write load (again, the most >> problematic CF is getting 1500 (w/s)/(RF=2) = 750 writes/second/node). >> Furthermore, it is highly probable that there are timestamp collisions. >> Could there be some issue with timestamp logic (ie, using > instead of >= > or >> some such) during flushes/compaction? >> >> - Once a node >> >> >> >> Cluster/system information: >> >> - 4 nodes with RF=2 >> >> - Nodes have 8 cores with 24 GB of RAM a piece. >> >> - 2 HDs, 1 for commit log/system, 1 for /var/lib/cassandra/data >> >> - OS is Ubuntu 10.04 (uname -r = 2.6.32-24-server) >> >> - Java: >> >> o java version "1.6.0_22" >> >> o Java(TM) SE Runtime Environment (build 1.6.0_22-b04) >> >> o Java HotSpot(TM) 64-Bit Server VM (build 17.1-b03, mixed mode) >> >> - Slightly modified (file information in exceptions) version of >> 0.7.0 >> >> >> >> The following non-standard cassandra.yaml properties have been changed: >> >> - commitlog_sync_period_in_ms: 100 (with commitlog_sync: > periodic) >> >> - disk_access_mode: mmap_index_only >> >> - concurrent_reads: 12 >> >> - concurrent_writes: 2 (was 32, but I dropped it to 2 to try and >> eliminate any mutation race conditions - did not seem to help) >> >> - sliced_buffer_size_in_kb: 128 >> >> - in_memory_compaction_limit_in_mb: 50 >> >> - rpc_timeout_in_ms: 15000 >> >> >> >> Schema for most problematic CF: >> >> name: DeviceEventsByDevice >> >> column_type: Standard >> >> memtable_throughput_in_mb: 150 >> >> memtable_operations_in_millions: 1.5 >> >> gc_grace_seconds: 172800 >> >> keys_cached: 1000000 >> >> rows_cached: 0 >> >> >> >> Dan Hendry >> >> (403) 660-2297 >> >> > > > > -- > Jonathan Ellis > Project Chair, Apache Cassandra > co-founder of DataStax, the source for professional Cassandra support > http://www.datastax.com > No virus found in this incoming message. > Checked by AVG - www.avg.com <http://www.avgcom> > Version: 9.0872 / Virus Database: 271.1.1/3432 - Release Date: 02/09/11 > 02:34:00 > > No virus found in this incoming message. > Checked by AVG - www.avg.com > Version: 9.0.872 / Virus Database: 271.1.1/3436 - Release Date: 02/11/11 > 02:34:00 >