[ https://issues.apache.org/jira/browse/CASSANDRA-6008?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13850697#comment-13850697 ]
Nikolai Grigoriev commented on CASSANDRA-6008: ---------------------------------------------- Not sure it is related, but I have noticed that I often have this issue when the node shuts down with this exception: {code} INFO [RMI TCP Connection(8)-10.3.45.158] 2013-12-17 17:22:31,782 StorageService.java (line 941) DRAINED ERROR [CompactionExecutor:2008] 2013-12-17 17:22:36,615 CassandraDaemon.java (line 187) Exception in thread Thread[CompactionExecutor:2008,1,main] java.util.concurrent.RejectedExecutionException: Task java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask@16e10a93 rejected from org.apache.cassandra.concurrent.DebuggableScheduledThreadPoolExecutor@107d44a1[Terminated, pool size = 0, active threads = 0, queued tasks = 0, completed tasks = 130876] at java.util.concurrent.ThreadPoolExecutor$AbortPolicy.rejectedExecution(ThreadPoolExecutor.java:2048) at java.util.concurrent.ThreadPoolExecutor.reject(ThreadPoolExecutor.java:821) at java.util.concurrent.ScheduledThreadPoolExecutor.delayedExecute(ScheduledThreadPoolExecutor.java:325) at java.util.concurrent.ScheduledThreadPoolExecutor.schedule(ScheduledThreadPoolExecutor.java:530) at java.util.concurrent.ScheduledThreadPoolExecutor.submit(ScheduledThreadPoolExecutor.java:629) at org.apache.cassandra.io.sstable.SSTableDeletingTask.schedule(SSTableDeletingTask.java:66) at org.apache.cassandra.io.sstable.SSTableReader.releaseReference(SSTableReader.java:1105) at org.apache.cassandra.db.DataTracker.removeOldSSTablesSize(DataTracker.java:388) at org.apache.cassandra.db.DataTracker.postReplace(DataTracker.java:353) at org.apache.cassandra.db.DataTracker.replace(DataTracker.java:347) at org.apache.cassandra.db.DataTracker.replaceCompactedSSTables(DataTracker.java:252) at org.apache.cassandra.db.ColumnFamilyStore.replaceCompactedSSTables(ColumnFamilyStore.java:1078) at org.apache.cassandra.db.compaction.CompactionTask.replaceCompactedSSTables(CompactionTask.java:296) at org.apache.cassandra.db.compaction.CompactionTask.runWith(CompactionTask.java:242) 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:197) 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) {code} I do disable thrift,gossip and drain the node before stopping Cassandra process. > Getting 'This should never happen' error at startup due to sstables missing > --------------------------------------------------------------------------- > > Key: CASSANDRA-6008 > URL: https://issues.apache.org/jira/browse/CASSANDRA-6008 > Project: Cassandra > Issue Type: Bug > Components: Core > Reporter: John Carrino > Assignee: Tyler Hobbs > Fix For: 2.0.4 > > Attachments: 6008-2.0-part2.patch, 6008-2.0-v1.patch, > 6008-trunk-v1.patch > > > Exception encountered during startup: "Unfinished compactions reference > missing sstables. This should never happen since compactions are marked > finished before we start removing the old sstables" > This happens when sstables that have been compacted away are removed, but > they still have entries in the system.compactions_in_progress table. > Normally this should not happen because the entries in > system.compactions_in_progress are deleted before the old sstables are > deleted. > However at startup recovery time, old sstables are deleted (NOT BEFORE they > are removed from the compactions_in_progress table) and then after that is > done it does a truncate using SystemKeyspace.discardCompactionsInProgress > We ran into a case where the disk filled up and the node died and was bounced > and then failed to truncate this table on startup, and then got stuck hitting > this exception in ColumnFamilyStore.removeUnfinishedCompactionLeftovers. > Maybe on startup we can delete from this table incrementally as we clean > stuff up in the same way that compactions delete from this table before they > delete old sstables. -- This message was sent by Atlassian JIRA (v6.1.4#6159)