You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@cassandra.apache.org by "Yang Yang (JIRA)" <ji...@apache.org> on 2011/09/23 19:53:26 UTC

[jira] [Created] (CASSANDRA-3253) inherent deadlock situation in commitLog flush?

inherent deadlock situation in commitLog flush?
-----------------------------------------------

                 Key: CASSANDRA-3253
                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3253
             Project: Cassandra
          Issue Type: Bug
    Affects Versions: 1.0.0
            Reporter: Yang Yang


after my system ran for a while, it consitently goes into frozen state where all the mutations stage threads are waiting
on the switchlock,

the reason is that the switchlock is held by commit log, as shown by the following thread dump:



"COMMIT-LOG-WRITER" prio=10 tid=0x00000000010df000 nid=0x32d3 waiting on condition [0x00007f2d81557000]
   java.lang.Thread.State: WAITING (parking)
        at sun.misc.Unsafe.park(Native Method)
        - parking to wait for  <0x00007f3579eec060> (a java.util.concurrent.FutureTask$Sync)
        at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:838)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
        at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
        at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:248)
        at java.util.concurrent.FutureTask.get(FutureTask.java:111)
        at org.apache.cassandra.db.commitlog.CommitLog.getContext(CommitLog.java:386)
        at org.apache.cassandra.db.ColumnFamilyStore.maybeSwitchMemtable(ColumnFamilyStore.java:650)
        at org.apache.cassandra.db.ColumnFamilyStore.forceFlush(ColumnFamilyStore.java:722)
        at org.apache.cassandra.db.commitlog.CommitLog.createNewSegment(CommitLog.java:573)
        at org.apache.cassandra.db.commitlog.CommitLog.access$300(CommitLog.java:81)
        at org.apache.cassandra.db.commitlog.CommitLog$LogRecordAdder.run(CommitLog.java:596)
        at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:49)
        at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
        at java.lang.Thread.run(Thread.java:679)


we can clearly see that the COMMIT-LOG-WRITER thread is running the regular appender , but the appender itself calls getContext(), which again submits a new Callable to be executed, and waits on the Callable. but the new Callable is never going to be executed since the executor has only *one* thread.


I believe this is a deterministic bug.





--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-3253) inherent deadlock situation in commitLog flush?

Posted by "Yang Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3253?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13113621#comment-13113621 ] 

Yang Yang commented on CASSANDRA-3253:
--------------------------------------

it seems to be added in the recent feature to flush earliest segment with dirty CFs : 


https://github.com/apache/cassandra/commit/f599559221ad074d9af0a99d7ffdd482c2b6b10c#diff-3

CFS.forceFlush() was added to the commit log writing path

> inherent deadlock situation in commitLog flush?
> -----------------------------------------------
>
>                 Key: CASSANDRA-3253
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3253
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.0
>            Reporter: Yang Yang
>
> after my system ran for a while, it consitently goes into frozen state where all the mutations stage threads are waiting
> on the switchlock,
> the reason is that the switchlock is held by commit log, as shown by the following thread dump:
> "COMMIT-LOG-WRITER" prio=10 tid=0x00000000010df000 nid=0x32d3 waiting on condition [0x00007f2d81557000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00007f3579eec060> (a java.util.concurrent.FutureTask$Sync)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:838)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>         at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:248)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:111)
>         at org.apache.cassandra.db.commitlog.CommitLog.getContext(CommitLog.java:386)
>         at org.apache.cassandra.db.ColumnFamilyStore.maybeSwitchMemtable(ColumnFamilyStore.java:650)
>         at org.apache.cassandra.db.ColumnFamilyStore.forceFlush(ColumnFamilyStore.java:722)
>         at org.apache.cassandra.db.commitlog.CommitLog.createNewSegment(CommitLog.java:573)
>         at org.apache.cassandra.db.commitlog.CommitLog.access$300(CommitLog.java:81)
>         at org.apache.cassandra.db.commitlog.CommitLog$LogRecordAdder.run(CommitLog.java:596)
>         at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:49)
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         at java.lang.Thread.run(Thread.java:679)
> we can clearly see that the COMMIT-LOG-WRITER thread is running the regular appender , but the appender itself calls getContext(), which again submits a new Callable to be executed, and waits on the Callable. but the new Callable is never going to be executed since the executor has only *one* thread.
> I believe this is a deterministic bug.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-3253) inherent deadlock situation in commitLog flush?

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3253?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-3253:
--------------------------------------

    Attachment: 3253.txt

excellent diagnosis of the problem, Yang.

patch attached to push the flush calls off of the CL executor.

> inherent deadlock situation in commitLog flush?
> -----------------------------------------------
>
>                 Key: CASSANDRA-3253
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3253
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.0
>            Reporter: Yang Yang
>         Attachments: 3253.txt
>
>
> after my system ran for a while, it consitently goes into frozen state where all the mutations stage threads are waiting
> on the switchlock,
> the reason is that the switchlock is held by commit log, as shown by the following thread dump:
> "COMMIT-LOG-WRITER" prio=10 tid=0x00000000010df000 nid=0x32d3 waiting on condition [0x00007f2d81557000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00007f3579eec060> (a java.util.concurrent.FutureTask$Sync)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:838)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>         at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:248)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:111)
>         at org.apache.cassandra.db.commitlog.CommitLog.getContext(CommitLog.java:386)
>         at org.apache.cassandra.db.ColumnFamilyStore.maybeSwitchMemtable(ColumnFamilyStore.java:650)
>         at org.apache.cassandra.db.ColumnFamilyStore.forceFlush(ColumnFamilyStore.java:722)
>         at org.apache.cassandra.db.commitlog.CommitLog.createNewSegment(CommitLog.java:573)
>         at org.apache.cassandra.db.commitlog.CommitLog.access$300(CommitLog.java:81)
>         at org.apache.cassandra.db.commitlog.CommitLog$LogRecordAdder.run(CommitLog.java:596)
>         at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:49)
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         at java.lang.Thread.run(Thread.java:679)
> we can clearly see that the COMMIT-LOG-WRITER thread is running the regular appender , but the appender itself calls getContext(), which again submits a new Callable to be executed, and waits on the Callable. but the new Callable is never going to be executed since the executor has only *one* thread.
> I believe this is a deterministic bug.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Updated] (CASSANDRA-3253) inherent deadlock situation in commitLog flush?

Posted by "Jonathan Ellis (JIRA)" <ji...@apache.org>.
     [ https://issues.apache.org/jira/browse/CASSANDRA-3253?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Jonathan Ellis updated CASSANDRA-3253:
--------------------------------------

    Component/s: Core
       Priority: Critical  (was: Major)

> inherent deadlock situation in commitLog flush?
> -----------------------------------------------
>
>                 Key: CASSANDRA-3253
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3253
>             Project: Cassandra
>          Issue Type: Bug
>          Components: Core
>    Affects Versions: 1.0.0
>            Reporter: Yang Yang
>            Assignee: Jonathan Ellis
>            Priority: Critical
>              Labels: commitlog
>             Fix For: 1.0.0
>
>         Attachments: 3253.txt
>
>
> after my system ran for a while, it consitently goes into frozen state where all the mutations stage threads are waiting
> on the switchlock,
> the reason is that the switchlock is held by commit log, as shown by the following thread dump:
> "COMMIT-LOG-WRITER" prio=10 tid=0x00000000010df000 nid=0x32d3 waiting on condition [0x00007f2d81557000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00007f3579eec060> (a java.util.concurrent.FutureTask$Sync)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:838)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>         at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:248)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:111)
>         at org.apache.cassandra.db.commitlog.CommitLog.getContext(CommitLog.java:386)
>         at org.apache.cassandra.db.ColumnFamilyStore.maybeSwitchMemtable(ColumnFamilyStore.java:650)
>         at org.apache.cassandra.db.ColumnFamilyStore.forceFlush(ColumnFamilyStore.java:722)
>         at org.apache.cassandra.db.commitlog.CommitLog.createNewSegment(CommitLog.java:573)
>         at org.apache.cassandra.db.commitlog.CommitLog.access$300(CommitLog.java:81)
>         at org.apache.cassandra.db.commitlog.CommitLog$LogRecordAdder.run(CommitLog.java:596)
>         at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:49)
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         at java.lang.Thread.run(Thread.java:679)
> we can clearly see that the COMMIT-LOG-WRITER thread is running the regular appender , but the appender itself calls getContext(), which again submits a new Callable to be executed, and waits on the Callable. but the new Callable is never going to be executed since the executor has only *one* thread.
> I believe this is a deterministic bug.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Commented] (CASSANDRA-3253) inherent deadlock situation in commitLog flush?

Posted by "Yang Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3253?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13113655#comment-13113655 ] 

Yang Yang commented on CASSANDRA-3253:
--------------------------------------

looks to be the same of https://issues.apache.org/jira/browse/CASSANDRA-1991

> inherent deadlock situation in commitLog flush?
> -----------------------------------------------
>
>                 Key: CASSANDRA-3253
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3253
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.0
>            Reporter: Yang Yang
>
> after my system ran for a while, it consitently goes into frozen state where all the mutations stage threads are waiting
> on the switchlock,
> the reason is that the switchlock is held by commit log, as shown by the following thread dump:
> "COMMIT-LOG-WRITER" prio=10 tid=0x00000000010df000 nid=0x32d3 waiting on condition [0x00007f2d81557000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00007f3579eec060> (a java.util.concurrent.FutureTask$Sync)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:838)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>         at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:248)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:111)
>         at org.apache.cassandra.db.commitlog.CommitLog.getContext(CommitLog.java:386)
>         at org.apache.cassandra.db.ColumnFamilyStore.maybeSwitchMemtable(ColumnFamilyStore.java:650)
>         at org.apache.cassandra.db.ColumnFamilyStore.forceFlush(ColumnFamilyStore.java:722)
>         at org.apache.cassandra.db.commitlog.CommitLog.createNewSegment(CommitLog.java:573)
>         at org.apache.cassandra.db.commitlog.CommitLog.access$300(CommitLog.java:81)
>         at org.apache.cassandra.db.commitlog.CommitLog$LogRecordAdder.run(CommitLog.java:596)
>         at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:49)
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         at java.lang.Thread.run(Thread.java:679)
> we can clearly see that the COMMIT-LOG-WRITER thread is running the regular appender , but the appender itself calls getContext(), which again submits a new Callable to be executed, and waits on the Callable. but the new Callable is never going to be executed since the executor has only *one* thread.
> I believe this is a deterministic bug.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

[jira] [Issue Comment Edited] (CASSANDRA-3253) inherent deadlock situation in commitLog flush?

Posted by "Yang Yang (JIRA)" <ji...@apache.org>.
    [ https://issues.apache.org/jira/browse/CASSANDRA-3253?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13113655#comment-13113655 ] 

Yang Yang edited comment on CASSANDRA-3253 at 9/23/11 7:27 PM:
---------------------------------------------------------------

looks to be related to https://issues.apache.org/jira/browse/CASSANDRA-1991

      was (Author: yangyangyyy):
    looks to be the same of https://issues.apache.org/jira/browse/CASSANDRA-1991
  
> inherent deadlock situation in commitLog flush?
> -----------------------------------------------
>
>                 Key: CASSANDRA-3253
>                 URL: https://issues.apache.org/jira/browse/CASSANDRA-3253
>             Project: Cassandra
>          Issue Type: Bug
>    Affects Versions: 1.0.0
>            Reporter: Yang Yang
>
> after my system ran for a while, it consitently goes into frozen state where all the mutations stage threads are waiting
> on the switchlock,
> the reason is that the switchlock is held by commit log, as shown by the following thread dump:
> "COMMIT-LOG-WRITER" prio=10 tid=0x00000000010df000 nid=0x32d3 waiting on condition [0x00007f2d81557000]
>    java.lang.Thread.State: WAITING (parking)
>         at sun.misc.Unsafe.park(Native Method)
>         - parking to wait for  <0x00007f3579eec060> (a java.util.concurrent.FutureTask$Sync)
>         at java.util.concurrent.locks.LockSupport.park(LockSupport.java:186)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:838)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:998)
>         at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
>         at java.util.concurrent.FutureTask$Sync.innerGet(FutureTask.java:248)
>         at java.util.concurrent.FutureTask.get(FutureTask.java:111)
>         at org.apache.cassandra.db.commitlog.CommitLog.getContext(CommitLog.java:386)
>         at org.apache.cassandra.db.ColumnFamilyStore.maybeSwitchMemtable(ColumnFamilyStore.java:650)
>         at org.apache.cassandra.db.ColumnFamilyStore.forceFlush(ColumnFamilyStore.java:722)
>         at org.apache.cassandra.db.commitlog.CommitLog.createNewSegment(CommitLog.java:573)
>         at org.apache.cassandra.db.commitlog.CommitLog.access$300(CommitLog.java:81)
>         at org.apache.cassandra.db.commitlog.CommitLog$LogRecordAdder.run(CommitLog.java:596)
>         at org.apache.cassandra.db.commitlog.PeriodicCommitLogExecutorService$1.runMayThrow(PeriodicCommitLogExecutorService.java:49)
>         at org.apache.cassandra.utils.WrappedRunnable.run(WrappedRunnable.java:30)
>         at java.lang.Thread.run(Thread.java:679)
> we can clearly see that the COMMIT-LOG-WRITER thread is running the regular appender , but the appender itself calls getContext(), which again submits a new Callable to be executed, and waits on the Callable. but the new Callable is never going to be executed since the executor has only *one* thread.
> I believe this is a deterministic bug.

--
This message is automatically generated by JIRA.
For more information on JIRA, see: http://www.atlassian.com/software/jira