cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Sylvain Lebresne (JIRA)" <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-2521) Move away from Phantom References for Compaction/Memtable
Date Mon, 27 Jun 2011 10:08:48 GMT

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

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

    Attachment: 2521-v4.txt

bq. why does DT.removeOldSSTableSize acquire/release around markCompacted?

For the record, this is because this make the logic in SSTR.markCompacted and SSTR.releaseReference
easier. If caller of markCompacted don't acquire a reference, markCompacted will have to deal
with two cases: either no thread have a reference acquired, in which case the current thread
should schedule the deletion, or other thread have a reference in which case it should let
them the task of scheduling the deletion where they are done. But making this thread safe
(so that we don't schedule twice or forget to schedule the deletion if the last thread holding
a reference release it at the same time as the markCompacted is called) is a bit of annoying.
Acquire a reference when markCompacted is called make this easier and move all the logic in
releaseReference.

bq. I believe currently, files are not deleted until the entire repair is finished

The file should get deleted as soon as they are not useful anymore, that is as soon as they
have been streamed. That being said, there was a bug, see below.

bq. Did another repair overnight, one minor compaction included some 20 small sstables, all
of them remains as well as a few from other compactions and the files from the repairs described
before

Yes, I did find a place where we were not correctly decrementing the reference count for streaming
(repair was not unmarking sstable that were not streamed because they had nothing for the
range to transfer). Attached v4 patch should fix that.

bq. As for the last version of this patch, a quick look tonight shows access problems with
markCurrentViewReferenced()

v4 is based on v3 and fixes this (it reintroduce a specific method instead of making View
public because I'm not too keen on doing that, but that can change if someone feels strongly
about that).


v4 also fix a bug in StreamingTransferTest and another one related to null segment in the
unmmapping cleanup code.

> 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
>          Components: Core
>            Reporter: Chris Goffinet
>            Assignee: Sylvain Lebresne
>             Fix For: 1.0
>
>         Attachments: 0001-Use-reference-counting-to-decide-when-a-sstable-can-.patch,
0001-Use-reference-counting-to-decide-when-a-sstable-can-v2.patch, 0002-Force-unmapping-files-before-deletion-v2.patch,
2521-v3.txt, 2521-v4.txt
>
>
> 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

        

Mime
View raw message