Obvisously, that should not happen. That being said, we did fix a bug yesterday that could produce that kind of trace: https://issues.apache.org/jira/browse/CASSANDRA-5799. It will be part of 1.2.7 that should be released tomorrow, though the artifacts that are being voted on are at https://repository.apache.org/content/repositories/orgapachecassandra-013/org/apache/cassandra/apache-cassandra/1.2.7/if you want to test sooner than later.
If you still can reproduce with that version, feel free to open a ticket on JIRA (in which case, the more information you can give us to reproduce, the better the change we'd be able to fix it quickly). -- Sylvain On Wed, Jul 24, 2013 at 10:54 PM, Paul Ingalls <paulinga...@gmail.com>wrote: > It is pretty much every row that hits the large threshold. I don't think > I can delete every row that hits that… > > you can see the db size in the stack trace, do you want a different type > of size? > > On Jul 24, 2013, at 11:07 AM, Jason Wee <peich...@gmail.com> wrote: > > Would it possible to delete this row and reinsert this row? By the way, > how large is that one row? > > Jason > > > On Wed, Jul 24, 2013 at 9:23 AM, Paul Ingalls <paulinga...@gmail.com>wrote: > >> I'm getting constant exceptions during compaction of large rows. In >> fact, I have not seen one work, even starting from an empty DB. As soon as >> I start pushing in data, when a row hits the large threshold, it fails >> compaction with this type of stack trace: >> >> INFO [CompactionExecutor:6] 2013-07-24 01:17:53,592 >> CompactionController.java (line 156) Compacting large row >> fanzo/tweets_by_id:352567939972603904 (153360688 bytes) incrementally >> ERROR [CompactionExecutor:6] 2013-07-24 01:18:12,496 CassandraDaemon.java >> (line 192) Exception in thread Thread[CompactionExecutor:6,1,main] >> java.lang.AssertionError: incorrect row data size 5722610 written to >> /mnt/datadrive/lib/cassandra/data/fanzo/tweets_by_id/fanzo-tweets_by_id-tmp-ic-1453-Data.db; >> correct is 5767384 >> at >> org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:162) >> at >> org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:162) >> at >> org.apache.cassandra.io.util.DiskAwareRunnable.runMayThrow(DiskAwareRunnable.java:48) >> at >> org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:28) >> at >> org.apache.cassandra.db.compaction.CompactionTask.executeInternal(CompactionTask.java:58) >> at >> org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:60) >> at >> org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:211) >> at >> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) >> at >> java.util.concurrent.FutureTask$Sync.innerRun(FutureTask.java:334) >> at java.util.concurrent.FutureTask.run(FutureTask.java:166) >> at >> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) >> at >> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) >> at java.lang.Thread.run(Thread.java:724) >> >> I'm not sure what to do or where to look. Help…:) >> >> Thanks, >> >> Paul >> >> >> > >