Return-Path: Delivered-To: apmail-cassandra-commits-archive@www.apache.org Received: (qmail 17924 invoked from network); 24 Jan 2011 22:02:09 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (140.211.11.3) by minotaur.apache.org with SMTP; 24 Jan 2011 22:02:09 -0000 Received: (qmail 67529 invoked by uid 500); 24 Jan 2011 22:02:09 -0000 Delivered-To: apmail-cassandra-commits-archive@cassandra.apache.org Received: (qmail 67454 invoked by uid 500); 24 Jan 2011 22:02:09 -0000 Mailing-List: contact commits-help@cassandra.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@cassandra.apache.org Delivered-To: mailing list commits@cassandra.apache.org Received: (qmail 67446 invoked by uid 99); 24 Jan 2011 22:02:08 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 24 Jan 2011 22:02:08 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=10.0 tests=ALL_TRUSTED X-Spam-Check-By: apache.org Received: from [140.211.11.22] (HELO thor.apache.org) (140.211.11.22) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 24 Jan 2011 22:02:06 +0000 Received: from thor (localhost [127.0.0.1]) by thor.apache.org (8.13.8+Sun/8.13.8) with ESMTP id p0OM1iQV022257 for ; Mon, 24 Jan 2011 22:01:44 GMT Message-ID: <31182341.171581295906504695.JavaMail.jira@thor> Date: Mon, 24 Jan 2011 17:01:44 -0500 (EST) From: "Peter Schuller (JIRA)" To: commits@cassandra.apache.org Subject: [jira] Commented: (CASSANDRA-1991) CFS.maybeSwitchMemtable() calls CommitLog.instance.getContext(), which may block, under flusher lock write lock In-Reply-To: <29802393.388081295116246041.JavaMail.jira@thor> MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 X-Virus-Checked: Checked by ClamAV on apache.org [ https://issues.apache.org/jira/browse/CASSANDRA-1991?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12986051#action_12986051 ] Peter Schuller commented on CASSANDRA-1991: ------------------------------------------- I'll revisit and try to figure out how to decrease fragility/complexity. CASSANDRA-1955 had the potential to increase complexity even more to this part of the code (I was close to suggesting a less complete solution there just to avoid tackling it properly as the "proper" fix required moving certain things out from under the flusher lock). I'm thinking a bigger change may be worth it if the result is clearer and less complex. Disregarding for a moment the additional complexity implied by the last patch, would you agree though that it is at least plausibly a good idea, given that it can be done cleanly, to separate checkpointing from flushing, and that it is a good idea to make checkpoint explicit rather than an implicit side-effect of all flushes? It "feels" like there exists a clean solution that would address both this and CASSANDRA-1955 implicitly without an increase in complexity. Just need to find it. > 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. - You can reply to this email to add a comment to the issue online.