[ 
https://issues.apache.org/jira/browse/CASSANDRA-2521?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Sylvain Lebresne updated CASSANDRA-2521:
----------------------------------------

    Attachment: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch

Attaching patch against trunk. Tests are passing and it seems to work, at least 
with small tests. I started a stress on a 3 node cluster with a repair and a 
major compaction started towards the end and compacted files did wait to be 
fully streamed to be removed and I didn't hit any bump (I did hit 
CASSANDRA-2769 a bunch of time but that's another story).

Still, this is a fairly tricky problem so it could use other eyes. The basics 
are fairly simple though: each time a thread want to do something with a 
SSTableReader, it "acquires" a reference to that sstable and releases it when 
done. SSTableReader just keep a counter of acquired references. When the 
sstable has been marked compacted, we start looking until all acquired 
reference has been released. When that's the case, the file can be removed.

Obviously the main drawback of this approach (compared to the phantomReference 
one) is that there is room for error. If a consumer forgot to acquire a 
reference (or do it in a non-thread-safe manner), the sstable can be removed.  
Thankfully there is not so many place in the code that needs to do this so 
hopefully I haven't missed any place.

The other thing is that if a reference on a sstable is acquired, it should be 
released (otherwise the sstable will not be removed until next restart). I've 
try to ensure this using try-catch block, but it's not really possible with the 
way streaming works. However, if streaming fails, it's not really worst than 
before since the files where not cleaned due to the (failed) session staying in 
the global map of streaming sessions. CASSANDRA-2433 should fix that in most 
cases anyway.

Last thing is http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=4715154.  In 
other words, the deletion of a file won't work until the mmapping is finalized 
(aka, GC, Where art thou), at least on windows. For that reason, when the 
deletion of file fails (after the usual number of retries, which btw may make 
less sense now), the deletion task is saved in a global list. If Cassandra is 
low on disk, it will still trigger a GC, after which it will reschedule all 
failed files in the hope they can now be deleted. There is also a JMX call to 
retry this rescheduling.


> Move away from Phantom References for Compaction/Memtable
> ---------------------------------------------------------
>
>                 Key: CASSANDRA-2521
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-2521
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Chris Goffinet
>            Assignee: Sylvain Lebresne
>             Fix For: 1.0
>
>         Attachments: 
> 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch
>
>
> http://wiki.apache.org/cassandra/MemtableSSTable
> Let's move to using reference counting instead of relying on GC to be called 
> in StorageService.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to