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

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

I think there are two further improvements that need to be made, excluding some 
tidying up, before we should think about integration:

1) Obviously address optimising the CL format; and
2) Consider scrapping cfLastWrite beyond a set of names present in the log file

I haven't worried about (1) up until now as I'm testing on SSDs so the seeking 
shouldn't cause a major performance penalty, and I've mostly been interested in 
finding out how we can improve performance, but if we were to scrap (2) we 
could make (1) resilient to reading past not-yet-synced records, and we could 
stop worrying about linearising the write positions, eliminating a huge amount 
of the threading overhead. We would just increment a counter indicating we've 
dirtied the segment. The work we currently do would still be necessary for 
batch executors.

The only negatives here are that we might unnecessarily flush a Cf that has 
already fully synced in flushOldestKeyspaces (when trying to reclaim logs) and 
that we will be slightly slower to reclaim log files. On the whole I think it 
is a cost well worth paying for reducing 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