[ https://issues.apache.org/jira/browse/CASSANDRA-15367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17026922#comment-17026922 ]
Blake Eggleston commented on CASSANDRA-15367: --------------------------------------------- Nice. I’m also fairly comfortable not addressing competition in this scenario. Ideally, we would, but I’m not sure it would be worth adding a secondary synchronization path. Although I guess we could synchronize on the writeOp. There still is (technically) a brief window for deadlock between {{setCommitLogUpperBound}} and {{writeBarrier.issue()}} in {{org.apache.cassandra.db.ColumnFamilyStore.Flush#Flush}}, but I’m not sure if it’s worth addressing, since we’d need to immediately waste 10MB on a partition as soon as a memtable is created, and it’s not exacerbated by flush queue length. Anyway, I think this qualifies as good enough. I’d also prefer it over waiting on the previous op group because it limits the window of potential bad behavior to a narrower set of circumstances. What do you think? About removing the lock, I’m sure 15511 will help with contention, and we should commit it, however I think there will still be pathological cases where faster updates won’t be enough. For instance, if there were 20 small updates and one much larger one contending with each other, I can imagine the large one would have a tough time making progress and end up wasting a lot of memory. <random-idea> This might be better illustrated with code, and would be a trunk-only follow on ticket, but instead of synchronizing writes on the partition object whenever there’s contention, what if we queued up contended writes on the partition? If a write comes in and there’s no longer contention, or the size of queued writes is too high, it could merge the updates and synchronize on applying them. By merging the updates, I think we’d end up allocating less memory in the contended case than the uncontended case. </random-idea> > Memtable memory allocations may deadlock > ---------------------------------------- > > Key: CASSANDRA-15367 > URL: https://issues.apache.org/jira/browse/CASSANDRA-15367 > Project: Cassandra > Issue Type: Bug > Components: Local/Commit Log, Local/Memtable > Reporter: Benedict Elliott Smith > Assignee: Benedict Elliott Smith > Priority: Normal > Fix For: 4.0, 2.2.x, 3.0.x, 3.11.x > > > * Under heavy contention, we guard modifications to a partition with a mutex, > for the lifetime of the memtable. > * Memtables block for the completion of all {{OpOrder.Group}} started before > their flush began > * Memtables permit operations from this cohort to fall-through to the > following Memtable, in order to guarantee a precise commitLogUpperBound > * Memtable memory limits may be lifted for operations in the first cohort, > since they block flush (and hence block future memory allocation) > With very unfortunate scheduling > * A contended partition may rapidly escalate to a mutex > * The system may reach memory limits that prevent allocations for the new > Memtable’s cohort (C2) > * An operation from C2 may hold the mutex when this occurs > * Operations from a prior Memtable’s cohort (C1), for a contended partition, > may fall-through to the next Memtable > * The operations from C1 may execute after the above is encountered by those > from C2 -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org For additional commands, e-mail: commits-h...@cassandra.apache.org