ignite-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Pavel Pereslegin (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (IGNITE-10250) Ignite Queue hangs after several read/write operations
Date Thu, 29 Nov 2018 08:55:00 GMT

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

Pavel Pereslegin commented on IGNITE-10250:
-------------------------------------------

[~sergey-chugunov], thank you very much for the explanation.

You are right, tx-cache listeners are called strictly sequentially (from GridCacheMapEntry).
{noformat}
    at org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryManager.onEntryUpdated(CacheContinuousQueryManager.java:459)
    at org.apache.ignite.internal.processors.cache.GridCacheMapEntry.innerSet(GridCacheMapEntry.java:1626)
    at org.apache.ignite.internal.processors.cache.transactions.IgniteTxLocalAdapter.userCommit(IgniteTxLocalAdapter.java:732)
{noformat}
But atomic cache listeners behavior has been changed in Apache Ignite 1.6 (see IGNITE-2004,
[changes|https://github.com/apache/ignite/commit/2ff64c2aa8f142f2918200e5ee85e6b7e83afbfa#diff-b4734203e762b8729a6bc109a760be14R411]),
now they are called outside of GridCacheMapEntry sync.

As I can see, we can safely invoke listener directly from GridCacheMapEntry for cluster with
single affinity node (this behavior was in AI 1.6-1.9), but for multinode cluster such change
looks dangerous (and I think I need to learn more about the meaning of the changes in IGNITE-2004
before doing that).

Maybe we should discuss this with the authors of IGNITE-2004?

> Ignite Queue hangs after several read/write operations
> ------------------------------------------------------
>
>                 Key: IGNITE-10250
>                 URL: https://issues.apache.org/jira/browse/IGNITE-10250
>             Project: Ignite
>          Issue Type: Bug
>          Components: data structures
>    Affects Versions: 2.7
>            Reporter: Anton Dmitriev
>            Assignee: Pavel Pereslegin
>            Priority: Major
>             Fix For: 2.8
>
>
> Ignite Queue hangs after several read/write operations. Code to reproduce:
> {code:java}
> try (Ignite ignite = Ignition.start()) {
>   IgniteQueue<Integer> queue = ignite.queue("TEST_QUEUE", 1, new CollectionConfiguration());
>   new Thread(() -> {
>     for (int i = 0;; i++) {
>       queue.put(i);
>       System.out.println("Put: " + i);
>     }
>   }).start();
>   new Thread(() -> {
>     for (int i = 0;; i++) {
>       queue.take();
>       System.out.println("Take: " + i);
>     }
>   }).start();
>   Thread.currentThread().join();
> }
> {code}
> *UPDATE AFTER REVIEW*
>  [~xtern], thank you for investigating this issue, great job!
> Unfortunately this hang has a deeper roots and highlights another bug we have in continuous
queries over atomic caches.
> Operations of modifying queue header (which results in invoking of CQ listener and subsequent
call to onHeaderChange callback) are performed on a single key (*queueKey*, see *GridAtomicCacheQueueImpl*)
so even if they are called from different threads they should remain serialized.
> But in case of atomic cache on single node CQ listeners are called synchronously from
the same thread that is making operation on the queue and they are called outside of section
where locks on GridCacheMapEntries objects are held (see method *GridDhtAtomicCache::updateAllAsyncInternal0*
and stack trace below) which breaks guarantee of serialized invocation of listeners.
> So we need to fix this behavior of ATOMIC caches and the issue with queues will disappear
without any changes in *onHeaderChanged* callback.
> Stack trace of the call looks like this:
> {code:java}
>   java.lang.Thread.State: RUNNABLE
> 	  at org.apache.ignite.internal.processors.datastructures.GridCacheQueueAdapter.onHeaderChanged(GridCacheQueueAdapter.java:517)
> 	  at org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager$1.onUpdated(CacheDataStructuresManager.java:305)
> 	  at org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler.notifyLocalListener(CacheContinuousQueryHandler.java:946)
> 	  at org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler.onEntryUpdate(CacheContinuousQueryHandler.java:877)
> 	  at org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler.access$600(CacheContinuousQueryHandler.java:85)
> 	  at org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler$2$1.apply(CacheContinuousQueryHandler.java:437)
> 	  at org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler$2$1.apply(CacheContinuousQueryHandler.java:432)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture.onDone(GridDhtAtomicAbstractUpdateFuture.java:560)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture.onDone(GridDhtAtomicAbstractUpdateFuture.java:62)
> 	  at org.apache.ignite.internal.util.future.GridFutureAdapter.onDone(GridFutureAdapter.java:452)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicAbstractUpdateFuture.map(GridDhtAtomicAbstractUpdateFuture.java:396)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal0(GridDhtAtomicCache.java:1874)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.updateAllAsyncInternal(GridDhtAtomicCache.java:1668)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.sendSingleRequest(GridNearAtomicAbstractUpdateFuture.java:299)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.map(GridNearAtomicSingleUpdateFuture.java:483)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicSingleUpdateFuture.mapOnTopology(GridNearAtomicSingleUpdateFuture.java:443)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicAbstractUpdateFuture.map(GridNearAtomicAbstractUpdateFuture.java:248)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.update0(GridDhtAtomicCache.java:1150)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.invoke0(GridDhtAtomicCache.java:831)
> 	  at org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.invoke(GridDhtAtomicCache.java:787)
> 	  at org.apache.ignite.internal.processors.datastructures.GridAtomicCacheQueueImpl.transformHeader(GridAtomicCacheQueueImpl.java:170)
> 	  at org.apache.ignite.internal.processors.datastructures.GridAtomicCacheQueueImpl.offer(GridAtomicCacheQueueImpl.java:53)
> 	  at org.apache.ignite.internal.processors.datastructures.GridCacheQueueAdapter.put(GridCacheQueueAdapter.java:252)
> 	  at org.apache.ignite.internal.processors.datastructures.GridCacheQueueProxy.put(GridCacheQueueProxy.java:359)
> 	  at org.apache.ignite.internal.processors.cache.datastructures.GridCacheQueueApiSelfAbstractTest.lambda$checkPutAndGetInDifferentThreadsMultithreaded$0(GridCacheQueueApiSelfAbstractTest.java:456)
> 	  at org.apache.ignite.internal.processors.cache.datastructures.GridCacheQueueApiSelfAbstractTest$$Lambda$85.291482897.run(Unknown
Source:-1)
> 	  at org.apache.ignite.testframework.GridTestUtils$7.call(GridTestUtils.java:1300)
> 	  at org.apache.ignite.testframework.GridTestThread.run(GridTestThread.java:84)
> {code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Mime
View raw message