[
https://issues.apache.org/jira/browse/CASSANDRA-1991?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12982297#action_12982297
]
Peter Schuller commented on CASSANDRA-1991:
-------------------------------------------
I realized my mistake. Of course in the absence of new writes, the sync() will
indeed have happened "late enough" since the only constraint is that no commit
log entries be added after the sync(); the flushing of dirty memtables is fine.
However, I still think there are several problem left:
(1) For clean memtables, maybeSwitchMemtable() is not called sa a result of a
forced flush (filtered in CFS.forceFlush()).
(2) The use of forceBlockingFlush() in truncate() seems to me to be of a kind
where, similar to drain(), it is important that past commit log entries are not
replayed (there is also the secondary index related calls, but I'm not sure of
the requirements there)
(3) If writes happen to a CF and then suddenly stop, one can either:
(a) retain commit logs forever if the final write happened to land exactly
such that a flush happened after it, or
(b) retain memtable_flush_after_mins worth of commit logs if it didn't
(c) maybe forever also if the memtable_flush_after_mins triggers after the
final write but before a commit log sync
It almost seems warranted to be to expose to interfaces; the current
forceFlush()/forceBlockingFlush() would just say "ensure all writes up to this
point are persistent". But then, add something (checkpoint()?) which means
"ensure that commit log won't be replayed for anything flushed prior to calling
this method". That should take care of (1) and (2) and hopefully make it a bit
more explicit.
It still does not directly address (3) though. I'm not sure what the cleanest
solution is there. Maybe just having a periodic checkpoint() going on (feels a
bit hacky)?
> 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-logchanges.txt, 1991-trunk-v2.txt, 1991-trunk.txt,
> 1991-v3.txt, 1991-v4.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.