[
https://issues.apache.org/jira/browse/CASSANDRA-3578?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13834545#comment-13834545
]
Jonathan Ellis commented on CASSANDRA-3578:
-------------------------------------------
Working through this still: https://github.com/jbellis/cassandra/commits/3578
Some comments so far:
Does it still make sense to track Pending Tasks when we don't really have an
executor per se? I'd be okay with removing it.
The guarantee people really want with PCLS is, "I won't lose more than X ms of
writes." That's not what we deliver, but that's what they want. Is that even
feasible?
Why does sync re-check non-current segments when shutting down? They are
synced in advanceAllocatingFrom.
We use "allocate" to mean both creating new Segments and reserving space within
a segment. Can we disambiguate these?
> Multithreaded commitlog
> -----------------------
>
> Key: CASSANDRA-3578
> URL: https://issues.apache.org/jira/browse/CASSANDRA-3578
> Project: Cassandra
> Issue Type: Improvement
> Reporter: Jonathan Ellis
> Assignee: Benedict
> Priority: Minor
> Labels: performance
> Attachments: 0001-CASSANDRA-3578.patch, ComitlogStress.java,
> Current-CL.png, Multi-Threded-CL.png, latency.svg, oprate.svg,
> 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)