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

Jonathan Ellis commented on CASSANDRA-1991:
-------------------------------------------

What if we just had getContext cheat and inject its task at the front of the 
commitlog executor's queue, instead of at the end?  This would mean we might 
replay data unnecessarily after a crash (the tasks we cut in front of), but 
that's Not A Big Deal.
                
> CFS.maybeSwitchMemtable() calls CommitLog.instance.getContext(), which may 
> block, under flusher lock write lock
> ---------------------------------------------------------------------------------------------------------------
>
>                 Key: CASSANDRA-1991
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-1991
>             Project: Cassandra
>          Issue Type: Improvement
>            Reporter: Peter Schuller
>            Assignee: Peter Schuller
>         Attachments: 1991-checkpointing-flush.txt, 1991-logchanges.txt, 
> 1991-trunk-v2.txt, 1991-trunk.txt, 1991-v3.txt, 1991-v4.txt, 1991-v5.txt, 
> 1991-v6.txt, 1991-v7.txt, 1991-v8.txt, 1991-v9.txt, trigger.py
>
>
> While investigate CASSANDRA-1955 I realized I was seeing very poor latencies 
> for reasons that had nothing to do with flush_writers, even when using 
> periodic commit log mode (and flush writers set ridiculously high, 500).
> It turns out writes blocked were slow because Table.apply() was spending lots 
> of time (I can easily trigger seconds on moderate work-load) trying to 
> acquire a flusher lock read lock ("flush lock millis" log printout in the 
> logging patch I'll attach).
> That in turns is caused by CFS.maybeSwitchMemtable() which acquires the 
> flusher lock write lock.
> Bisecting further revealed that the offending line of code that blocked was:
>                     final CommitLogSegment.CommitLogContext ctx = 
> writeCommitLog ? CommitLog.instance.getContext() : null;
> Indeed, CommitLog.getContext() simply returns currentSegment().getContext(), 
> but does so by submitting a callable on the service executor. So 
> independently of flush writers, this can block all (global, for all cf:s) 
> writes very easily, and does.
> I'll attach a file that is an independent Python script that triggers it on 
> my macos laptop (with an intel SSD, which is why I was particularly 
> surprised) (it assumes CPython, out-of-the-box-or-almost Cassandra on 
> localhost that isn't in a cluster, and it will drop/recreate a keyspace 
> called '1955').
> I'm also attaching, just FYI, the patch with log entries that I used while 
> tracking it down.
> Finally, I'll attach a patch with a suggested solution of keeping track of 
> the latest commit log with an AtomicReference (as an alternative to 
> synchronizing all access to segments). With that patch applied, latencies are 
> not affected by my trigger case like they were before. There are some 
> sub-optimal > 100 ms cases on my test machine, but for other reasons. I'm no 
> longer able to trigger the extremes.

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

        

Reply via email to