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

Tyler Hobbs commented on CASSANDRA-7234:
----------------------------------------

bq. I couldn't really reproduce this, could you elaborate when this happens? - 
If we drop a CF, we always interrupt the ongoing compactions.

The timing has to be just right.  It only checks to see if the compaction has 
been interrupted before writing each row.  If the drop happens after the final 
row is written but before {{replaceCompactedSSTables()}} is run (basically, the 
{{finally}} block), you'll see this problem.  I saw it occasionally while 
trying to reproduce CASSANDRA-6525, where a lot of compactions were backed up 
when the table was dropped.

+1 on the patch, though.

> Post-compaction cache preheating can result in FileNotFoundExceptions when 
> tables are dropped
> ---------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-7234
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-7234
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>            Reporter: Tyler Hobbs
>            Assignee: Marcus Eriksson
>            Priority: Minor
>              Labels: compaction
>             Fix For: 2.0.9
>
>         Attachments: 0001-reference-sstable-before-populating-keycache.patch
>
>
> In CompactionTask.java, after a compaction finishes, we do this:
> {code}
>         replaceCompactedSSTables(toCompact, sstables);
>         // TODO: this doesn't belong here, it should be part of the reader to 
> load when the tracker is wired up
>         for (SSTableReader sstable : sstables)
>             sstable.preheat(cachedKeyMap.get(sstable.descriptor));
> {code}
> The problem is that if the table was dropped, {{replaceCompactedSSTables}} 
> will release its references on the new {{sstables}}, resulting in them being 
> closed.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to