Details
-
Bug
-
Status: Closed
-
Major
-
Resolution: Fixed
-
None
-
None
Description
An integration test running transactions in client caches while servers are killed and restarted one by one encountered a hang when the server currrently serving as the distributed-lock "grantor" was killed and another server attempted to take on the role.
The stack traces below are from a build prior to package renaming.
TX commit thread:
vm_0_bridge1_w1-gst-dev23_17330:ServerConnection on port 21566 Thread 352 ID=861 state=TIMED_WAITING waiting to lock <java.util.concurrent.CountDownLatch$Sync@41c71f7d> at sun.misc.Unsafe.park(Native Method) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1033) at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1326) at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:282) at com.gemstone.gemfire.internal.util.concurrent.StoppableCountDownLatch.await(StoppableCountDownLatch.java:55) at com.gemstone.gemfire.internal.util.concurrent.FutureResult.get(FutureResult.java:54) at com.gemstone.gemfire.distributed.internal.locks.DLockService.waitForLockGrantorFutureResult(DLockService.java:774) at com.gemstone.gemfire.distributed.internal.locks.DLockService.notLockGrantorId(DLockService.java:837) at com.gemstone.gemfire.distributed.internal.locks.DLockService.releaseTryLocks(DLockService.java:2216) at com.gemstone.gemfire.internal.cache.locks.TXLockServiceImpl.release(TXLockServiceImpl.java:222) locked <java.util.ArrayList@34bbf1cf> at com.gemstone.gemfire.internal.cache.TXLockRequest.releaseDistributed(TXLockRequest.java:91) at com.gemstone.gemfire.internal.cache.TXLockRequest.cleanup(TXLockRequest.java:120) at com.gemstone.gemfire.internal.cache.TXState.cleanup(TXState.java:730) at com.gemstone.gemfire.internal.cache.TXState.commit(TXState.java:447) at com.gemstone.gemfire.internal.cache.TXStateProxyImpl.commit(TXStateProxyImpl.java:234) at com.gemstone.gemfire.internal.cache.TXManagerImpl.commit(TXManagerImpl.java:325) at com.gemstone.gemfire.internal.cache.tier.sockets.command.CommitCommand.cmdExecute(CommitCommand.java:80) at com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand.execute(BaseCommand.java:177) at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.doNormalMsg(ServerConnection.java:799) at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.doOneMessage(ServerConnection.java:930) at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.run(ServerConnection.java:1179) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl$1$1.run(AcceptorImpl.java:555) at java.lang.Thread.run(Thread.java:745) Locked synchronizers: java.util.concurrent.ThreadPoolExecutor$Worker@549109017 java.util.concurrent.locks.ReentrantLock$NonfairSync@317942158
Grantor trying to recover:
"ServerConnection on port 21566 Thread 356" daemon prio=10 tid=0x00007f9cf8020800 nid=0x5c9e waiting on condition [0x00007f9c698d6000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000000f4f2e4d0> (a java.util.concurrent.CountDownLatch$Sync) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(AbstractQueuedSynchronizer.java:1033) at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(AbstractQueuedSynchronizer.java:1326) at java.util.concurrent.CountDownLatch.await(CountDownLatch.java:282) at com.gemstone.gemfire.internal.util.concurrent.StoppableCountDownLatch.await(StoppableCountDownLatch.java:55) at com.gemstone.gemfire.distributed.internal.ReplyProcessor21.basicWait(ReplyProcessor21.java:743) at com.gemstone.gemfire.distributed.internal.ReplyProcessor21.waitForRepliesUninterruptibly(ReplyProcessor21.java:819) at com.gemstone.gemfire.distributed.internal.ReplyProcessor21.waitForRepliesUninterruptibly(ReplyProcessor21.java:796) at com.gemstone.gemfire.distributed.internal.ReplyProcessor21.waitForRepliesUninterruptibly(ReplyProcessor21.java:886) at com.gemstone.gemfire.distributed.internal.locks.DLockRecoverGrantorProcessor.recoverLockGrantor(DLockRecoverGrantorProcessor.java:96) at com.gemstone.gemfire.distributed.internal.locks.DLockService.makeLocalGrantor(DLockService.java:457) at com.gemstone.gemfire.distributed.internal.locks.DLockService.createLocalGrantor(DLockService.java:408) at com.gemstone.gemfire.distributed.internal.locks.DLockService.getLockGrantorId(DLockService.java:347) at com.gemstone.gemfire.distributed.internal.locks.DLockService.acquireTryLocks(DLockService.java:2278) at com.gemstone.gemfire.internal.cache.locks.TXLockServiceImpl.txLock(TXLockServiceImpl.java:132) at com.gemstone.gemfire.internal.cache.TXLockRequest.obtain(TXLockRequest.java:72) at com.gemstone.gemfire.internal.cache.TXState.reserveAndCheck(TXState.java:287) at com.gemstone.gemfire.internal.cache.TXState.commit(TXState.java:346) at com.gemstone.gemfire.internal.cache.TXStateProxyImpl.commit(TXStateProxyImpl.java:234) at com.gemstone.gemfire.internal.cache.TXManagerImpl.commit(TXManagerImpl.java:325) at com.gemstone.gemfire.internal.cache.tier.sockets.command.CommitCommand.cmdExecute(CommitCommand.java:80) at com.gemstone.gemfire.internal.cache.tier.sockets.BaseCommand.execute(BaseCommand.java:177) at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.doNormalMsg(ServerConnection.java:799) at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.doOneMessage(ServerConnection.java:930) at com.gemstone.gemfire.internal.cache.tier.sockets.ServerConnection.run(ServerConnection.java:1179) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at com.gemstone.gemfire.internal.cache.tier.sockets.AcceptorImpl$1$1.run(AcceptorImpl.java:555) at java.lang.Thread.run(Thread.java:745)
And the message-processing thread that is gathering lock service state for the new grantor but is blocked:
"Pooled Waiting Message Processor 16" daemon prio=10 tid=0x00007f9c9c01d800 nid=0x618c waiting on condition [0x00007f9c6a1e0000] java.lang.Thread.State: TIMED_WAITING (parking) at sun.misc.Unsafe.park(Native Method) - parking to wait for <0x00000000f0d906c0> (a java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync) at java.util.concurrent.locks.LockSupport.parkNanos(LockSupport.java:226) at java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireNanos(AbstractQueuedSynchronizer.java:929) at java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireNanos(AbstractQueuedSynchronizer.java:1245) at java.util.concurrent.locks.ReentrantReadWriteLock$WriteLock.tryLock(ReentrantReadWriteLock.java:1115) at com.gemstone.gemfire.internal.util.concurrent.StoppableReentrantReadWriteLock$StoppableWriteLock.lockInterruptibly(StoppableReentrantReadWriteLock.java:198) at com.gemstone.gemfire.internal.cache.locks.TXLockServiceImpl.acquireRecoveryWriteLock(TXLockServiceImpl.java:240) at com.gemstone.gemfire.internal.cache.locks.TXRecoverGrantorMessageProcessor.processDLockRecoverGrantorMessage(TXRecoverGrantorMessageProcessor.java:79) at com.gemstone.gemfire.internal.cache.locks.TXRecoverGrantorMessageProcessor$1.run(TXRecoverGrantorMessageProcessor.java:44) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615) at com.gemstone.gemfire.distributed.internal.DistributionManager.runUntilShutdown(DistributionManager.java:692) at com.gemstone.gemfire.distributed.internal.DistributionManager$6$1.run(DistributionManager.java:1029) at java.lang.Thread.run(Thread.java:745)
There were no other threads in the distributed system blocked in TXLockServiceImpl.acquireRecoveryWriteLock so in this case it was a simple deadlock involving three threads in a single server.
A fix has been posted for review: https://reviews.apache.org/r/53150/