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


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.
-
You can reply to this email to add a comment to the issue online.

Reply via email to