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

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

bq. 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.

I think it still serves a purpose - it now means something different, though, 
which is 'those tasks that are blocked on a sync()'. This can happen for both 
PCLE and BCLE, and I expect if some scenario crops up that holds the sync() up, 
it could well be useful for debugging what's happening if you can see that 
statistic. Now that I think about it, another 'statistic' should be exposed as 
well, perhaps, which is timeOfLastSync(), which would help spot that scenario.

bq. 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?

We provide this guarantee just as well as we used to - the maybeWaitForSync() 
in your modified patch blocks if the sync() ever gets more than pollInterval 
behind (i.e. doesn't update its lastAliveAt heartbeat in that period), until it 
catches up. I've tested this by pumping too much data through, so that the 
sync() couldn't quite keep up, and it works as intended. The only major 
difference is I tack an extra 10ms on as a cludgey guarantee we don't wait when 
the poll is just waking up and getting itself together. We could, and perhaps 
should, instead subtract this from the pollInterval (with perhaps the cludge 
dropping to 1ms if the pollInterval is < 20ms).

bq. Why does sync re-check non-current segments when shutting down? They are 
synced in advanceAllocatingFrom.

advanceAllocatingFrom only requests a sync(), it doesn't actually perform one - 
shutdown and the normal pollInterval sync are now the same process, so they 
actually both "sync" CLS we have already finished syncing. This is the same 
behaviour as before wrt resyncing already synced segments. I'm not too worried 
about this, since they're simply a monitor acquire/release followed by a 
comparison of two integers if they're already synced, so we're not wasting any 
time going to disk, or even more than a hundred cycles or so per segment, 
probably fewer since the monitor will be biased. We could move them into 
another queue of completed CLS once we've finished syncing, and only check that 
queue for recycling. I'd be happy with that. But I don't think it's an issue 
unless you've got tens of thousands of finished-but-not-yet-recycled segments.

bq. We use "allocate" to mean both creating new Segments and reserving space 
within a segment. Can we disambiguate these?

You're right, and it looks to me like you've achieved this with your 
segmentManagementTasks rename - if we rename the thread to 
CommitLogSegmentManager or something, and the wake?() method accordingly, I 
think that probably suffices.

One thing I would say about your cleanup: I deliberately allocated the 
Allocation object in the caller and set its fields in the callee to guarantee 
escape analysis kicks in to avoid allocating the objects on the heap. It's 
likely that it will find the allocations don't escape, but I'm not sure how 
well it handles back-tracking out of the allocating method to do so.

Also, my IDEA fu is weak, I tried to find a SameThreadES in our dependencies, 
but couldn't! Obviously I didn't try very hard.

> 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)

Reply via email to