Uploaded image for project: 'Geode'
  1. Geode
  2. GEODE-9269

Deadlock in dlock caused by lock ordering for TXLockService

    XMLWordPrintableJSON

Details

    Description

      In TXLockService implementation, some method gets the synchronized lock for batchLocks first before acquiring Grantor Destroy ReadLock. Another method does the opposite way. This could lead to potential dead lock when dlock is being destroyed.

      The stack showing the dead lock is that destroy dlock is trying to get the destroy write lock:

      "Distributed system shutdown hook" #29 prio=5 os_prio=0 tid=0x00007ff9d0067800 nid=0x619e waiting on condition [0x00007ff9eaded000]
         java.lang.Thread.State: TIMED_WAITING (parking)
              at sun.misc.Unsafe.park(Native Method)
              - parking to wait for  <0x00000000e1921070> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync)
              at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireNanos(AbstractQueuedSynchronizer.java:934)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireNanos(AbstractQueuedSynchronizer.java:1247)
              at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.tryLock(ReentrantReadWriteLock.java:1115)
              at org.apache.geode.internal.util.concurrent.StoppableReentrantReadWriteLock$StoppableWriteLock.tryLock(StoppableReentrantReadWriteLock.java:216)
              at org.apache.geode.distributed.internal.locks.DLockGrantor.acquireDestroyWriteLock(DLockGrantor.java:1429)
              at org.apache.geode.distributed.internal.locks.DLockGrantor.destroy(DLockGrantor.java:1217)
              - locked <0x00000000e1920bb8> (a org.apache.geode.distributed.internal.locks.DLockGrantor)
              at org.apache.geode.distributed.internal.locks.DLockService.nullLockGrantorId(DLockService.java:649)
              at org.apache.geode.distributed.internal.locks.DLockService.basicDestroy(DLockService.java:2336)
              at org.apache.geode.distributed.internal.locks.DLockService.destroyAndRemove(DLockService.java:2256)
              - locked <0x00000000e1413af0> (a java.lang.Object)
              at org.apache.geode.internal.cache.locks.TXLockServiceImpl.basicDestroy(TXLockServiceImpl.java:278)
              at org.apache.geode.internal.cache.locks.TXLockService.destroy(TXLockService.java:143)
              - locked <0x00000000e13a4f28> (a java.lang.Class for org.apache.geode.internal.cache.locks.TXLockService)
              at org.apache.geode.internal.cache.locks.TXLockService.destroyServices(TXLockService.java:65)
              - locked <0x00000000e13a4f28> (a java.lang.Class for org.apache.geode.internal.cache.locks.TXLockService)
              at org.apache.geode.internal.cache.GemFireCacheImpl.doClose(GemFireCacheImpl.java:2417)
              - locked <0x00000000e1160530> (a java.lang.Class for org.apache.geode.internal.cache.GemFireCacheImpl)
              at org.apache.geode.internal.cache.GemFireCacheImpl.close(GemFireCacheImpl.java:2151)
              at org.apache.geode.distributed.internal.InternalDistributedSystem.disconnect(InternalDistributedSystem.java:1559)
              - locked <0x00000000e1160530> (a java.lang.Class for org.apache.geode.internal.cache.GemFireCacheImpl)
              at org.apache.geode.distributed.internal.InternalDistributedSystem.lambda$static$7(InternalDistributedSystem.java:2203)
              at org.apache.geode.distributed.internal.InternalDistributedSystem$$Lambda$35/1697126778.run(Unknown Source)
              at java.lang.Thread.run(Thread.java:748)
      

      The read lock was held by the thread processing member depart event. It is blocked trying to get the synchronized lock of batchLocks.

      "Pooled Waiting Message Processor 1" #742 daemon prio=5 os_prio=0 tid=0x00007ff984035800 nid=0x4fad waiting for monitor entry [0x00007ff9f211a000]
         java.lang.Thread.State: BLOCKED (on object monitor)
              at org.apache.geode.distributed.internal.locks.DLockGrantor.getLockBatches(DLockGrantor.java:549)
              - waiting to lock <0x00000000e19210d8> (a java.util.HashMap)
              at org.apache.geode.internal.cache.locks.TXLessorDepartureHandler.handleDepartureOf(TXLessorDepartureHandler.java:73)
              at org.apache.geode.distributed.internal.locks.DLockGrantor.handleDepartureOf(DLockGrantor.java:1117)
              at org.apache.geode.distributed.internal.locks.DLockGrantor$1.processMemberDeparted(DLockGrantor.java:3625)
              at org.apache.geode.distributed.internal.locks.DLockGrantor$1$1.run(DLockGrantor.java:3597)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at org.apache.geode.distributed.internal.ClusterOperationExecutors.runUntilShutdown(ClusterOperationExecutors.java:446)
              at org.apache.geode.distributed.internal.ClusterOperationExecutors.doWaitingThread(ClusterOperationExecutors.java:415)
              at org.apache.geode.distributed.internal.ClusterOperationExecutors$$Lambda$92/156807545.invoke(Unknown Source)
              at org.apache.geode.logging.internal.executors.LoggingThreadFactory.lambda$newThread$0(LoggingThreadFactory.java:120)
              at org.apache.geode.logging.internal.executors.LoggingThreadFactory$$Lambda$88/1101735232.run(Unknown Source)
              at java.lang.Thread.run(Thread.java:748)
      

      The thread holds the synchronized lock is waiting for the grantor to be destroyed.

      "Pooled Waiting Message Processor 6" #1375 daemon prio=10 os_prio=0 tid=0x00007ff91c01f000 nid=0x5dae waiting on condition [0x00007ff9e8b06000]
         java.lang.Thread.State: TIMED_WAITING (parking)
              at sun.misc.Unsafe.park(Native Method)
              - parking to wait for  <0x00000000e1921530> (a java.util.concurrent.CountDownLatch$Sync)
              at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:215)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1037)
              at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1328)
              at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:277)
              at org.apache.geode.internal.util.concurrent.StoppableCountDownLatch.await(StoppableCountDownLatch.java:72)
              at org.apache.geode.distributed.internal.locks.DLockGrantor.waitUntilDestroyed(DLockGrantor.java:401)
              at org.apache.geode.distributed.internal.locks.DLockGrantor.releaseLockBatch(DLockGrantor.java:680)
              - locked <0x00000000e19210d8> (a java.util.HashMap)
              at org.apache.geode.distributed.internal.locks.DLockReleaseProcessor$DLockReleaseMessage.basicProcess(DLockReleaseProcessor.java:270)
              at org.apache.geode.distributed.internal.locks.DLockReleaseProcessor$DLockReleaseMessage$1.run(DLockReleaseProcessor.java:234)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at org.apache.geode.distributed.internal.ClusterOperationExecutors.runUntilShutdown(ClusterOperationExecutors.java:446)
              at org.apache.geode.distributed.internal.ClusterOperationExecutors.doWaitingThread(ClusterOperationExecutors.java:415)
              at org.apache.geode.distributed.internal.ClusterOperationExecutors$$Lambda$92/156807545.invoke(Unknown Source)
              at org.apache.geode.logging.internal.executors.LoggingThreadFactory.lambda$newThread$0(LoggingThreadFactory.java:120)
              at org.apache.geode.logging.internal.executors.LoggingThreadFactory$$Lambda$88/1101735232.run(Unknown Source)
              at java.lang.Thread.run(Thread.java:748)
      

      Attachments

        Activity

          People

            eshu Eric Shu
            eshu Eric Shu
            Votes:
            0 Vote for this issue
            Watchers:
            2 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: