[ 
https://issues.apache.org/jira/browse/CASSANDRA-6086?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13817757#comment-13817757
 ] 

J. Ryan Earl commented on CASSANDRA-6086:
-----------------------------------------

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.

> Node refuses to start with exception in 
> ColumnFamilyStore.removeUnfinishedCompactionLeftovers when find that some to 
> be removed files are already removed
> ---------------------------------------------------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-6086
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-6086
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Oleg Anastasyev
>            Assignee: Oleg Anastasyev
>             Fix For: 2.0.3
>
>         Attachments: 6086-v2.txt, removeUnfinishedCompactionLeftovers.txt
>
>
> Node refuses to start with
> {code}
> Caused by: 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:544)
>       at 
> org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:262)
> {code}
> IMO, there is no reason to refuse to start discivering files that must be 
> removed are already removed. It looks like pure bug diagnostic code and mean 
> nothing to operator (nor he can do anything about this).
> Replaced throw of excepion with dump of diagnostic warning and continue 
> startup.



--
This message was sent by Atlassian JIRA
(v6.1#6144)

Reply via email to