[
https://issues.apache.org/jira/browse/CASSANDRA-3578?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13815069#comment-13815069
]
Benedict commented on CASSANDRA-3578:
-------------------------------------
I liked the basic approach of your patch, Vijay. There were a number of
problems unaddressed:
- sync() would mark things as flushed to disk that weren't, which would result
in log messages never being persisted
- BatchCommitLogExecutor would ack the first message before syncing it
- markClean would mark current segment entirely clean, as opposed to up to
latest ReplayPosition; guessing this was for testing, but has performance
implications. Fixed.
- log replay was broken, minor fix
- checksum calculation was broken, minor fix
- sync() could be called simultaneously at shutdown, and wasn't threadsafe -
could have persisted garbled end pointer at start of log
I've made the following changes:
- data is marked as written in CLS after serializing to the buffer, so sync()
only advances the replay position to the most recent contiguously written point
in the buffer. Initially I used a skip list for this, but this seemed to be a
bit of a bottleneck. Now I buffer write positions in the order they arrive, and
merge them periodically (generally asynchronously, though blocking if we run
out of room to buffer). Possibly needs a slight tweak to absolutely guarantee
it can never run out of buffer space, but as it stands it's pretty much
impossible due to compaction of any adjacent records.
- Periodic and Batch CLEs are now the same class, with the only difference
being batch commit waits for a signal from the CommitLogSegment that it has
been syncd to disk
- PeriodicCLE is now fixed rate rather than fixed delay, and blocks any writes
if the previous sync hasn't completed after one commit log poll interval. This
is to maintain similar guarantees as before, although potentially we should
commit every 1/2 configured period, as we can lose up to 2 periods of data if
the previous sync is failing badly.
- Moved flushOldestKeyspaces into the CLE thread, and it now flushes keyspaces
from multiple old log files, not just the oldest, up to the number needed to
bring us below our limit. Since we can rapidly add many log files now, this
seemed necessary
I've also made some changes to encapsulate concurrency better, to minimise risk
of bugs. Allocation is all done inside CLA, as opposed to split between CLA and
CL. CLA also now allocates new segments as soon as the last reserve segment is
used, as opposed to every 100ms, in case we ever have a situation where we
exhaust segments in < 100ms.
After making all of these changes, I actually found very little improvement in
performance. It turns out this was because the point of contention is moving
from the CL to the switchLock, and I still see huge spikes when waiting for a
flush here. I've tested these changes with my in progress patch for
[6271|https://issues.apache.org/jira/browse/CASSANDRA-6271] and found up to 2x
performance boost for high thread counts. It's quite likely a patch for
[5549|https://issues.apache.org/jira/browse/CASSANDRA-5549] will help further.
[https://github.com/belliottsmith/cassandra/tree/iss-3578]
[https://github.com/belliottsmith/cassandra/tree/iss-3578+6271] - note this is
far from prod ready, just useful for performance testing. might only work with
stress.
> 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)