You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Aleksey Plekhanov (Jira)" <ji...@apache.org> on 2020/04/02 09:15:00 UTC

[jira] [Created] (IGNITE-12855) Node failed after get operation when entries from the cache expired concurrently

Aleksey Plekhanov created IGNITE-12855:
------------------------------------------

             Summary: Node failed after get operation when entries from the cache expired concurrently 
                 Key: IGNITE-12855
                 URL: https://issues.apache.org/jira/browse/IGNITE-12855
             Project: Ignite
          Issue Type: Bug
            Reporter: Aleksey Plekhanov
            Assignee: Aleksey Plekhanov


Failed with the error:
{noformat}
[12:10:50] (err) Failed to notify listener: o.a.i.i.processors.cache.distributed.dht.GridDhtCacheAdapter$6@7c956694java.lang.AssertionError
    at org.apache.ignite.internal.processors.cache.persistence.GridCacheOffheapManager$GridCacheDataStore.remove(GridCacheOffheapManager.java:2456)
    at org.apache.ignite.internal.processors.cache.IgniteCacheOffheapManagerImpl.remove(IgniteCacheOffheapManagerImpl.java:619)
    at org.apache.ignite.internal.processors.cache.GridCacheMapEntry.removeValue(GridCacheMapEntry.java:4401)
    at org.apache.ignite.internal.processors.cache.GridCacheMapEntry.onExpired(GridCacheMapEntry.java:4095)
    at org.apache.ignite.internal.processors.cache.GridCacheMapEntry.innerGet0(GridCacheMapEntry.java:767)
    at org.apache.ignite.internal.processors.cache.GridCacheMapEntry.innerGetVersioned(GridCacheMapEntry.java:694)
    at org.apache.ignite.internal.processors.cache.GridCacheAdapter.getAllAsync0(GridCacheAdapter.java:2175)
    at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter.getDhtAllAsync(GridDhtCacheAdapter.java:709)
    at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtGetSingleFuture.getAsync(GridDhtGetSingleFuture.java:413)
    at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtGetSingleFuture.map0(GridDhtGetSingleFuture.java:279)
    at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtGetSingleFuture.map(GridDhtGetSingleFuture.java:261)
    at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtGetSingleFuture.init(GridDhtGetSingleFuture.java:182)
    at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter.getDhtSingleAsync(GridDhtCacheAdapter.java:821)
    at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter.processNearSingleGetRequest(GridDhtCacheAdapter.java:836)
    at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$2.apply(GridDhtTransactionalCacheAdapter.java:152)
    at org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTransactionalCacheAdapter$2.apply(GridDhtTransactionalCacheAdapter.java:150)
    at org.apache.ignite.internal.processors.cache.GridCacheIoManager.processMessage(GridCacheIoManager.java:1142)
    at org.apache.ignite.internal.processors.cache.GridCacheIoManager.onMessage0(GridCacheIoManager.java:591)
    at org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:392)
    at org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:318)
    at org.apache.ignite.internal.processors.cache.GridCacheIoManager$1.onMessage(GridCacheIoManager.java:308)
    at org.apache.ignite.internal.managers.communication.GridIoManager.invokeListener(GridIoManager.java:1847)
    at org.apache.ignite.internal.managers.communication.GridIoManager.processRegularMessage0(GridIoManager.java:1472)
    at org.apache.ignite.internal.managers.communication.GridIoManager$9.run(GridIoManager.java:1367)
    at org.apache.ignite.internal.util.StripedExecutor$Stripe.body(StripedExecutor.java:565)
    at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
{noformat}
Reproducer:

 
{code:java}
@Test
public void shouldNotBeProblemToPutToExpiredCacheConcurrently() throws Exception {
    final AtomicBoolean end = new AtomicBoolean();

    final IgniteEx srv = startGrid(3);

    srv.cluster().active(true);

    IgniteInternalFuture loadFut = runMultiThreadedAsync(() -> {
        while (!end.get() && !fail) {
            IgniteCache<Integer, byte[]> cache = srv.cache(CACHE_NAME);

            for (int i = 0; i < ENTRIES; i++)
                cache.put(i, new byte[1024]);

            for (int i = 0; i < ENTRIES; i++)
                cache.get(i); // touch entries
        }
    }, WORKLOAD_THREADS_CNT, "high-workload");

    try {
        loadFut.get(10, TimeUnit.SECONDS);
    }
    catch (Exception e) {
        assertFalse("Failure handler was called. See log above.", fail);

        assertTrue(X.hasCause(e, IgniteFutureTimeoutCheckedException.class));
    }
    finally {
        end.set(true);
    }

    assertFalse("Failure handler was called. See log above.", fail);
}
{code}
 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)