[jira] [Updated] (CASSANDRA-13587) Deadlock during CommitLog replay when Cassandra restarts

2017-08-30 Thread Jason Brown (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jason Brown updated CASSANDRA-13587:

   Resolution: Fixed
Fix Version/s: (was: 3.0.x)
   3.0.15
   Status: Resolved  (was: Patch Available)

I reread the 3.0 CommitLog code this morning, and determined this change is 
actually pretty safe. We don't recycle the replayed commit log files (just 
delete them as of 2.2), so the {{createReserveSegments}} is pretty useless 
anyway.

+1 and committed as sha {{d03c046acbcfe3e3a9d8fafa628030cc3fc40f34}} to 3.0 
only.

Thanks for the patch [~chovatia.jayd...@gmail.com], and for the nice analysis!

> Deadlock during CommitLog replay when Cassandra restarts
> 
>
> Key: CASSANDRA-13587
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13587
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Jaydeepkumar Chovatia
>Assignee: Jaydeepkumar Chovatia
> Fix For: 3.0.15
>
> Attachments: 13587-3.0.txt, Reproduce_CASSANDRA-13587.txt
>
>
> Possible deadlock found when Cassandra is replaying commit log and at the 
> same time Mutation gets triggered by 
> SSTableReader(SystemKeyspace.persistSSTableReadMeter). As a result Cassandra 
> restart hangs forever
> Please find details of stack trace here:
> *Frame#1* This thread is trying to apply {{persistSSTableReadMeter}} mutation 
> and as a result it has called {{writeOrder.start()}} in {{Keyspace.java:533}}
> but there are no Commitlog Segments available because {{createReserveSegments 
> (CommitLogSegmentManager.java)}} is not yet {{true}} 
> Hence this thread is blocked on {{createReserveSegments}} to become {{true}}, 
> please note this thread has already started {{writeOrder}}
> {quote}
> "pool-11-thread-1" #251 prio=5 os_prio=0 tid=0x7fe128478400 nid=0x1b274 
> waiting on condition [0x7fe1389a]
>java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
> at 
> org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279)
> at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.advanceAllocatingFrom(CommitLogSegmentManager.java:277)
> at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.allocate(CommitLogSegmentManager.java:196)
> at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:260)
> at org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:540)
> at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:421)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:210)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:215)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:224)
> at 
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternalWithoutCondition(ModificationStatement.java:566)
> at 
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(ModificationStatement.java:556)
> at 
> org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:295)
> at 
> org.apache.cassandra.db.SystemKeyspace.persistSSTableReadMeter(SystemKeyspace.java:1181)
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$GlobalTidy$1.run(SSTableReader.java:2202)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> {quote}
> *Frame#2* This thread is trying to recover commit logs and as a result it 
> tries to flush Memtable by calling following code:
> {{futures.add(Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceFlush());}}
> As a result Frame#3 (below) gets created
> {quote}
> "main" #1 prio=5 os_prio=0 tid=0x7fe1c64ec400 nid=0x1af29 waiting on 
> condition [0x7fe1c94a1000]
>java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> parking to wait for  <0x0006370da0c0> (a 
> com.google.common.util.concurrent.ListenableFutureTask)
> at 

[jira] [Updated] (CASSANDRA-13587) Deadlock during CommitLog replay when Cassandra restarts

2017-08-29 Thread Jason Brown (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jason Brown updated CASSANDRA-13587:

Reviewer: Jason Brown

> Deadlock during CommitLog replay when Cassandra restarts
> 
>
> Key: CASSANDRA-13587
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13587
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Jaydeepkumar Chovatia
>Assignee: Jaydeepkumar Chovatia
> Fix For: 3.0.x
>
> Attachments: 13587-3.0.txt, Reproduce_CASSANDRA-13587.txt
>
>
> Possible deadlock found when Cassandra is replaying commit log and at the 
> same time Mutation gets triggered by 
> SSTableReader(SystemKeyspace.persistSSTableReadMeter). As a result Cassandra 
> restart hangs forever
> Please find details of stack trace here:
> *Frame#1* This thread is trying to apply {{persistSSTableReadMeter}} mutation 
> and as a result it has called {{writeOrder.start()}} in {{Keyspace.java:533}}
> but there are no Commitlog Segments available because {{createReserveSegments 
> (CommitLogSegmentManager.java)}} is not yet {{true}} 
> Hence this thread is blocked on {{createReserveSegments}} to become {{true}}, 
> please note this thread has already started {{writeOrder}}
> {quote}
> "pool-11-thread-1" #251 prio=5 os_prio=0 tid=0x7fe128478400 nid=0x1b274 
> waiting on condition [0x7fe1389a]
>java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
> at 
> org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279)
> at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.advanceAllocatingFrom(CommitLogSegmentManager.java:277)
> at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.allocate(CommitLogSegmentManager.java:196)
> at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:260)
> at org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:540)
> at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:421)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:210)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:215)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:224)
> at 
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternalWithoutCondition(ModificationStatement.java:566)
> at 
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(ModificationStatement.java:556)
> at 
> org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:295)
> at 
> org.apache.cassandra.db.SystemKeyspace.persistSSTableReadMeter(SystemKeyspace.java:1181)
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$GlobalTidy$1.run(SSTableReader.java:2202)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> {quote}
> *Frame#2* This thread is trying to recover commit logs and as a result it 
> tries to flush Memtable by calling following code:
> {{futures.add(Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceFlush());}}
> As a result Frame#3 (below) gets created
> {quote}
> "main" #1 prio=5 os_prio=0 tid=0x7fe1c64ec400 nid=0x1af29 waiting on 
> condition [0x7fe1c94a1000]
>java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> parking to wait for  <0x0006370da0c0> (a 
> com.google.common.util.concurrent.ListenableFutureTask)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:429)
> at java.util.concurrent.FutureTask.get(FutureTask.java:191)
> at 
> org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:383)
> at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.blockForWrites(CommitLogReplayer.java:207)
> at 
> org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:182)
> at 
> 

[jira] [Updated] (CASSANDRA-13587) Deadlock during CommitLog replay when Cassandra restarts

2017-06-22 Thread Michael Shuler (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Michael Shuler updated CASSANDRA-13587:
---
Fix Version/s: (was: 3.0.14)
   3.0.x

> Deadlock during CommitLog replay when Cassandra restarts
> 
>
> Key: CASSANDRA-13587
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13587
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Jaydeepkumar Chovatia
>Assignee: Jaydeepkumar Chovatia
> Fix For: 3.0.x
>
> Attachments: 13587-3.0.txt, Reproduce_CASSANDRA-13587.txt
>
>
> Possible deadlock found when Cassandra is replaying commit log and at the 
> same time Mutation gets triggered by 
> SSTableReader(SystemKeyspace.persistSSTableReadMeter). As a result Cassandra 
> restart hangs forever
> Please find details of stack trace here:
> *Frame#1* This thread is trying to apply {{persistSSTableReadMeter}} mutation 
> and as a result it has called {{writeOrder.start()}} in {{Keyspace.java:533}}
> but there are no Commitlog Segments available because {{createReserveSegments 
> (CommitLogSegmentManager.java)}} is not yet {{true}} 
> Hence this thread is blocked on {{createReserveSegments}} to become {{true}}, 
> please note this thread has already started {{writeOrder}}
> {quote}
> "pool-11-thread-1" #251 prio=5 os_prio=0 tid=0x7fe128478400 nid=0x1b274 
> waiting on condition [0x7fe1389a]
>java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
> at 
> org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279)
> at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.advanceAllocatingFrom(CommitLogSegmentManager.java:277)
> at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.allocate(CommitLogSegmentManager.java:196)
> at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:260)
> at org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:540)
> at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:421)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:210)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:215)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:224)
> at 
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternalWithoutCondition(ModificationStatement.java:566)
> at 
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(ModificationStatement.java:556)
> at 
> org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:295)
> at 
> org.apache.cassandra.db.SystemKeyspace.persistSSTableReadMeter(SystemKeyspace.java:1181)
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$GlobalTidy$1.run(SSTableReader.java:2202)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> {quote}
> *Frame#2* This thread is trying to recover commit logs and as a result it 
> tries to flush Memtable by calling following code:
> {{futures.add(Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceFlush());}}
> As a result Frame#3 (below) gets created
> {quote}
> "main" #1 prio=5 os_prio=0 tid=0x7fe1c64ec400 nid=0x1af29 waiting on 
> condition [0x7fe1c94a1000]
>java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> parking to wait for  <0x0006370da0c0> (a 
> com.google.common.util.concurrent.ListenableFutureTask)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:429)
> at java.util.concurrent.FutureTask.get(FutureTask.java:191)
> at 
> org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:383)
> at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.blockForWrites(CommitLogReplayer.java:207)
> at 
> org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:182)
>

[jira] [Updated] (CASSANDRA-13587) Deadlock during CommitLog replay when Cassandra restarts

2017-06-12 Thread Jaydeepkumar Chovatia (JIRA)

 [ 
https://issues.apache.org/jira/browse/CASSANDRA-13587?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jaydeepkumar Chovatia updated CASSANDRA-13587:
--
Summary: Deadlock during CommitLog replay when Cassandra restarts  (was: 
Deadlock during CommitLog replay when Cassandra restart)

> Deadlock during CommitLog replay when Cassandra restarts
> 
>
> Key: CASSANDRA-13587
> URL: https://issues.apache.org/jira/browse/CASSANDRA-13587
> Project: Cassandra
>  Issue Type: Bug
>  Components: Core
>Reporter: Jaydeepkumar Chovatia
>Assignee: Jaydeepkumar Chovatia
> Fix For: 3.0.14
>
> Attachments: 13587-3.0.txt, Reproduce_CASSANDRA-13587.txt
>
>
> Possible deadlock found when Cassandra is replaying commit log and at the 
> same time Mutation gets triggered by 
> SSTableReader(SystemKeyspace.persistSSTableReadMeter). As a result Cassandra 
> restart hangs forever
> Please find details of stack trace here:
> *Frame#1* This thread is trying to apply {{persistSSTableReadMeter}} mutation 
> and as a result it has called {{writeOrder.start()}} in {{Keyspace.java:533}}
> but there are no Commitlog Segments available because {{createReserveSegments 
> (CommitLogSegmentManager.java)}} is not yet {{true}} 
> Hence this thread is blocked on {{createReserveSegments}} to become {{true}}, 
> please note this thread has already started {{writeOrder}}
> {quote}
> "pool-11-thread-1" #251 prio=5 os_prio=0 tid=0x7fe128478400 nid=0x1b274 
> waiting on condition [0x7fe1389a]
>java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:304)
> at 
> org.apache.cassandra.utils.concurrent.WaitQueue$AbstractSignal.awaitUninterruptibly(WaitQueue.java:279)
> at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.advanceAllocatingFrom(CommitLogSegmentManager.java:277)
> at 
> org.apache.cassandra.db.commitlog.CommitLogSegmentManager.allocate(CommitLogSegmentManager.java:196)
> at org.apache.cassandra.db.commitlog.CommitLog.add(CommitLog.java:260)
> at org.apache.cassandra.db.Keyspace.applyInternal(Keyspace.java:540)
> at org.apache.cassandra.db.Keyspace.apply(Keyspace.java:421)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:210)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:215)
> at org.apache.cassandra.db.Mutation.apply(Mutation.java:224)
> at 
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternalWithoutCondition(ModificationStatement.java:566)
> at 
> org.apache.cassandra.cql3.statements.ModificationStatement.executeInternal(ModificationStatement.java:556)
> at 
> org.apache.cassandra.cql3.QueryProcessor.executeInternal(QueryProcessor.java:295)
> at 
> org.apache.cassandra.db.SystemKeyspace.persistSSTableReadMeter(SystemKeyspace.java:1181)
> at 
> org.apache.cassandra.io.sstable.format.SSTableReader$GlobalTidy$1.run(SSTableReader.java:2202)
> at 
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.runAndReset(FutureTask.java:308)
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.access$301(ScheduledThreadPoolExecutor.java:180)
> at 
> java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:294)
> at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
> at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
> at java.lang.Thread.run(Thread.java:745)
> {quote}
> *Frame#2* This thread is trying to recover commit logs and as a result it 
> tries to flush Memtable by calling following code:
> {{futures.add(Keyspace.open(SystemKeyspace.NAME).getColumnFamilyStore(SystemKeyspace.BATCHES).forceFlush());}}
> As a result Frame#3 (below) gets created
> {quote}
> "main" #1 prio=5 os_prio=0 tid=0x7fe1c64ec400 nid=0x1af29 waiting on 
> condition [0x7fe1c94a1000]
>java.lang.Thread.State: WAITING (parking)
> at sun.misc.Unsafe.park(Native Method)
> parking to wait for  <0x0006370da0c0> (a 
> com.google.common.util.concurrent.ListenableFutureTask)
> at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
> at java.util.concurrent.FutureTask.awaitDone(FutureTask.java:429)
> at java.util.concurrent.FutureTask.get(FutureTask.java:191)
> at 
> org.apache.cassandra.utils.FBUtilities.waitOnFutures(FBUtilities.java:383)
> at 
> org.apache.cassandra.db.commitlog.CommitLogReplayer.blockForWrites(CommitLogReplayer.java:207)
>