cassandra-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Peter Schuller (JIRA)" <j...@apache.org>
Subject [jira] Commented: (CASSANDRA-1991) CFS.maybeSwitchMemtable() calls CommitLog.instance.getContext(), which may block, under flusher lock write lock
Date Mon, 17 Jan 2011 08:41:44 GMT

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

Peter Schuller commented on CASSANDRA-1991:
-------------------------------------------

If there is no contract to guarantee that we don't do log replay if all CF:s have been flushed
without any write happening after the flush - even in the case of drain - then no, there's
no problem as far as I can tell. I was under the impression this (a "checkpoint" if you will,
in postgresql speak; ensuring that log reply is not needed further back than a certain point)
was a design goal of drain() for the purpose of e.g. upgrades across incompatible rowmutation
serializations.

If not, then nevermind.

If yes, the simplest solution I can think of (only got code partially written yet though)
is to just leave forceFlush() exactly the way it originally was but introduce a checkpoint()
which is only called upon these administrative type actions like drain or schema changes (so
there's no concern that it always implies a commit log sync for simplicity). Checkpoint would
flush if memtable is dirty (after syncing) followed by the the discard (still after syncing
even if memtable was clean).

That should make it very explicit that you're relying on a checkpoint happening at the call
site, and it seems pretty simple in terms of understanding what's going on.

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


Mime
View raw message