Uploaded image for project: 'Cassandra'
  1. Cassandra
  2. CASSANDRA-13587

Deadlock during CommitLog replay when Cassandra restarts

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Normal
    • Resolution: Fixed
    • 3.0.15
    • Legacy/Core
    • None
    • Normal

    Description

      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

      "pool-11-thread-1" #251 prio=5 os_prio=0 tid=0x00007fe128478400 nid=0x1b274 waiting on condition [0x00007fe1389a0000]
      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)

      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

      "main" #1 prio=5 os_prio=0 tid=0x00007fe1c64ec400 nid=0x1af29 waiting on condition [0x00007fe1c94a1000]
      java.lang.Thread.State: WAITING (parking)
      at sun.misc.Unsafe.park(Native Method)
      parking to wait for <0x00000006370da0c0> (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 org.apache.cassandra.db.commitlog.CommitLog.recover(CommitLog.java:161)
      at org.apache.cassandra.service.CassandraDaemon.setup(CassandraDaemon.java:295)
      at org.apache.cassandra.service.CassandraDaemon.activate(CassandraDaemon.java:569)
      at org.apache.cassandra.service.CassandraDaemon.main(CassandraDaemon.java:697)

      Frame#3 This thread is waiting at writeBarrier.await(); in ColumnFamilyStore.java:1027
      but writeBarrier is locked by thread in Frame#1, and Frame#1 thread is waiting for more CommitlogSegements to be available.
      Frame#1 thread will not get new segment because variable createReserveSegments(CommitLogSegmentManager.java) is not yet true.
      This variable gets set to true after successful execution of Frame#2.

      Here we can see Frame#3 and Frame#1 are in deadlock state and Cassandra restart hangs forever.

      "MemtableFlushWriter:5" #433 daemon prio=5 os_prio=0 tid=0x00007e7a4b8b0400 nid=0x1dea8 waiting on condition [0x00007e753c2ca000]
      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.utils.concurrent.OpOrder$Barrier.await(OpOrder.java:419)
      at org.apache.cassandra.db.ColumnFamilyStore$Flush.run(ColumnFamilyStore.java:1027)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
      at org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$4/1527007086.run(Unknown Source)
      at java.lang.Thread.run(Thread.java:745)

      "MemtablePostFlush:3" #432 daemon prio=5 os_prio=0 tid=0x00007e7a4b8b0000 nid=0x1dea7 waiting on condition [0x00007e753c30b000]
      java.lang.Thread.State: WAITING (parking)
      at sun.misc.Unsafe.park(Native Method)
      parking to wait for <0x00000006370d9cd0> (a java.util.concurrent.CountDownLatch$Sync)
      at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
      at java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(AbstractQueuedSynchronizer.java:836)
      at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedInterruptibly(AbstractQueuedSynchronizer.java:997)
      at java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireSharedInterruptibly(AbstractQueuedSynchronizer.java:1304)
      at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:231)
      at org.apache.cassandra.db.ColumnFamilyStore$PostFlush.call(ColumnFamilyStore.java:941)
      at org.apache.cassandra.db.ColumnFamilyStore$PostFlush.call(ColumnFamilyStore.java:924)
      at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
      at java.util.concurrent.FutureTask.run(FutureTask.java:266)
      at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
      at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
      at org.apache.cassandra.concurrent.NamedThreadFactory.lambda$threadLocalDeallocator$0(NamedThreadFactory.java:79)
      at org.apache.cassandra.concurrent.NamedThreadFactory$$Lambda$4/1527007086.run(Unknown Source)
      at java.lang.Thread.run(Thread.java:745)

      Reproducible steps: Reproducing this problem is tricky as it involves multiple conditions to happen at the same time and is timing bases, so I have done some small code change to reproduce this:
      1. Create a Keyspace and table
      2. Inject data until there are few SSTables generated and CommitLog available
      3. Kill Cassandra process
      4. Use the custom code (in the attached file "Reproduce_CASSANDRA-13587.txt") on top of 3.0.14 branch
      5. Build Cassandra jar and use this custom jar
      6. Restart Cassandra
      Here you will see Cassandra is hanging forever
      7. Now apply this fix on top of "Reproduce_CASSANDRA-13587.txt", and repeat step-6
      Here you should see Cassandra is starting normally

      Solution: I am proposing that we should enable variable createReserveSegments(CommitLogSegmentManager.java) before recovering any CommitLogs in CommitLog.java file
      so this will not block Frame#1 from acquiring new segment as a result Frame#1 will finish and then Frame#2 will also finish.
      Please note, this variable createReserveSegments has been removed from the trunk branch as part of (https://issues.apache.org/jira/browse/CASSANDRA-10202), also in the trunk branch CommitLog segments gets created when needed. So as per my understanding enabling this variable before CommitLog recovery should not create any other side effect, please let me know your comments.

      Attachments

        1. Reproduce_CASSANDRA-13587.txt
          4 kB
          Jaydeepkumar Chovatia
        2. 13587-3.0.txt
          1 kB
          Jaydeepkumar Chovatia

        Activity

          People

            chovatia.jaydeep@gmail.com Jaydeepkumar Chovatia
            chovatia.jaydeep@gmail.com Jaydeepkumar Chovatia
            Jaydeepkumar Chovatia
            Jason Brown
            Votes:
            0 Vote for this issue
            Watchers:
            7 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: