You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Joshua McKenzie (JIRA)" <ji...@apache.org> on 2016/07/13 22:42:20 UTC

[jira] [Commented] (CASSANDRA-12198) Deadlock in CDC during segment flush

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

Joshua McKenzie commented on CASSANDRA-12198:
---------------------------------------------

Changed synchronization to {{CommitLogSegment.cdcState}} in CDCSizeTracker and {{CommitLogSegment.setCDCState}}. This should give us the previously desired effect of atomic changes to this state without exposing us to the risk of deadlock by other unrelated methods synchronizing on the segment.

The other 2 uses of the cdcState should be unaffected by this (write path allocation check, discard handling in segment manager) due to rules of transition (only set FORBIDDEN on segment creation, only transition from PERMITTED to CONTAINS) and discard check should be guarded by OpOrder barrier and flushing mechanisms.

Given I only saw this once in the wild while working on 12148 and the very infrequent nature of it due to segment sync interaction, I'd prefer we get a review and get this into 3.8 rather than blocking release to try and get a reproduction test.

Ran some targeted unit tests locally w/test-cdc and things look fine (CommitLogSegmentManagerCDCTest, CommitLogTest, CommitLogStressTests). CI is running now.

||branch||testall||dtest||
|[12198|https://github.com/apache/cassandra/compare/cassandra-3.8...josh-mckenzie:12198?expand=1]|[testall|http://cassci.datastax.com/view/Dev/view/josh-mckenzie/job/josh-mckenzie-12198-testall]|[dtest|http://cassci.datastax.com/view/Dev/view/josh-mckenzie/job/josh-mckenzie-12198-dtest]|

> Deadlock in CDC during segment flush
> ------------------------------------
>
>                 Key: CASSANDRA-12198
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-12198
>             Project: Cassandra
>          Issue Type: Bug
>            Reporter: Joshua McKenzie
>            Assignee: Joshua McKenzie
>            Priority: Blocker
>             Fix For: 3.8
>
>
> In the patch for CASSANDRA-8844, we added a {{synchronized(this)}} block inside CommitLogSegment.setCDCState. This introduces the possibility of deadlock in the following scenario:
> # A {{CommitLogSegment.sync()}} call is made (synchronized method)
> # A {{CommitLogSegment.allocate}} call from a cdc-enabled write is in flight and acquires a reference to the Group on appendOrder (the OpOrder in the Segment)
> # {{CommmitLogSegment.sync}} hits {{waitForModifications}} which calls {{appendOrder.awaitNewBarrier}}
> # The in-flight write, if changing the state of the segment from CDCState.PERMITTED to CDCState.CONTAINS, enters {{setCDCState}} and blocks on synchronized(this)
> And neither of them ever come back. This came up while doing some further work on CASSANDRA-12148.



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)