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

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

Okay, so thinking on this a little further, I am going to try the following for 
the log file:

1) Before each sync, we will reserve a portion of the log file for writing 
another header, and will update the previous header to point to this new 
header, so we chain headers without needing to seek;
2) In order to allocate ourselves a position in the buffer, we place ourselves 
on a reverse-order linked list, with our position and size. Once we have 
written this we clear the size, but in the sync() we iterate over the list and 
ensure that the size (and size checksum) are written if they have not yet been 
cleared, so that the replay can be certain that an incomplete write of the size 
doesn't garble the log file. A race to write the size shouldn't matter, as they 
will both write the same value.
3) Each log message's tail checksum will be salted with the segment id. This is 
only to ensure we never see an object that is exactly the right size in exactly 
the write position in the file.

In doing this we should make ourselves able to cleanly replay with incompletely 
written log messages, and not worry about tracking exactly which log messages 
have been written (beyond whether or not we're dirty and/or completely 
written), shortening the CL write path.

> 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