You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@geode.apache.org by "Barry Oglesby (JIRA)" <ji...@apache.org> on 2017/10/06 20:55:00 UTC

[jira] [Commented] (GEODE-3709) Geode Version: 1.1.1 In one of the project we a...

    [ https://issues.apache.org/jira/browse/GEODE-3709?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16195230#comment-16195230 ] 

Barry Oglesby commented on GEODE-3709:
--------------------------------------

The {{Client Message Dispatcher}} is thread on the server sending messages to the client. The {{Cache Client Updater}} is the thread on the client reading and processing those messages.

Both server thread dumps show the {{Client Message Dispatcher}} in the same place, and both client thread dumps show the {{Cache Client Updater}} in the same place.

The {{Client Message Dispatcher}} is writing to the socket:
{noformat}
"Client Message Dispatcher for 10.55.12.224(32617:loner):46450:647034e9" #172 daemon prio=5 os_prio=0 tid=0x00007f579c03e000 nid=0x3a9f runnable [0x00007f57851ff000]
   java.lang.Thread.State: RUNNABLE
	at java.net.SocketOutputStream.socketWrite0(Native Method)
	at java.net.SocketOutputStream.socketWrite(SocketOutputStream.java:111)
	at java.net.SocketOutputStream.write(SocketOutputStream.java:155)
	at org.apache.geode.internal.cache.tier.sockets.Message.flushBuffer(Message.java:645)
	at org.apache.geode.internal.cache.tier.sockets.Message.sendBytes(Message.java:623)
	- locked <0x00007f5927d8da88> (a java.nio.HeapByteBuffer)
	at org.apache.geode.internal.cache.tier.sockets.Message.send(Message.java:1096)
	at org.apache.geode.internal.cache.tier.sockets.Message.send(Message.java:1083)
	at org.apache.geode.internal.cache.tier.sockets.CacheClientProxy$MessageDispatcher.sendMessage(CacheClientProxy.java:2835)
	at org.apache.geode.internal.cache.tier.sockets.CacheClientProxy$MessageDispatcher.dispatchMessage(CacheClientProxy.java:2792)
	at org.apache.geode.internal.cache.tier.sockets.CacheClientProxy$MessageDispatcher.run(CacheClientProxy.java:2544)
{noformat}
The {{Cache Client Updater}} is reading from the socket:
{noformat}
"Cache Client Updater Thread  on 10.55.13.79(DCA-PRD-GDCS13:13955)<v47>:1024 port 40404" #154 daemon prio=5 os_prio=0 tid=0x00007fab3301d000 nid=0x31c runnable [0x00007fab2e7d4000]
   java.lang.Thread.State: RUNNABLE
	at java.net.SocketInputStream.socketRead0(Native Method)
	at java.net.SocketInputStream.socketRead(SocketInputStream.java:116)
	at java.net.SocketInputStream.read(SocketInputStream.java:171)
	at java.net.SocketInputStream.read(SocketInputStream.java:141)
	at com.gemstone.gemfire.internal.cache.tier.sockets.Message.readPartChunk(Message.java:964)
	at com.gemstone.gemfire.internal.cache.tier.sockets.Message.readPayloadFields(Message.java:819)
	at com.gemstone.gemfire.internal.cache.tier.sockets.Message.readHeaderAndPayload(Message.java:740)
	at com.gemstone.gemfire.internal.cache.tier.sockets.Message.read(Message.java:585)
	at com.gemstone.gemfire.internal.cache.tier.sockets.Message.recv(Message.java:1075)
	- locked <0x0000000349c68518> (a java.nio.HeapByteBuffer)
	at com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientUpdater.processMessages(CacheClientUpdater.java:1649)
	at com.gemstone.gemfire.internal.cache.tier.sockets.CacheClientUpdater.run(CacheClientUpdater.java:467)
{noformat}
Both server and client are waiting on the network.

The client looks pretty much idle. I don't see any thread activity.

The server is pretty busy with putAll activity, but I don't see any issues. There are a bunch of {{p2pDestreamer for P2P message reader}} threads (77 in thread dump 0900 and 79 in thread dump 0912) deserializing {{PutAllMessages}} like:
{noformat}
"p2pDestreamer for P2P message reader@4dcb82fe msgId=2122" #1551 daemon prio=10 os_prio=0 tid=0x00007f5768dd2800 nid=0x6b3 in Object.wait() [0x00007f5736a04000]
   java.lang.Thread.State: WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	at java.lang.Object.wait(Object.java:502)
	at org.apache.geode.internal.tcp.MsgDestreamer$DestreamerIS.waitForData(MsgDestreamer.java:385)
	- locked <0x00007f59016e00e8> (a java.lang.Object)
	at org.apache.geode.internal.tcp.MsgDestreamer$DestreamerIS.waitForAvailableData(MsgDestreamer.java:464)
	at org.apache.geode.internal.tcp.MsgDestreamer$DestreamerIS.read(MsgDestreamer.java:518)
	at java.io.DataInputStream.readFully(DataInputStream.java:195)
	at org.apache.geode.DataSerializer.readByteArray(DataSerializer.java:1349)
	at org.apache.geode.internal.cache.DistributedPutAllOperation$PutAllEntryData.<init>(DistributedPutAllOperation.java:338)
	at org.apache.geode.internal.cache.DistributedPutAllOperation$PutAllMessage.fromData(DistributedPutAllOperation.java:1206)
	at org.apache.geode.internal.InternalDataSerializer.invokeFromData(InternalDataSerializer.java:2477)
	at org.apache.geode.internal.DSFIDFactory.create(DSFIDFactory.java:979)
	at org.apache.geode.internal.InternalDataSerializer.readDSFID(InternalDataSerializer.java:2718)
	at org.apache.geode.internal.tcp.MsgDestreamer$DestreamerThread.run(MsgDestreamer.java:261)
{noformat}
Then there are 3 {{P2P message reader}} threads in each thread dump stuck updating the local client queue with those {{PutAllMessages}}:
{noformat}
"P2P message reader for 10.55.13.77(DCA-PRD-GDCS11:18095)<v51>:1024 shared ordered uid=10 port=34590" #167 daemon prio=10 os_prio=0 tid=0x00007f575800c000 nid=0x3747 in Object.wait() [0x00007f578519d000]
   java.lang.Thread.State: TIMED_WAITING (on object monitor)
	at java.lang.Object.wait(Native Method)
	at org.apache.geode.internal.cache.ha.HARegionQueue$BlockingHARegionQueue.checkQueueSizeConstraint(HARegionQueue.java:2266)
	- locked <0x00007f5938f04f60> (a java.lang.Object)
	- locked <0x00007f5938f04f48> (a java.lang.Object)
	at org.apache.geode.internal.cache.ha.HARegionQueue.basicPut(HARegionQueue.java:612)
	at org.apache.geode.internal.cache.ha.HARegionQueue.put(HARegionQueue.java:591)
	at org.apache.geode.internal.cache.tier.sockets.CacheClientProxy$MessageDispatcher.enqueueMessage(CacheClientProxy.java:2852)
	at org.apache.geode.internal.cache.tier.sockets.CacheClientProxy.deliverMessage(CacheClientProxy.java:1623)
	at org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier.singletonRouteClientMessage(CacheClientNotifier.java:1041)
	at org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier.singletonNotifyClients(CacheClientNotifier.java:955)
	at org.apache.geode.internal.cache.tier.sockets.CacheClientNotifier.notifyClients(CacheClientNotifier.java:819)
	at org.apache.geode.internal.cache.LocalRegion.notifyBridgeClients(LocalRegion.java:6303)
	at org.apache.geode.internal.cache.LocalRegion.invokePutCallbacks(LocalRegion.java:6051)
	at org.apache.geode.internal.cache.EntryEventImpl.invokeCallbacks(EntryEventImpl.java:2287)
	at org.apache.geode.internal.cache.AbstractRegionEntry.dispatchListenerEvents(AbstractRegionEntry.java:140)
	at org.apache.geode.internal.cache.LocalRegion.basicPutPart2(LocalRegion.java:5916)
	at org.apache.geode.internal.cache.AbstractRegionMap.basicPut(AbstractRegionMap.java:2800)
	- locked <0x00007f5b48cb5260> (a org.apache.geode.internal.cache.VMStatsRegionEntryHeapObjectKey)
	at org.apache.geode.internal.cache.LocalRegion.virtualPut(LocalRegion.java:5752)
	at org.apache.geode.internal.cache.DistributedRegion.virtualPut(DistributedRegion.java:337)
	at org.apache.geode.internal.cache.LocalRegionDataView.putEntry(LocalRegionDataView.java:151)
	at org.apache.geode.internal.cache.LocalRegion.basicUpdate(LocalRegion.java:5732)
	at org.apache.geode.internal.cache.AbstractUpdateOperation.doPutOrCreate(AbstractUpdateOperation.java:184)
	at org.apache.geode.internal.cache.AbstractUpdateOperation$AbstractUpdateMessage.basicOperateOnRegion(AbstractUpdateOperation.java:291)
	at org.apache.geode.internal.cache.DistributedPutAllOperation$PutAllMessage.doEntryPut(DistributedPutAllOperation.java:1100)
	at org.apache.geode.internal.cache.DistributedPutAllOperation$PutAllMessage$1.run(DistributedPutAllOperation.java:1185)
{noformat}
From the attached {{DistributionStats_receivedBytes_CacheClientProxyStats_sentBytes.gif}}, you can see the {{DistributionStats}} shows ~1.5 GB {{receivedBytes}} leading up to the issue. The {{CacheClientProxyStats sentBytes}} shows those messages being sent to the client successfully. Suddenly {{sentBytes}} drops to a trickle. At that point, the queue grows to its maximum of 500000.

From the attached {{CacheClientProxyStats_sentBytes.gif}}, you can see the {{CacheClientProxyStats sentBytes}} is regular, but reduced. The {{Client Message Dispatcher}} is sending ~29,000 bytes roughly every 5 minutes.

I'm not sure, but it looks like the network is suddenly not allowing throughput.

> Geode Version: 1.1.1    In one of the project we a...
> -----------------------------------------------------
>
>                 Key: GEODE-3709
>                 URL: https://issues.apache.org/jira/browse/GEODE-3709
>             Project: Geode
>          Issue Type: Improvement
>          Components: client queues
>            Reporter: Gregory Chase
>         Attachments: 20171006-logs-stats-tds.zip, gf-rest-stats-12-05.gfs, myStatisticsArchiveFile-04-01.gfs
>
>
> Geode Version: 1.1.1
> In one of the project we are using Geode. Here is a summary of how we use it.
> - Geode servers have multiple regions. 
> - Clients subscribe to the data from these regions.
> - Clients subscribe interest in all the entries, therefore they get updates about all the entries from creation to modification to deletion.
> - One of the regions usually has 5-10 million entries with a TTL of 24 hours. Most entries are added in an hour's span one after other. So when TTL kicks in, they are often destroyed in an hour.
> Problem:
> Every now and then we observe following message: 
> 	Client queue for _gfe_non_durable_client_with_id_x.x.x.x(14229:loner):42754:e4266fc4_2_queue client is full.
> This seems to happen when the TTL kicks in on the region with 5-10 million entries. Entries start getting evicted (deleted); the updates (destroys) now must be sent to clients. We see that the updates do happen for a while but suddenly the updates stop and the queue size starts growing. This is becoming a major issue for smooth functioning of our production setup. Any help will be much appreciated. 
> I did some ground work by downloading and looking at the code. I see reference to 2 issues #37581, #51400. But I am unable to view actual JIRA tickets (needs login credentials) Hopefully, it helps someone looking at the issue.
> Here is the pertinent code:
>    @Override
>     @edu.umd.cs.findbugs.annotations.SuppressWarnings("TLW_TWO_LOCK_WAIT")
>     void checkQueueSizeConstraint() throws InterruptedException {
>       if (this.haContainer instanceof HAContainerMap && isPrimary()) { // Fix for bug 39413
>         if (Thread.interrupted())
>           throw new InterruptedException();
>         synchronized (this.putGuard) {
>           if (putPermits <= 0) {
>             synchronized (this.permitMon) {
>               if (reconcilePutPermits() <= 0) {
>                 if (region.getSystem().getConfig().getRemoveUnresponsiveClient()) {
>                   isClientSlowReciever = true;
>                 } else {
>                   try {
>                     long logFrequency = CacheClientNotifier.DEFAULT_LOG_FREQUENCY;
>                     CacheClientNotifier ccn = CacheClientNotifier.getInstance();
>                     if (ccn != null) { // check needed for junit tests
>                       logFrequency = ccn.getLogFrequency();
>                     }
>                     if ((this.maxQueueSizeHitCount % logFrequency) == 0) {
>                       logger.warn(LocalizedMessage.create(
>                           LocalizedStrings.HARegionQueue_CLIENT_QUEUE_FOR_0_IS_FULL,
>                           new Object[] {region.getName()}));
>                       this.maxQueueSizeHitCount = 0;
>                     }
>                     ++this.maxQueueSizeHitCount;
>                     this.region.checkReadiness(); // fix for bug 37581
>                     // TODO: wait called while holding two locks
>                     this.permitMon.wait(CacheClientNotifier.eventEnqueueWaitTime);
>                     this.region.checkReadiness(); // fix for bug 37581
>                     // Fix for #51400. Allow the queue to grow beyond its
>                     // capacity/maxQueueSize, if it is taking a long time to
>                     // drain the queue, either due to a slower client or the
>                     // deadlock scenario mentioned in the ticket.
>                     reconcilePutPermits();
>                     if ((this.maxQueueSizeHitCount % logFrequency) == 1) {
>                       logger.info(LocalizedMessage
>                           .create(LocalizedStrings.HARegionQueue_RESUMING_WITH_PROCESSING_PUTS));
>                     }
>                   } catch (InterruptedException ex) {
>                     // TODO: The line below is meaningless. Comment it out later
>                     this.permitMon.notifyAll();
>                     throw ex;
>                   }
>                 }
>               }
>             } // synchronized (this.permitMon)
>           } // if (putPermits <= 0)
>           --putPermits;
>         } // synchronized (this.putGuard)
>       }
>     }
> *Reporter*: Mangesh Deshmukh
> *E-mail*: [mailto:mdeshmukh@quotient.com]



--
This message was sent by Atlassian JIRA
(v6.4.14#64029)