Cassandra
  1. Cassandra
  2. CASSANDRA-3253

inherent deadlock situation in commitLog flush?

    Details

    • Type: Bug Bug
    • Status: Resolved
    • Priority: Critical Critical
    • Resolution: Fixed
    • Fix Version/s: 1.0.0
    • Component/s: Core
    • Labels:

      Description

      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.

      1. 3253.txt
        2 kB
        Jonathan Ellis

        Activity

        Yang Yang created issue -
        Hide
        Yang Yang added a comment -

        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

        Show
        Yang Yang added a comment - 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
        Hide
        Yang Yang added a comment - - edited
        Show
        Yang Yang added a comment - - edited looks to be related to https://issues.apache.org/jira/browse/CASSANDRA-1991
        Hide
        Jonathan Ellis added a comment -

        excellent diagnosis of the problem, Yang.

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

        Show
        Jonathan Ellis added a comment - excellent diagnosis of the problem, Yang. patch attached to push the flush calls off of the CL executor.
        Jonathan Ellis made changes -
        Field Original Value New Value
        Attachment 3253.txt [ 12496376 ]
        Jonathan Ellis made changes -
        Status Open [ 1 ] Patch Available [ 10002 ]
        Labels commitlog
        Assignee Jonathan Ellis [ jbellis ]
        Reviewer yangyangyyy
        Fix Version/s 1.0.0 [ 12316349 ]
        Jonathan Ellis made changes -
        Priority Major [ 3 ] Critical [ 2 ]
        Component/s Core [ 12312978 ]
        Hide
        Sylvain Lebresne added a comment -

        +1
        Committed, thanks

        Show
        Sylvain Lebresne added a comment - +1 Committed, thanks
        Sylvain Lebresne made changes -
        Status Patch Available [ 10002 ] Resolved [ 5 ]
        Reviewer yangyangyyy slebresne
        Resolution Fixed [ 1 ]
        Gavin made changes -
        Workflow no-reopen-closed, patch-avail [ 12635036 ] patch-available, re-open possible [ 12753010 ]
        Gavin made changes -
        Workflow patch-available, re-open possible [ 12753010 ] reopen-resolved, no closed status, patch-avail, testing [ 12758584 ]
        Transition Time In Source Status Execution Times Last Executer Last Execution Date
        Open Open Patch Available Patch Available
        1d 7h 31m 1 Jonathan Ellis 25/Sep/11 02:25
        Patch Available Patch Available Resolved Resolved
        1d 7h 42m 1 Sylvain Lebresne 26/Sep/11 10:07

          People

          • Assignee:
            Jonathan Ellis
            Reporter:
            Yang Yang
            Reviewer:
            Sylvain Lebresne
          • Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

            • Created:
              Updated:
              Resolved:

              Development