[
https://issues.apache.org/jira/browse/CASSANDRA-6008?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13817760#comment-13817760
]
J. Ryan Earl commented on CASSANDRA-6008:
-----------------------------------------
Hi, we are able to consistently reproduce this issue:
{noformat}
ERROR 23:14:06,001 Exception encountered during startup
java.lang.IllegalStateException: Unfinished compactions reference missing
sstables. This should never happen since compactions are marked finished before
we start removing the old sstables.
at
org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:489)
at
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:264)
at
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:461)
at
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:504)
java.lang.IllegalStateException: Unfinished compactions reference missing
sstables. This should never happen since compactions are marked finished before
we start removing the old sstables.
at
org.apache.cassandra.db.ColumnFamilyStore.removeUnfinishedCompactionLeftovers(ColumnFamilyStore.java:489)
at
org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:264)
at
org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:461)
at
org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:504)
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.
{noformat}
Here are the two ways in which we have found to reproduce this issue:
# Buffer a large amount of CL.LOCAL_QUORUM writes into a Cassandra CF, then
drop the CF while the writes are still buffered.
# Buffer a large amount of CL.LOCAL_QUORUM writes into a Cassandra CF, then
restart some nodes while before the buffer has finished draining.
> 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: Yuki Morishita
> Fix For: 2.0.3
>
>
> 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#6144)