Fix potential deadlock in CDC state tracking Patch by jmckenzie; reviewed by cyeksigian for CASSANDRA-12198
Project: http://git-wip-us.apache.org/repos/asf/cassandra/repo Commit: http://git-wip-us.apache.org/repos/asf/cassandra/commit/90afc58d Tree: http://git-wip-us.apache.org/repos/asf/cassandra/tree/90afc58d Diff: http://git-wip-us.apache.org/repos/asf/cassandra/diff/90afc58d Branch: refs/heads/cassandra-3.9 Commit: 90afc58d3df912c720aff63de0506019b8b9af48 Parents: e3f9b7a Author: Josh McKenzie <jmcken...@apache.org> Authored: Wed Jul 13 18:30:40 2016 -0400 Committer: Josh McKenzie <jmcken...@apache.org> Committed: Thu Jul 14 10:36:43 2016 -0400 ---------------------------------------------------------------------- src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java | 3 ++- .../cassandra/db/commitlog/CommitLogSegmentManagerCDC.java | 4 ++-- 2 files changed, 4 insertions(+), 3 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/cassandra/blob/90afc58d/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java index 2e97fd5..a1158be 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegment.java @@ -61,6 +61,7 @@ public abstract class CommitLogSegment FORBIDDEN, CONTAINS } + Object cdcStateLock = new Object(); private final static AtomicInteger nextId = new AtomicInteger(1); private static long replayLimitId; @@ -614,7 +615,7 @@ public abstract class CommitLogSegment return; // Also synchronized in CDCSizeTracker.processNewSegment and .processDiscardedSegment - synchronized(this) + synchronized(cdcStateLock) { if (cdcState == CDCState.CONTAINS && newState != CDCState.CONTAINS) throw new IllegalArgumentException("Cannot transition from CONTAINS to any other state."); http://git-wip-us.apache.org/repos/asf/cassandra/blob/90afc58d/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java ---------------------------------------------------------------------- diff --git a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java index 5c6fd3f..04beb20 100644 --- a/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java +++ b/src/java/org/apache/cassandra/db/commitlog/CommitLogSegmentManagerCDC.java @@ -187,7 +187,7 @@ public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager void processNewSegment(CommitLogSegment segment) { // See synchronization in CommitLogSegment.setCDCState - synchronized(segment) + synchronized(segment.cdcStateLock) { segment.setCDCState(defaultSegmentSize() + totalCDCSizeOnDisk() > allowableCDCBytes() ? CDCState.FORBIDDEN @@ -203,7 +203,7 @@ public class CommitLogSegmentManagerCDC extends AbstractCommitLogSegmentManager void processDiscardedSegment(CommitLogSegment segment) { // See synchronization in CommitLogSegment.setCDCState - synchronized(segment) + synchronized(segment.cdcStateLock) { // Add to flushed size before decrementing unflushed so we don't have a window of false generosity if (segment.getCDCState() == CDCState.CONTAINS)