[ https://issues.apache.org/jira/browse/CASSANDRA-8190?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14188364#comment-14188364 ]
Nikolai Grigoriev edited comment on CASSANDRA-8190 at 10/29/14 2:10 PM: ------------------------------------------------------------------------ [~krummas] Marcus, believe me I do not really enjoy hitting this "weird stuff lately" ;) Most of the background is in CASSANDRA-7949 (the one you have marked resolved, although I am not sure I fully agree with that resolution). The only detail I would add to CASSANDRA-7949 is that finally, after ~3 weeks the cluster has managed to finish all compactions. 3 weeks to compact the data created in ~4 days. In between I have lost the patience, stopped it and ran sstablesplit on all large sstables (anything larger than 1Gb) on each node. And then I started the nodes one by one once they were done with the split. Upon restart each node had between ~2K and 7K compactions to complete. I had to let them finish them. On the way I have seen these errors on different nodes at different time - so I reported them. My goal was to get the system to the state with no pending compactions and all sstables having the size close to the target one. This is why I used the flag from CASSANDRA-6621 (cassandra.disable_stcs_in_l0), otherwise the cluster would stay in unusable state forever. Yesterday night the last node has finished the compactions. I've been scrubbing each node after the compactions were done to make sure the data integrity is not broken. Now I am about to restart the load that updates and fetches the data. We are doing some kind of modelling for our real data, a capacity exercise to determine the size of the production cluster. Note that the configuration I am attaching was modified a bit to attempt to speed up compactions. There was not too much to tune but still....Like 0 compaction throughput limit etc. was (Author: ngrigor...@gmail.com): [~krummas] Marcus, believe me I do not really enjoy hitting this "weird stuff lately" ;) Most of the background is in CASSANDRA-7949 (the one you have marked resolved, although I am not sure I fully agree with that resolution). The only detail I would add to CASSANDRA-7949 is that finally, after ~3 weeks the cluster has managed to finish all compactions. 3 weeks to compact the data created in ~4 days. In between I have lost the patience, stopped it and ran sstablesplit on all large sstables (anything larger than 1Gb) on each node. And then I started the nodes one by one once they were done with the split. Upon restart each node had between ~2K and 7K compactions to complete. I had to let them finish them. On the way I have seen these errors on different nodes at different time - so I reported them. My goal was to get the system to the state with no pending compactions and all sstables having the size close to the target one. This is why I used the flag from CASSANDRA-6621 (cassandra.disable_stcs_in_l0), otherwise the cluster would stay in unusable state forever. Yesterday night the last node has finished the compactions. I've been scrubbing each node after the compactions were done to make sure the data integrity is not broken. Now I am about to restart the load that updates and fetches the data. We are doing some kind of modelling for our real data, a capacity exercise to determine the size of the production cluster. > Compactions stop completely because of RuntimeException in CompactionExecutor > ----------------------------------------------------------------------------- > > Key: CASSANDRA-8190 > URL: https://issues.apache.org/jira/browse/CASSANDRA-8190 > Project: Cassandra > Issue Type: Bug > Components: Core > Environment: DSE 4.5.2 (Cassandra 2.0.10) > Reporter: Nikolai Grigoriev > Assignee: Marcus Eriksson > Attachments: cassandra-env.sh, cassandra.yaml, jstack.txt.gz, > system.log.gz > > > I have a cluster that is recovering from being overloaded with writes. I am > using the workaround from CASSANDRA-6621 to prevent the STCS fallback (which > is killing the cluster - see CASSANDRA-7949). > I have observed that after one or more exceptions like this > {code} > ERROR [CompactionExecutor:4087] 2014-10-26 22:50:05,016 CassandraDaemon.java > (line 199) Exception in thread Thread[CompactionExecutor:4087,1,main] > java.lang.RuntimeException: Last written key DecoratedKey(425124616570337476, > 0010000000001111000000000000033523da00001000000000033523da000000001111000000001000000000 > 00004000000000000000000100) >= current key DecoratedKey(-8778432288598355336, > 0010000000001111000000000000040c7a8f00001000000000040c7a8f000000001111000000001000000000 > 00004000000000000000000100) writing into > /cassandra-data/disk2/myks/mytable/myks-mytable-tmp-jb-130379-Data.db > at > org.apache.cassandra.io.sstable.SSTableWriter.beforeAppend(SSTableWriter.java:142) > at > org.apache.cassandra.io.sstable.SSTableWriter.append(SSTableWriter.java:165) > at > org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:160) > 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:60) > at > org.apache.cassandra.db.compaction.AbstractCompactionTask.execute(AbstractCompactionTask.java:59) > at > org.apache.cassandra.db.compaction.CompactionManager$BackgroundCompactionTask.run(CompactionManager.java:198) > at > java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471) > at java.util.concurrent.FutureTask.run(FutureTask.java:262) > 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:745) > {code} > the node completely stops the compactions and I end up in the state like this: > {code} > # nodetool compactionstats > pending tasks: 1288 > compaction type keyspace table completed > total unit progress > Active compaction remaining time : n/a > {code} > The node recovers if restarted and starts compactions - until getting more > exceptions like this. -- This message was sent by Atlassian JIRA (v6.3.4#6332)