cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From Piotr Kołaczkowski (Updated) (JIRA) <j...@apache.org>
Subject [jira] [Updated] (CASSANDRA-3578) Multithreaded commitlog
Date Wed, 11 Jan 2012 14:01:39 GMT

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

Piotr Kołaczkowski updated CASSANDRA-3578:
------------------------------------------

    Attachment: parallel_commit_log_2.patch

Changes contained in this patch:

1. Serialization and CRC is moved to a separate threadpool, making these operations concurrent.
However, appending serialized buffers to the commit log is *still serial*, so this patch cannot
be viewed as the final fix for the issue.

2. Semantic of the CommitLog.add method when configured with periodic CLES is slightly changed
- in that case the add method enqueues the request and returns immediately. It doesn't wait
even for the serialization, CRC and copying the RM into the commit log memory mapped segment.
If this behaviour makes some problems, the old behaviour can be easily brought back, with
only a performance penalty for additional synchronisation.

3. Segment syncing is done in parallel to CLS appending. This works perfectly at least on
Linux.

My observations while performing some limited performance testing while developing this patch:

1. CRC calculation is the CPU-heaviest operation while saving the RM.

2. Writing to the memory mapped buffer is extremely fast. My Dell Latitude can easily achieve
copying speeds of several GB/s. The serial commit log executor was not loaded fully, even
when everything was running on a RAMDISK and with 4 parallel serializer threads running on
all the 4 cores of the CPU. Parallelizing CL appends might not improve performance by a huge
factor, because probably we hit the memory throughput limit first, not the CPU. But anyway,
I think it still makes sense to parallelize it in order to avoid temporary serialized buffer
creation, which would offload GC.

3. When tested on a RAMDISK, I was able to get some minor performance improvement by being
careful not-waiting unnecesarily on locks e.g. Future objects. It is very important that for
small RMs, queues are long enough. If CL.add is blocking, the queues are usually short - their
size is limited by the number of active worker threads using CL. And with short queues, the
frequency of thread context switches rises.

4. I propose to limit the capacity of the BlockingQueues not by the number of RMs, but by
the predicted size of RMs. For large RMs, we probably don't want to enqueue too many of them,
not to waste memory or even get out of memory. On the other hand, for small RMs, longer queues
are better for keeping thread context switches low.
                
> Multithreaded commitlog
> -----------------------
>
>                 Key: CASSANDRA-3578
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3578
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Jonathan Ellis
>            Priority: Minor
>         Attachments: 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 is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

       

Mime
View raw message