[
https://issues.apache.org/jira/browse/CASSANDRA-3578?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13835265#comment-13835265
]
Benedict edited comment on CASSANDRA-3578 at 11/29/13 9:23 AM:
---------------------------------------------------------------
bq. By the time we request sync in advanceAllocatingFrom, we've already
modified allocatingFrom so the new empty segment is what will get synced. No?
We only request an out-of-band regular sync, so it will just try to sync
anything that hasn't yet been synced (which should only be the just finished
segment). It's possible we will *also* sync the beginning of the new segment if
it has already been written to. We could put in behaviour to prevent that if we
want to avoid the potential seek cost, but since it's only oncurred once per
CLS allocation, I'm not sure it's worth making the code anymore convoluted than
it currently is.
bq. "Now recycle segments that are unused, as we may not have triggered a
discardCompletedSegments." Why not?
discardCompletedSegments() is only called if a memtable flushes something. We
could already be marked clean (both in CL and Memtable), but the most recent
segment would have still been "in use" (because "in use" is effectively
!current && clean), so once we switch in a new current segment we can recycle
it, but discardCompletedSegments() won't be invoked to do so.
A couple of comments on your changes:
{noformat}
List<CommitLogSegment> segmentsToRecycle = new ArrayList<>(availableSegments);
{noformat}
should be
{noformat}
List<CommitLogSegment> segmentsToRecycle = new ArrayList<>(activeSegments);
{noformat}
... and I'm a little concerned about bringing the flushDataFrom() calls to
forceFlush() into the same thread in general, as most calls will be from the
CLSManager thread, and switchMemtable() is currently (and may well remain, to
some degree) a blocking call. So we could block incoming CL.add() calls
unnecessarily as they wait for a new segment. We could perform the entire
flushDataFrom() call from the CLSManager thread on the optionalTasks executor,
though, to keep your clarification.
was (Author: benedict):
bq. By the time we request sync in advanceAllocatingFrom, we've already
modified allocatingFrom so the new empty segment is what will get synced. No?
We only request an out-of-band regular sync, so it will just try to sync
anything that hasn't yet been synced (which should only be the just finished
segment). It's possible we will *also* sync the beginning of the new segment if
it has already been written to. We could put in behaviour to prevent that if we
want to avoid the potential seek cost, but since it's only oncurred once per
CLS allocation, I'm not sure it's worth making the code anymore convoluted than
it currently is.
bq. "Now recycle segments that are unused, as we may not have triggered a
discardCompletedSegments." Why not?
discardCompletedSegments() is only called if a memtable flushes something. We
could already be marked clean (both in CL and Memtable), but the most recent
segment would have still been "in use" (because "in use" is effectively
!current && clean), so once we switch in a new current segment we can recycle
it, but discardCompletedSegments() won't be invoked to do so.
A couple of comments on your changes:
{noformat}
List<CommitLogSegment> segmentsToRecycle = new ArrayList<>(availableSegments);
{noformat}
should be
{noformat}
List<CommitLogSegment> segmentsToRecycle = new ArrayList<>(activeSegments);
{noformat}
... and I'm a little concerned about bringing the flushDataFrom() calls to
forceFlush() into the same thread in general, as most calls will be from the
CLSManager, and switchMemtable() is currently (and may well remain, to some
degree) a blocking call. So we could block incoming CL.add() calls
unnecessarily as they wait for a new segment. We could perform the entire
flushDataFrom() call in CLSManager on the optionalTasks executor, though, to
keep your clarification.
> 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, TestEA.java, 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)