[ https://issues.apache.org/jira/browse/CASSANDRA-15367?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17025850#comment-17025850 ]
Benedict Elliott Smith commented on CASSANDRA-15367: ---------------------------------------------------- I've pushed a branch that implements your suggested approach. I wanted to get your feedback on whether or not you think we should try to mitigate the issue of competition in these overflow scenario. One approach would be to ignore most of the work I did initially wrt {{extractUnshadowed}}, and _only in the event we will refuse to lock_: # Do not {{abortEarly}} for our first update attempt # For remaining attempts, apply the whole new partition result to the current head, until we succeed This may mean increased heap allocations for each failed attempt, and each individual update may be slightly slower, but there will be: # No extra heap allocations for successful attempts # We can avoid re-cloning values into the memtable space (which is not reclaimable until flush) # Majority of operations are entirely unaffected, so we don't have to worry much about performance impact WDYT? I'm _relatively_ comfortable doing nothing to mitigate this scenario, too. But it _might_ be preferable to do this. > 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