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

Benedict commented on CASSANDRA-3578:
-------------------------------------

bq. archiver.maybeArchive(segment.getPath(), segment.getName()) is a blocking 
call and will need to be a separate thread it might involve user defined 
archival.

This is already an asynchronous call, which is from your patch :-)

bq. how will we loose the log messages?

You only call force when you think there is something dirty, not when the 
buffer does, and this is determined by where the allocation position is at the 
time we write the buffer; if it has been updated since the last force. A thread 
may have allocated itself some room but not yet written to it before the 
force() call, so the end point in the log will point past the position we have 
last written to (and any log records after it will be unreadable due to the 
junk inbetween, or worse we may have some old records), and we may never update 
them if the allocation position has reached its limit, or we die.

If we're scrapping cfLastWrite we could plausibly just fall back on calling 
force() for every sync() and let it decide if any work needs to be done, 
though. I would prefer to still use a counter, as there's still a minor 
improbably concurrency bug where we could have a log file recycled before a 
thread has finished updating its portion of the buffer.

bq. I still like the original approach  of creating files (it may be just me) 
because of simplicity and we can be aggressive in allocator threads similar to 
your patch (to create empty files and deleting them).

Not sure I follow this one :-)



> Multithreaded commitlog
> -----------------------
>
>                 Key: CASSANDRA-3578
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3578
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Jonathan Ellis
>            Assignee: Vijay
>            Priority: Minor
>              Labels: performance
>         Attachments: 0001-CASSANDRA-3578.patch, ComitlogStress.java, 
> Current-CL.png, Multi-Threded-CL.png, parallel_commit_log_2.patch
>
>
> Brian Aker pointed out a while ago that allowing multiple threads to modify 
> the commitlog simultaneously (reserving space for each with a CAS first, the 
> way we do in the SlabAllocator.Region.allocate) can improve performance, 
> since you're not bottlenecking on a single thread to do all the copying and 
> CRC computation.
> Now that we use mmap'd CommitLog segments (CASSANDRA-3411) this becomes 
> doable.
> (moved from CASSANDRA-622, which was getting a bit muddled.)



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

Reply via email to