Alberto, I'd have to see thread dumps on all the members on the site that has the stuck thread, but that sounds like you're hitting the limitation with conserve sockets and WAN. Are any of the stuck threads shared P2P message readers? If so, that is almost definitely a distributed deadlock. Setting conserve-sockets=false addresses that deadlock. If you post thread dumps, I'll take a look.
Barry ________________________________ From: Dave Barnes <dbar...@apache.org> Sent: Tuesday, July 6, 2021 4:05 PM To: dev@geode.apache.org <dev@geode.apache.org> Subject: Re: Questions about conserve-sockets and WAN replication Alberto, I recently updated some of the descriptions regarding conserve-sockets. Please check out this PR and see if it addresses any of your concerns. https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fgithub.com%2Fapache%2Fgeode%2Fpull%2F6516&data=04%7C01%7Cboglesby%40vmware.com%7C7c778f4148de4b7ddde508d940d28557%7Cb39138ca3cee4b4aa4d6cd83d9dd62f0%7C0%7C1%7C637612095204720548%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C3000&sdata=4TI7abW8wsWBXqQP8a74G5Gf2%2FR8Qf%2BhZDJZM4FK90k%3D&reserved=0 On Tue, Jul 6, 2021 at 9:57 AM Alberto Gomez <alberto.go...@est.tech> wrote: > Hi, > > The Geode documentation states the following about conserve-sockets and > WAN deployments in [1]: > > "WAN deployments increase the messaging demands on a Geode system. To > avoid hangs related to WAN messaging, always set `conserve-sockets=false` > for Geode members that participate in a WAN deployment." > > Could anyone please provide some more detailed information about why and > where these hangs could happen? Is this a hard limitation or something to > be considered under certain circumstances? > > We have run into an unexpected situation which we wonder if it is related > to the documentation statement above: > > In a system like the following: > - 2 WAN sites and 3 servers each > - several partitioned regions with parallel senders > - several replicated regions with serial senders > - conserve-sockets set to true > > We have sometimes observed, when trying to stop a parallel gateway sender > while puts are being sent to both sites, that the thread stopping the > gateway sender in one of the members gets stuck waiting to receive a reply > from the other members (trying to get the size of the queue, see [2]). We > see also other threads stuck, some trying to get a lock held by the stuck > thread and others waiting in > ReplyProcessor21.waitForRepliesUninterruptibly() trying to put or get data > remotely (See [3] and [4]). > If we set conserve-sockets to false we do not experience any hang. > > Could these stuck threads be related to what is stated in the > documentation about WAN deployments and conserve-sockets set to true or > should we rather think that it is an unrelated bug that needs to be solved? > > Thanks in advance for your help, > > Alberto > > [1] > https://nam04.safelinks.protection.outlook.com/?url=https%3A%2F%2Fgeode.apache.org%2Fdocs%2Fguide%2F113%2Fmanaging%2Fmonitor_tune%2Fsockets_and_gateways.html&data=04%7C01%7Cboglesby%40vmware.com%7C7c778f4148de4b7ddde508d940d28557%7Cb39138ca3cee4b4aa4d6cd83d9dd62f0%7C0%7C1%7C637612095204730504%7CUnknown%7CTWFpbGZsb3d8eyJWIjoiMC4wLjAwMDAiLCJQIjoiV2luMzIiLCJBTiI6Ik1haWwiLCJXVCI6Mn0%3D%7C3000&sdata=PomlekaoPDoy%2FIqqTOJSaUuQT0BT0VaiwAhBKCd25sY%3D&reserved=0 > > [2] > "ConcurrentParallelGatewaySenderEventProcessor Stopper Thread1" #1316 > daemon prio=10 os_prio=0 cpu=18.86ms elapsed=1544.80s > tid=0x00007f92bc1c2000 nid=0x2154 waiting on condition [0x00007f9179cd2000] > java.lang.Thread.State: TIMED_WAITING (parking) > at jdk.internal.misc.Unsafe.park(java.base@11.0.11/Native Method) > - parking to wait for <0x000000031ca2be50> (a > java.util.concurrent.CountDownLatch$Sync) > at > java.util.concurrent.locks.LockSupport.parkNanos(java.base@11.0.11 > /LockSupport.java:234) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(java.base@11.0.11 > /AbstractQueuedSynchronizer.java:1079) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(java.base@11.0.11 > /AbstractQueuedSynchronizer.java:1369) > at java.util.concurrent.CountDownLatch.await(java.base@11.0.11 > /CountDownLatch.java:278) > at > org.apache.geode.internal.util.concurrent.StoppableCountDownLatch.await(StoppableCountDownLatch.java:72) > at > org.apache.geode.distributed.internal.ReplyProcessor21.basicWait(ReplyProcessor21.java:731) > at > org.apache.geode.distributed.internal.ReplyProcessor21.waitForRepliesUninterruptibly(ReplyProcessor21.java:802) > at > org.apache.geode.distributed.internal.ReplyProcessor21.waitForRepliesUninterruptibly(ReplyProcessor21.java:779) > at > org.apache.geode.distributed.internal.ReplyProcessor21.waitForRepliesUninterruptibly(ReplyProcessor21.java:865) > at > org.apache.geode.internal.cache.partitioned.SizeMessage$SizeResponse.waitBucketSizes(SizeMessage.java:344) > at > org.apache.geode.internal.cache.PartitionedRegion.getSizeRemotely(PartitionedRegion.java:6758) > at > org.apache.geode.internal.cache.PartitionedRegion.entryCount(PartitionedRegion.java:6709) > at > org.apache.geode.internal.cache.PartitionedRegion.entryCount(PartitionedRegion.java:6691) > at > org.apache.geode.internal.cache.PartitionedRegion.getRegionSize(PartitionedRegion.java:6663) > at > org.apache.geode.internal.cache.LocalRegionDataView.entryCount(LocalRegionDataView.java:99) > at > org.apache.geode.internal.cache.LocalRegion.entryCount(LocalRegion.java:2078) > at > org.apache.geode.internal.cache.LocalRegion.size(LocalRegion.java:8301) > at > org.apache.geode.internal.cache.wan.parallel.ParallelGatewaySenderQueue.size(ParallelGatewaySenderQueue.java:1670) > at > org.apache.geode.internal.cache.wan.AbstractGatewaySenderEventProcessor.closeProcessor(AbstractGatewaySenderEventProcessor.java:1259) > at > org.apache.geode.internal.cache.wan.AbstractGatewaySenderEventProcessor.stopProcessing(AbstractGatewaySenderEventProcessor.java:1247) > at > org.apache.geode.internal.cache.wan.AbstractGatewaySenderEventProcessor$SenderStopperCallable.call(AbstractGatewaySenderEventProcessor.java:1399) > at > org.apache.geode.internal.cache.wan.AbstractGatewaySenderEventProcessor$SenderStopperCallable.call(AbstractGatewaySenderEventProcessor.java:1387) > at java.util.concurrent.FutureTask.run(java.base@11.0.11 > /FutureTask.java:264) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.11 > /ThreadPoolExecutor.java:1128) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.11 > /ThreadPoolExecutor.java:628) > at java.lang.Thread.run(java.base@11.0.11/Thread.java:829) > > [3] > "P2P message reader for 192.168.78.164(server-0:1)<v31>:41000 shared > ordered uid=6 local port=60360 remote port=57246" #56 daemon prio=10 > os_prio=0 cpu=462104.83ms elapsed=4487.06s tid=0x00007f93a8007800 nid=0x50 > waiting on condition [0x00007f93e59d0000] > java.lang.Thread.State: WAITING (parking) > at jdk.internal.misc.Unsafe.park(java.base@11.0.11/Native Method) > - parking to wait for <0x00000000ed9cb9f0> (a > java.util.concurrent.locks.ReentrantReadWriteLock$NonfairSync) > at java.util.concurrent.locks.LockSupport.park(java.base@11.0.11 > /LockSupport.java:194) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.parkAndCheckInterrupt(java.base@11.0.11 > /AbstractQueuedSynchronizer.java:885) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireShared(java.base@11.0.11 > /AbstractQueuedSynchronizer.java:1009) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireShared(java.base@11.0.11 > /AbstractQueuedSynchronizer.java:1324) > at > java.util.concurrent.locks.ReentrantReadWriteLock$ReadLock.lock(java.base@11.0.11 > /ReentrantReadWriteLock.java:738) > at > org.apache.geode.internal.cache.wan.AbstractGatewaySender.distribute(AbstractGatewaySender.java:1104) > at > org.apache.geode.internal.cache.LocalRegion.notifyGatewaySender(LocalRegion.java:6144) > at > org.apache.geode.internal.cache.LocalRegion.notifyGatewaySender(LocalRegion.java:6108) > at > org.apache.geode.internal.cache.BucketRegion.notifyGatewaySender(BucketRegion.java:719) > at > org.apache.geode.internal.cache.LocalRegion.basicPutPart2(LocalRegion.java:5775) > at > org.apache.geode.internal.cache.BucketRegion.basicPutPart2(BucketRegion.java:704) > at > org.apache.geode.internal.cache.map.RegionMapPut.doBeforeCompletionActions(RegionMapPut.java:282) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.doPutAndDeliverEvent(AbstractRegionMapPut.java:301) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut$$Lambda$515/0x00000008006e0440.run(Unknown > Source) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.runWithIndexUpdatingInProgress(AbstractRegionMapPut.java:308) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.doPutIfPreconditionsSatisified(AbstractRegionMapPut.java:296) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.doPutOnSynchronizedRegionEntry(AbstractRegionMapPut.java:282) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.doPutOnRegionEntryInMap(AbstractRegionMapPut.java:273) > - locked <0x0000000136123330> (a > org.apache.geode.internal.cache.entries.VersionedThinDiskRegionEntryOffHeapObjectKey) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.addRegionEntryToMapAndDoPut(AbstractRegionMapPut.java:251) > - locked <0x0000000136123330> (a > org.apache.geode.internal.cache.entries.VersionedThinDiskRegionEntryOffHeapObjectKey) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.doPutRetryingIfNeeded(AbstractRegionMapPut.java:216) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut$$Lambda$514/0x00000008006e0040.run(Unknown > Source) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.doWithIndexInUpdateMode(AbstractRegionMapPut.java:198) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.doPut(AbstractRegionMapPut.java:180) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut$$Lambda$513/0x00000008006ca440.run(Unknown > Source) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.runWhileLockedForCacheModification(AbstractRegionMapPut.java:119) > at > org.apache.geode.internal.cache.map.RegionMapPut.runWhileLockedForCacheModification(RegionMapPut.java:161) > at > org.apache.geode.internal.cache.map.AbstractRegionMapPut.put(AbstractRegionMapPut.java:169) > at > org.apache.geode.internal.cache.AbstractRegionMap.basicPut(AbstractRegionMap.java:2033) > at > org.apache.geode.internal.cache.BucketRegion.virtualPut(BucketRegion.java:530) > at > org.apache.geode.internal.cache.LocalRegionDataView.putEntry(LocalRegionDataView.java:170) > at > org.apache.geode.internal.cache.LocalRegion.basicUpdate(LocalRegion.java:5571) > at > org.apache.geode.internal.cache.AbstractUpdateOperation.doPutOrCreate(AbstractUpdateOperation.java:194) > at > org.apache.geode.internal.cache.AbstractUpdateOperation$AbstractUpdateMessage.basicOperateOnRegion(AbstractUpdateOperation.java:307) > at > org.apache.geode.internal.cache.AbstractUpdateOperation$AbstractUpdateMessage.operateOnRegion(AbstractUpdateOperation.java:278) > at > org.apache.geode.internal.cache.DistributedCacheOperation$CacheOperationMessage.basicProcess(DistributedCacheOperation.java:1208) > at > org.apache.geode.internal.cache.DistributedCacheOperation$CacheOperationMessage.process(DistributedCacheOperation.java:1110) > at > org.apache.geode.distributed.internal.DistributionMessage.scheduleAction(DistributionMessage.java:376) > at > org.apache.geode.distributed.internal.DistributionMessage.schedule(DistributionMessage.java:432) > at > org.apache.geode.distributed.internal.ClusterDistributionManager.scheduleIncomingMessage(ClusterDistributionManager.java:2060) > at > org.apache.geode.distributed.internal.ClusterDistributionManager.handleIncomingDMsg(ClusterDistributionManager.java:1826) > at > org.apache.geode.distributed.internal.ClusterDistributionManager$$Lambda$178/0x0000000800380440.messageReceived(Unknown > Source) > at > org.apache.geode.distributed.internal.membership.gms.GMSMembership.dispatchMessage(GMSMembership.java:936) > at > org.apache.geode.distributed.internal.membership.gms.GMSMembership.handleOrDeferMessage(GMSMembership.java:867) > at > org.apache.geode.distributed.internal.membership.gms.GMSMembership.processMessage(GMSMembership.java:1209) > at > org.apache.geode.distributed.internal.DistributionImpl$MyDCReceiver.messageReceived(DistributionImpl.java:828) > at > org.apache.geode.distributed.internal.direct.DirectChannel.receive(DirectChannel.java:614) > at > org.apache.geode.internal.tcp.TCPConduit.messageReceived(TCPConduit.java:679) > at > org.apache.geode.internal.tcp.Connection.dispatchMessage(Connection.java:3261) > at > org.apache.geode.internal.tcp.Connection.readMessage(Connection.java:2988) > at > org.apache.geode.internal.tcp.Connection.processInputBuffer(Connection.java:2794) > at > org.apache.geode.internal.tcp.Connection.readMessages(Connection.java:1648) > at > org.apache.geode.internal.tcp.Connection.run(Connection.java:1479) > at java.lang.Thread.run(java.base@11.0.11/Thread.java:829) > > > [4] > "ServerConnection on port 40404 Thread 2" #89 daemon prio=5 os_prio=0 > cpu=79476.76ms elapsed=4485.21s tid=0x00007f92e0001800 nid=0x73 waiting on > condition [0x00007f931ebd6000] > java.lang.Thread.State: TIMED_WAITING (parking) > at jdk.internal.misc.Unsafe.park(java.base@11.0.11/Native Method) > - parking to wait for <0x000000031ca28640> (a > java.util.concurrent.CountDownLatch$Sync) > at > java.util.concurrent.locks.LockSupport.parkNanos(java.base@11.0.11 > /LockSupport.java:234) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.doAcquireSharedNanos(java.base@11.0.11 > /AbstractQueuedSynchronizer.java:1079) > at > java.util.concurrent.locks.AbstractQueuedSynchronizer.tryAcquireSharedNanos(java.base@11.0.11 > /AbstractQueuedSynchronizer.java:1369) > at java.util.concurrent.CountDownLatch.await(java.base@11.0.11 > /CountDownLatch.java:278) > at > org.apache.geode.internal.util.concurrent.StoppableCountDownLatch.await(StoppableCountDownLatch.java:72) > at > org.apache.geode.distributed.internal.ReplyProcessor21.basicWait(ReplyProcessor21.java:731) > at > org.apache.geode.distributed.internal.ReplyProcessor21.waitForRepliesUninterruptibly(ReplyProcessor21.java:802) > at > org.apache.geode.distributed.internal.ReplyProcessor21.waitForRepliesUninterruptibly(ReplyProcessor21.java:779) > at > org.apache.geode.distributed.internal.ReplyProcessor21.waitForRepliesUninterruptibly(ReplyProcessor21.java:865) > at > org.apache.geode.internal.cache.partitioned.PartitionMessage$PartitionResponse.waitForCacheException(PartitionMessage.java:808) > at > org.apache.geode.internal.cache.partitioned.PutMessage$PutResponse.waitForResult(PutMessage.java:1037) > at > org.apache.geode.internal.cache.PartitionedRegion.putRemotely(PartitionedRegion.java:3389) > at > org.apache.geode.internal.cache.PartitionedRegion.putInBucket(PartitionedRegion.java:3020) > at > org.apache.geode.internal.cache.PartitionedRegion.virtualPut(PartitionedRegion.java:2215) > at > org.apache.geode.internal.cache.LocalRegionDataView.putEntry(LocalRegionDataView.java:170) > at > org.apache.geode.internal.cache.LocalRegion.basicUpdate(LocalRegion.java:5571) > at > org.apache.geode.internal.cache.LocalRegion.basicUpdate(LocalRegion.java:5531) > at > org.apache.geode.internal.cache.LocalRegion.basicBridgePut(LocalRegion.java:5210) > at > org.apache.geode.internal.cache.tier.sockets.command.Put65.cmdExecute(Put65.java:411) > at > org.apache.geode.internal.cache.tier.sockets.BaseCommand.execute(BaseCommand.java:183) > at > org.apache.geode.internal.cache.tier.sockets.ServerConnection.doNormalMessage(ServerConnection.java:848) > at > org.apache.geode.internal.cache.tier.sockets.OriginalServerConnection.doOneMessage(OriginalServerConnection.java:72) > at > org.apache.geode.internal.cache.tier.sockets.ServerConnection.run(ServerConnection.java:1181) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(java.base@11.0.11 > /ThreadPoolExecutor.java:1128) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(java.base@11.0.11 > /ThreadPoolExecutor.java:628) > at > org.apache.geode.internal.cache.tier.sockets.AcceptorImpl.lambda$initializeServerConnectionThreadPool$3(AcceptorImpl.java:691) > at > org.apache.geode.internal.cache.tier.sockets.AcceptorImpl$$Lambda$496/0x00000008006c8040.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$166/0x000000080034c040.run(Unknown > Source) > at java.lang.Thread.run(java.base@11.0.11/Thread.java:829) >