You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@ignite.apache.org by ve...@gmail.com on 2022/07/01 09:12:02 UTC

Re: Delay in receiving notifications 2.8.1

Just to mention we do have @IgniteAsyncCallback annotation on our remote filter and local listener.We use Continuous query with transformer. Will raise a bug

Sent from my iPhone

> On 23 Jun 2022, at 08:35, Veena Mithare <ve...@gmail.com> wrote:
> 
> 
> Hi , 
> 
> Can someone kindly help me know if this is a known issue .
> 
> Regards,
> Veena.
> 
>> On Wed, Jun 22, 2022 at 9:12 AM <ve...@gmail.com> wrote:
>> Hi, 
>> Any idea if this is fixed any later version of ignite?
>> If not I will raise a bug .
>> Regards 
>> Veena
>> 
>> Sent from my iPhone
>> 
>>>> On 20 Jun 2022, at 09:19, Veena Mithare <ve...@gmail.com> wrote:
>>>> 
>>> 
>>> 
>>> Hi , 
>>> 
>>> This issue was observed again in our environment. We have a cluster of 20 client nodes +3 server nodes. 
>>> 
>>> Around 10 nodes listen issue a continuous query to listen to updates from the same cache. If for some reason an exception is thrown while  sending the cache update notification to one of the clients, the rest of the clients get delayed updates .It looks  as if the process of sending notifications to clients is in a synchronous loop.  
>>> 
>>> This was observed when the issue mentioned in here happened : 
>>> 
>>> https://lists.apache.org/thread/tzzksk2cm4dwhd84bcswgosvbvjv01nq
>>> 
>>> I can see the below code for continuous query manager which seems to send notifications synchronously . 
>>> 
>>> Is there any ignite test case, which I can use to test/reproduce this? It will be very difficult to produce a reproducer for a condition where a discovery spi thinks a client as live, but communication spi isn’t able to reach it.
>>> CacheContinuousQueryManager.java
>>>  
>>> ===================================
>>> public void onEntryUpdated(
>>>     Map<UUID, CacheContinuousQueryListener> lsnrCol,
>>>     KeyCacheObject key,
>>>     CacheObject newVal,
>>>     CacheObject oldVal,
>>>     boolean internal,
>>>     int partId,
>>>     boolean primary,
>>>     boolean preload,
>>>     long updateCntr,
>>>     @Nullable GridDhtAtomicAbstractUpdateFuture fut,
>>>     AffinityTopologyVersion topVer)
>>>     throws IgniteCheckedException
>>> {
>>>     assert key != null;
>>>     assert lsnrCol != null;
>>> 
>>>     boolean hasNewVal = newVal != null;
>>>     boolean hasOldVal = oldVal != null;
>>> 
>>>     if (!hasNewVal && !hasOldVal) {
>>>         skipUpdateEvent(lsnrCol, key, partId, updateCntr, primary, topVer);
>>> 
>>>         return;
>>>     }
>>> 
>>>     EventType evtType = !hasNewVal ? REMOVED : !hasOldVal ? CREATED : UPDATED;
>>> 
>>>     boolean initialized = false;
>>> 
>>>     boolean recordIgniteEvt = primary && !internal && cctx.events().isRecordable(EVT_CACHE_QUERY_OBJECT_READ);
>>> 
>>>     for (CacheContinuousQueryListener lsnr : lsnrCol.values()) {
>>>         if (preload && !lsnr.notifyExisting() || lsnr.isPrimaryOnly() && !primary)
>>>             continue;
>>> 
>>>         if (!initialized) {
>>>             if (lsnr.oldValueRequired()) {
>>>                 oldVal = (CacheObject)cctx.unwrapTemporary(oldVal);
>>> 
>>>                 if (oldVal != null)
>>>                     oldVal.finishUnmarshal(cctx.cacheObjectContext(), cctx.deploy().globalLoader());
>>>             }
>>> 
>>>             if (newVal != null)
>>>                 newVal.finishUnmarshal(cctx.cacheObjectContext(), cctx.deploy().globalLoader());
>>> 
>>>             initialized = true;
>>>         }
>>> 
>>>         CacheContinuousQueryEntry e0 = new CacheContinuousQueryEntry(
>>>             cctx.cacheId(),
>>>             evtType,
>>>             key,
>>>             (!internal && evtType == REMOVED && lsnr.oldValueRequired()) ? oldVal : newVal,
>>>             lsnr.oldValueRequired() ? oldVal : null,
>>>             lsnr.keepBinary(),
>>>             partId,
>>>             updateCntr,
>>>             topVer,
>>>             (byte)0);
>>> 
>>>         IgniteCacheProxy jcache = cctx.kernalContext().cache().jcacheProxy(cctx.name(), true);
>>> 
>>>         assert jcache != null : "Failed to get cache proxy [name=" + cctx.name() +
>>>             ", locStart=" + cctx.startTopologyVersion() +
>>>             ", locNode=" + cctx.localNode() +
>>>             ", stopping=" + cctx.kernalContext().isStopping();
>>> 
>>>         CacheContinuousQueryEvent evt = new CacheContinuousQueryEvent<>(jcache, cctx, e0);
>>> 
>>>         lsnr.onEntryUpdated(evt, primary, recordIgniteEvt, fut);
>>>     }
>>> }
>>>  
>>> 
>>> 
>>> regards,
>>> Veena.
>>> 
>>>> On Tue, Dec 14, 2021 at 2:11 PM Veena Mithare <V....@cmcmarkets.com> wrote:
>>>>  
>>>> 
>>>> Hi Team,
>>>> 
>>>>  
>>>> 
>>>> Just following up on this .
>>>> 
>>>>  
>>>> 
>>>> My question is ,
>>>> 
>>>>  
>>>> 
>>>> If : There is an update on a cache and there are a few clients listening to the cache update changes - >
>>>> should notification failure to a few clients( because of network issues etc. experienced by the Communication SPI ), delay notification to the other clients interested.
>>>>  
>>>> 
>>>> If point b is true, wouldn’t that be a bug ..
>>>> 
>>>>  
>>>> 
>>>> Regards,
>>>> 
>>>> Veena.
>>>> 
>>>>  
>>>> 
>>>> From: Veena Mithare 
>>>> Sent: 10 December 2021 17:22
>>>> To: 'user@ignite.apache.org' <us...@ignite.apache.org>
>>>> Subject: RE: Delay in receiving notifications 2.8.1
>>>> 
>>>>  
>>>> 
>>>> The delay seems to match the connection time outs we see in the logs :
>>>> 
>>>>  
>>>> 
>>>> x.log:2021-12-10T14:25:06,167 WARN  o.a.i.s.c.t.TcpCommunicationSpi [callback-#6346]: Connection timed out (will stop attempts to perform the connect) [node=c61379ba-0776-40cf-aa95-d40fb0d057dc, connTimeoutStgy=ExponentialBackoffTimeoutStrategy [maxTimeout=10000, totalTimeout=15000, startNanos=3732477449666384, currTimeout=10000], failureDetectionTimeoutEnabled=false, timeout=2776, err=null, addr=/fde1:53ba:e9a0:de11:f602:70ff:fef0:ecab%idrac:47103]
>>>> 
>>>> x.log:2021-12-10T14:25:21,182 WARN  o.a.i.s.c.t.TcpCommunicationSpi [callback-#6346]: Connection timed out (will stop attempts to perform the connect) [node=28ada9a9-f863-458c-aa4a-368d14f9d917, connTimeoutStgy=ExponentialBackoffTimeoutStrategy [maxTimeout=10000, totalTimeout=15000, startNanos=3732492463335720, currTimeout=10000], failureDetectionTimeoutEnabled=false, timeout=2510, err=null, addr=/fde1:53ba:e9a0:de11:f602:70ff:fef0:ecab%idrac:47100]
>>>> 
>>>>  
>>>> 
>>>> the configuration give is as below :
>>>> 
>>>> <property name="communicationSpi">
>>>>   <bean class="org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi" scope="prototype">
>>>>     <property name="connectTimeout" value="5000"/>
>>>>     <property name="maxConnectTimeout" value="10000"/>
>>>>     <property name="localPort" value="${ignite.communicationSpiPort:47100}"/>
>>>>     <property name="localPortRange" value="20"/>
>>>> 
>>>>   </bean>
>>>> </property>
>>>> 
>>>>  
>>>> 
>>>>  
>>>> 
>>>> From: Veena Mithare 
>>>> Sent: 10 December 2021 17:16
>>>> To: user@ignite.apache.org
>>>> Subject: Delay in receiving notifications 2.8.1
>>>> 
>>>>  
>>>> 
>>>> Hi Team,
>>>> 
>>>>  
>>>> 
>>>> We have a 3 node cluster on 2.8.1 .  And we have around 30 clients and around 20 of them have continuous query registered against the same cache ( say CACHEA )
>>>> 
>>>>  
>>>> 
>>>> We faced the below scenario :
>>>> 
>>>>  
>>>> 
>>>> Cache A was updated .
>>>> A few clients( say 4 or 5 ) received the update with 1 or 2 seconds.
>>>> For about 2 or 3 clients, we can see that server times out connecting to them – ( this could be a valid timeout – because of network issues etc. )
>>>> A few more clients  received the update about 25 seconds later.
>>>>  
>>>> 
>>>> It looks like the point 3 above seems to have affected the clients in point 4. Request you to kindly clarify .
>>>> 
>>>>  
>>>> 
>>>> If not , what could be the reason for the delay for the clients in point 4.
>>>> 
>>>>  
>>>> 
>>>> Regards,
>>>> 
>>>> Veena.
>>>> 
>>>>  
>>>> 
>>>>  
>>>> 
>>>>  
>>>> 
>>>>  
>>>> 
>>>> 
>>>> 
>>>> Spread bets and CFDs are complex instruments and come with a high risk of losing money rapidly due to leverage. 66% of retail investor accounts lose money when spread betting and/or trading CFDs with this provider. You should consider whether you understand how spread bets and CFDs work and whether you can afford to take the high risk of losing your money. 
>>>> 
>>>> Professional clients: Losses can exceed deposits when spread betting and trading CFDs. Countdowns carry a level of risk to your capital as you could lose all of your investment. These products may not be suitable for all clients therefore ensure you understand the risks and seek independent advice. Invest only what you can afford to lose. 
>>>> 
>>>> CMC Markets UK plc (173730) and CMC Spreadbet plc (170627) are authorised and regulated by the Financial Conduct Authority in the United Kingdom. CMC Markets UK plc and CMC Spreadbet plc are registered in England and Wales with Company Numbers 02448409 and 02589529 and with their registered offices at 133 Houndsditch, London, EC3A 7BX. 
>>>> 
>>>> The content of this e-mail (including any attachments) is strictly confidential and is for the sole use of the intended addressee(s). If you are not the intended recipient of this e-mail please notify the sender immediately and delete this e-mail from your system. Any disclosure, copying, dissemination or use of its content (including any attachments) is strictly prohibited. CMC Markets UK plc and CMC Spreadbet plc reserve the right to intercept and monitor the content of the e-mail messages to and from its systems. 
>>>> 
>>>> E-mails may be interfered with or may contain viruses or other defects for which CMC Markets UK plc and CMC Spreadbet plc accept no responsibility. It is the responsibility of the recipient to carry out a virus check on the e-mail and any attachment(s). 
>>>> 
>>>> This communication is not intended as an offer or solicitation for the purchase or sale of a financial instrument or as an official confirmation of any transaction unless specifically presented as such.