[ 
https://issues.apache.org/jira/browse/IGNITE-28381?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Mikhail Petrov updated IGNITE-28381:
------------------------------------
    Description: 
The problem was reproduced during 
org.apache.ignite.client.ReliabilityTest#testFailover test on TC. See 
attachments for the full log.

Exception:

{code:java}
java.lang.AssertionError
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.onPrimaryResponse(GridNearAtomicUpdateFuture.java:400)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.processNearAtomicUpdateResponse(GridDhtAtomicCache.java:3236)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5.apply(GridDhtAtomicCache.java:311)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5.apply(GridDhtAtomicCache.java:306)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.GridCacheIoManager.processMessage(GridCacheIoManager.java:1084)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.GridCacheIoManager.onMessage0(GridCacheIoManager.java:594)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:396)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.GridCacheIoManager$1.onMessage(GridCacheIoManager.java:306)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.managers.communication.GridIoManager.invokeListener(GridIoManager.java:1862)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.managers.communication.GridIoManager.processRegularMessage0(GridIoManager.java:1483)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.managers.communication.GridIoManager$9.execute(GridIoManager.java:1387)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.managers.communication.TraceRunnable.run(TraceRunnable.java:55)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.thread.pool.IgniteStripedExecutor$Stripe.body(IgniteStripedExecutor.java:638)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:125)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
java.base/java.lang.Thread.run(Thread.java:829)
{code}


The root cause of the assertion error:

Consider a cluster consisting of 3 nodes - node0, node1, node2

1. node2 is gracefully stopped
2. PME is started on node0 and node 1
3. PME is completed on node 0. So its topology version is now V+1.
4. Ignite#close method is called to gracefully stop the node 1
5. PME future on node1 is cancelled with the following exception 

{code:java}
org.apache.ignite.internal.NodeStoppingException: Node is stopping: 
e2c0e324-70b6-45b8-925d-fe9a9fa5d877
        at 
org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager.onKernalStop0(GridCachePartitionExchangeManager.java:842)
        at 
org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter.onKernalStop(GridCacheSharedManagerAdapter.java:131)
        at 
org.apache.ignite.internal.processors.cache.GridCacheProcessor.onKernalStop(GridCacheProcessor.java:819)
        at org.apache.ignite.internal.IgniteKernal.stop0(IgniteKernal.java:1782)
        at org.apache.ignite.internal.IgniteKernal.stop(IgniteKernal.java:1731)
        at 
org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop0(IgnitionEx.java:2297)
        at 
org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop(IgnitionEx.java:2117)
        at org.apache.ignite.internal.IgnitionEx.stop(IgnitionEx.java:308)
        at org.apache.ignite.Ignition.stop(Ignition.java:202)
        at org.apache.ignite.internal.IgniteKernal.close(IgniteKernal.java:2686)
        at 
org.apache.ignite.client.LocalIgniteCluster.failNode(LocalIgniteCluster.java:110)
        at 
org.apache.ignite.client.ReliabilityTest.lambda$assertOnUnstableCluster$17(ReliabilityTest.java:714)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
        at 
java.base/java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:264)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java)
{code}
6. GridCachePartitionExchangeManager#onExchangeDone and 
GridDhtPartitionTopology#onExchangeDone are not called on node 1 because the 
PME future was completed with the exception.
7.Even when stopped, node 1 can receive and process requests from other nodes 
for a short period of time. Node 1 (primary) receives 
GridDhtAtomicAbstractUpdateRequest from node 0 (near). Which was mapped by node 
0 to V+1 topology version.
8. PME future was completed on node 1 so nothing prevents it from handling the 
request (see GridMessageListener#onMessage) 
9. The cache request is handled by GridDhtAtomicCache#updateAllAsyncInternal0
10. GridDhtCacheAdapter#needRemap check returns true because cache request was 
mapped on V+1 topology version but node1 local version is V (see 
`cctx.topology().readyTopologyVersion()` call in 
GridCacheAffinityManager#isCompatibleWithCurrentTopologyVersion method and 
clause 6). 
11. node1 sends response to node0 with remap topology version == V+1. See 
`res.remapTopologyVersion(top.lastTopologyChangeVersion());` call. 
`top.lastTopologyChangeVersion()` returns V+1 because even PME future failed, 
it was started and related to V+1 topology version.
12. node0 receives node1 response  and fails with the mentioned above stack 
trace 
Actually it may differ whether single cache put is called or putAll.

Reproducer:

{code:java}
/*
 * Licensed to the Apache Software Foundation (ASF) under one or more
 * contributor license agreements.  See the NOTICE file distributed with
 * this work for additional information regarding copyright ownership.
 * The ASF licenses this file to You under the Apache License, Version 2.0
 * (the "License"); you may not use this file except in compliance with
 * the License.  You may obtain a copy of the License at
 *
 *      http://www.apache.org/licenses/LICENSE-2.0
 *
 * Unless required by applicable law or agreed to in writing, software
 * distributed under the License is distributed on an "AS IS" BASIS,
 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
 * See the License for the specific language governing permissions and
 * limitations under the License.
 */

package org.apache.ignite;

import java.util.concurrent.CountDownLatch;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.ignite.configuration.IgniteConfiguration;
import org.apache.ignite.failure.StopNodeOrHaltFailureHandler;
import org.apache.ignite.internal.IgniteEx;
import org.apache.ignite.internal.IgniteInternalFuture;
import 
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
import 
org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware;
import org.apache.ignite.testframework.GridTestUtils;
import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
import org.junit.Test;

import static java.util.concurrent.TimeUnit.MILLISECONDS;

/** */
public class NodeStopCacheOperationRemappingTest extends GridCommonAbstractTest 
{
    /** {@inheritDoc} */
    @Override protected IgniteConfiguration getConfiguration() throws Exception 
{
        return super.getConfiguration().setFailureHandler(new 
StopNodeOrHaltFailureHandler());
    }

    /** */
    @Test
    public void test() throws Exception {
        startGrids(3);

        grid(0).createCache(DEFAULT_CACHE_NAME);

        IgniteEx firstNode = grid(1);

        CountDownLatch zeroNodePmeFinishedLatch = new CountDownLatch(1);
        CountDownLatch firstNodePmeStartedLatch = new CountDownLatch(1);
        CountDownLatch firstNodePmeUnblockedLatch = new CountDownLatch(1);
        CountDownLatch firstNodePmeFinishedLatch = new CountDownLatch(1);
        CountDownLatch firstNodeStopUnblockedLatch = new CountDownLatch(1);

        
grid(0).context().cache().context().exchange().registerExchangeAwareComponent(new
 PartitionsExchangeAware() {
            @Override public void 
onDoneAfterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {
                zeroNodePmeFinishedLatch.countDown();
            }
        });

        
firstNode.context().cache().context().exchange().registerExchangeAwareComponent(new
 PartitionsExchangeAware() {
            @Override public void 
onInitBeforeTopologyLock(GridDhtPartitionsExchangeFuture fut) {
                try {
                    firstNodePmeStartedLatch.countDown();

                    firstNodePmeUnblockedLatch.await(getTestTimeout(), 
MILLISECONDS);
                }
                catch (InterruptedException e) {
                    throw new RuntimeException(e);
                }
            }
        });

        stopGrid(2);

        firstNodePmeStartedLatch.await(getTestTimeout(), MILLISECONDS);

        
firstNode.context().cache().context().exchange().lastTopologyFuture().listen(() 
-> {
            try {
                firstNodePmeFinishedLatch.countDown();

                firstNodeStopUnblockedLatch.await(getTestTimeout(), 
MILLISECONDS);
            }
            catch (InterruptedException e) {
                throw new RuntimeException(e);
            }
        });

        IgniteInternalFuture<Object> firstNodeStopFut = 
GridTestUtils.runAsync(() -> stopGrid(1));

        firstNodePmeUnblockedLatch.countDown();

        firstNodePmeFinishedLatch.await(getTestTimeout(), MILLISECONDS);

        zeroNodePmeFinishedLatch.await(getTestTimeout(), MILLISECONDS);

        
grid(0).cache(DEFAULT_CACHE_NAME).put(keyForNode(grid(0).affinity(DEFAULT_CACHE_NAME),
 new AtomicInteger(), firstNode.localNode()), "test-val");

        firstNodeStopUnblockedLatch.countDown();

        firstNodeStopFut.get(getTestTimeout(), MILLISECONDS);
    }

    /** {@inheritDoc} */
    @Override protected long getTestTimeout() {
        return 20_000;
    }
}

{code}





  was:
The problem was reproduced during 
org.apache.ignite.client.ReliabilityTest#testFailover test on TC. See 
attachments for the full log.

Exception:

{code:java}
java.lang.AssertionError
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.onPrimaryResponse(GridNearAtomicUpdateFuture.java:400)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.processNearAtomicUpdateResponse(GridDhtAtomicCache.java:3236)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5.apply(GridDhtAtomicCache.java:311)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5.apply(GridDhtAtomicCache.java:306)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.GridCacheIoManager.processMessage(GridCacheIoManager.java:1084)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.GridCacheIoManager.onMessage0(GridCacheIoManager.java:594)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:396)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.processors.cache.GridCacheIoManager$1.onMessage(GridCacheIoManager.java:306)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.managers.communication.GridIoManager.invokeListener(GridIoManager.java:1862)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.managers.communication.GridIoManager.processRegularMessage0(GridIoManager.java:1483)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.managers.communication.GridIoManager$9.execute(GridIoManager.java:1387)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.managers.communication.TraceRunnable.run(TraceRunnable.java:55)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.thread.pool.IgniteStripedExecutor$Stripe.body(IgniteStripedExecutor.java:638)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:125)
[04:31:10]      [org.apache.ignite:ignite-indexing]     at 
java.base/java.lang.Thread.run(Thread.java:829)
{code}


The root cause of the assertion error:

Consider a cluster consisting of 3 nodes - node0, node1, node2

1. node2 is gracefully stopped
2. PME is started on node0 and node 1
3. PME is completed on node 0. So its topology version is now V+1.
4. Ignite#close method is called to gracefully stop the node 1
5. PME future on node1 is cancelled with the following exception 

{code:java}
org.apache.ignite.internal.NodeStoppingException: Node is stopping: 
e2c0e324-70b6-45b8-925d-fe9a9fa5d877
        at 
org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager.onKernalStop0(GridCachePartitionExchangeManager.java:842)
        at 
org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter.onKernalStop(GridCacheSharedManagerAdapter.java:131)
        at 
org.apache.ignite.internal.processors.cache.GridCacheProcessor.onKernalStop(GridCacheProcessor.java:819)
        at org.apache.ignite.internal.IgniteKernal.stop0(IgniteKernal.java:1782)
        at org.apache.ignite.internal.IgniteKernal.stop(IgniteKernal.java:1731)
        at 
org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop0(IgnitionEx.java:2297)
        at 
org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop(IgnitionEx.java:2117)
        at org.apache.ignite.internal.IgnitionEx.stop(IgnitionEx.java:308)
        at org.apache.ignite.Ignition.stop(Ignition.java:202)
        at org.apache.ignite.internal.IgniteKernal.close(IgniteKernal.java:2686)
        at 
org.apache.ignite.client.LocalIgniteCluster.failNode(LocalIgniteCluster.java:110)
        at 
org.apache.ignite.client.ReliabilityTest.lambda$assertOnUnstableCluster$17(ReliabilityTest.java:714)
        at 
java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
        at 
java.base/java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:264)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java)
{code}
6. GridCachePartitionExchangeManager#onExchangeDone and 
GridDhtPartitionTopology#onExchangeDone are not called on node 1 because the 
PME future was completed with the exception.
7.Even when stopped, node 1 can receive and process requests from other nodes 
for a short period of time. Node 1 (primary) receives 
GridDhtAtomicAbstractUpdateRequest from node 0 (near). Which was mapped by node 
0 to V+1 topology version.
8. PME future was completed on node 1 so nothing prevents it from handling the 
request (see GridMessageListener#onMessage) 
9. The cache request is handled by GridDhtAtomicCache#updateAllAsyncInternal0
10. GridDhtCacheAdapter#needRemap check returns true because cache request was 
mapped on V+1 topology version but node1 local version is V (see 
`cctx.topology().readyTopologyVersion()` call in 
GridCacheAffinityManager#isCompatibleWithCurrentTopologyVersion method and 
clause 6). 
11. node1 sends response to node0 with remap topology version == V+1. See 
`res.remapTopologyVersion(top.lastTopologyChangeVersion());` call. 
`top.lastTopologyChangeVersion()` returns V+1 because even PME future failed, 
it was started and related to V+1 topology version.
12. node0 receives node1 response  and fails with the mentioned above stack 
trace 
Actually it may differ whether single cache put is called or putAll.







> Node failure with Assertion Error during Atomic Cache operation topology 
> remapping
> ----------------------------------------------------------------------------------
>
>                 Key: IGNITE-28381
>                 URL: https://issues.apache.org/jira/browse/IGNITE-28381
>             Project: Ignite
>          Issue Type: Bug
>            Reporter: Mikhail Petrov
>            Assignee: Mikhail Petrov
>            Priority: Major
>              Labels: ise
>             Fix For: 2.19
>
>         Attachments: _Apache_Ignite_2.x_Tests_Thin_Client_Java_16197.log.zip
>
>
> The problem was reproduced during 
> org.apache.ignite.client.ReliabilityTest#testFailover test on TC. See 
> attachments for the full log.
> Exception:
> {code:java}
> java.lang.AssertionError
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridNearAtomicUpdateFuture.onPrimaryResponse(GridNearAtomicUpdateFuture.java:400)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache.processNearAtomicUpdateResponse(GridDhtAtomicCache.java:3236)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5.apply(GridDhtAtomicCache.java:311)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.processors.cache.distributed.dht.atomic.GridDhtAtomicCache$5.apply(GridDhtAtomicCache.java:306)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.processMessage(GridCacheIoManager.java:1084)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.onMessage0(GridCacheIoManager.java:594)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager.handleMessage(GridCacheIoManager.java:396)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.processors.cache.GridCacheIoManager$1.onMessage(GridCacheIoManager.java:306)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.managers.communication.GridIoManager.invokeListener(GridIoManager.java:1862)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.managers.communication.GridIoManager.processRegularMessage0(GridIoManager.java:1483)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.managers.communication.GridIoManager$9.execute(GridIoManager.java:1387)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.managers.communication.TraceRunnable.run(TraceRunnable.java:55)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.thread.pool.IgniteStripedExecutor$Stripe.body(IgniteStripedExecutor.java:638)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:125)
> [04:31:10]    [org.apache.ignite:ignite-indexing]     at 
> java.base/java.lang.Thread.run(Thread.java:829)
> {code}
> The root cause of the assertion error:
> Consider a cluster consisting of 3 nodes - node0, node1, node2
> 1. node2 is gracefully stopped
> 2. PME is started on node0 and node 1
> 3. PME is completed on node 0. So its topology version is now V+1.
> 4. Ignite#close method is called to gracefully stop the node 1
> 5. PME future on node1 is cancelled with the following exception 
> {code:java}
> org.apache.ignite.internal.NodeStoppingException: Node is stopping: 
> e2c0e324-70b6-45b8-925d-fe9a9fa5d877
>       at 
> org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager.onKernalStop0(GridCachePartitionExchangeManager.java:842)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter.onKernalStop(GridCacheSharedManagerAdapter.java:131)
>       at 
> org.apache.ignite.internal.processors.cache.GridCacheProcessor.onKernalStop(GridCacheProcessor.java:819)
>       at org.apache.ignite.internal.IgniteKernal.stop0(IgniteKernal.java:1782)
>       at org.apache.ignite.internal.IgniteKernal.stop(IgniteKernal.java:1731)
>       at 
> org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop0(IgnitionEx.java:2297)
>       at 
> org.apache.ignite.internal.IgnitionEx$IgniteNamedInstance.stop(IgnitionEx.java:2117)
>       at org.apache.ignite.internal.IgnitionEx.stop(IgnitionEx.java:308)
>       at org.apache.ignite.Ignition.stop(Ignition.java:202)
>       at org.apache.ignite.internal.IgniteKernal.close(IgniteKernal.java:2686)
>       at 
> org.apache.ignite.client.LocalIgniteCluster.failNode(LocalIgniteCluster.java:110)
>       at 
> org.apache.ignite.client.ReliabilityTest.lambda$assertOnUnstableCluster$17(ReliabilityTest.java:714)
>       at 
> java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
>       at 
> java.base/java.util.concurrent.FutureTask.run$$$capture(FutureTask.java:264)
>       at java.base/java.util.concurrent.FutureTask.run(FutureTask.java)
> {code}
> 6. GridCachePartitionExchangeManager#onExchangeDone and 
> GridDhtPartitionTopology#onExchangeDone are not called on node 1 because the 
> PME future was completed with the exception.
> 7.Even when stopped, node 1 can receive and process requests from other nodes 
> for a short period of time. Node 1 (primary) receives 
> GridDhtAtomicAbstractUpdateRequest from node 0 (near). Which was mapped by 
> node 0 to V+1 topology version.
> 8. PME future was completed on node 1 so nothing prevents it from handling 
> the request (see GridMessageListener#onMessage) 
> 9. The cache request is handled by GridDhtAtomicCache#updateAllAsyncInternal0
> 10. GridDhtCacheAdapter#needRemap check returns true because cache request 
> was mapped on V+1 topology version but node1 local version is V (see 
> `cctx.topology().readyTopologyVersion()` call in 
> GridCacheAffinityManager#isCompatibleWithCurrentTopologyVersion method and 
> clause 6). 
> 11. node1 sends response to node0 with remap topology version == V+1. See 
> `res.remapTopologyVersion(top.lastTopologyChangeVersion());` call. 
> `top.lastTopologyChangeVersion()` returns V+1 because even PME future failed, 
> it was started and related to V+1 topology version.
> 12. node0 receives node1 response  and fails with the mentioned above stack 
> trace 
> Actually it may differ whether single cache put is called or putAll.
> Reproducer:
> {code:java}
> /*
>  * Licensed to the Apache Software Foundation (ASF) under one or more
>  * contributor license agreements.  See the NOTICE file distributed with
>  * this work for additional information regarding copyright ownership.
>  * The ASF licenses this file to You under the Apache License, Version 2.0
>  * (the "License"); you may not use this file except in compliance with
>  * the License.  You may obtain a copy of the License at
>  *
>  *      http://www.apache.org/licenses/LICENSE-2.0
>  *
>  * Unless required by applicable law or agreed to in writing, software
>  * distributed under the License is distributed on an "AS IS" BASIS,
>  * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
>  * See the License for the specific language governing permissions and
>  * limitations under the License.
>  */
> package org.apache.ignite;
> import java.util.concurrent.CountDownLatch;
> import java.util.concurrent.atomic.AtomicInteger;
> import org.apache.ignite.configuration.IgniteConfiguration;
> import org.apache.ignite.failure.StopNodeOrHaltFailureHandler;
> import org.apache.ignite.internal.IgniteEx;
> import org.apache.ignite.internal.IgniteInternalFuture;
> import 
> org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
> import 
> org.apache.ignite.internal.processors.cache.distributed.dht.preloader.PartitionsExchangeAware;
> import org.apache.ignite.testframework.GridTestUtils;
> import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
> import org.junit.Test;
> import static java.util.concurrent.TimeUnit.MILLISECONDS;
> /** */
> public class NodeStopCacheOperationRemappingTest extends 
> GridCommonAbstractTest {
>     /** {@inheritDoc} */
>     @Override protected IgniteConfiguration getConfiguration() throws 
> Exception {
>         return super.getConfiguration().setFailureHandler(new 
> StopNodeOrHaltFailureHandler());
>     }
>     /** */
>     @Test
>     public void test() throws Exception {
>         startGrids(3);
>         grid(0).createCache(DEFAULT_CACHE_NAME);
>         IgniteEx firstNode = grid(1);
>         CountDownLatch zeroNodePmeFinishedLatch = new CountDownLatch(1);
>         CountDownLatch firstNodePmeStartedLatch = new CountDownLatch(1);
>         CountDownLatch firstNodePmeUnblockedLatch = new CountDownLatch(1);
>         CountDownLatch firstNodePmeFinishedLatch = new CountDownLatch(1);
>         CountDownLatch firstNodeStopUnblockedLatch = new CountDownLatch(1);
>         
> grid(0).context().cache().context().exchange().registerExchangeAwareComponent(new
>  PartitionsExchangeAware() {
>             @Override public void 
> onDoneAfterTopologyUnlock(GridDhtPartitionsExchangeFuture fut) {
>                 zeroNodePmeFinishedLatch.countDown();
>             }
>         });
>         
> firstNode.context().cache().context().exchange().registerExchangeAwareComponent(new
>  PartitionsExchangeAware() {
>             @Override public void 
> onInitBeforeTopologyLock(GridDhtPartitionsExchangeFuture fut) {
>                 try {
>                     firstNodePmeStartedLatch.countDown();
>                     firstNodePmeUnblockedLatch.await(getTestTimeout(), 
> MILLISECONDS);
>                 }
>                 catch (InterruptedException e) {
>                     throw new RuntimeException(e);
>                 }
>             }
>         });
>         stopGrid(2);
>         firstNodePmeStartedLatch.await(getTestTimeout(), MILLISECONDS);
>         
> firstNode.context().cache().context().exchange().lastTopologyFuture().listen(()
>  -> {
>             try {
>                 firstNodePmeFinishedLatch.countDown();
>                 firstNodeStopUnblockedLatch.await(getTestTimeout(), 
> MILLISECONDS);
>             }
>             catch (InterruptedException e) {
>                 throw new RuntimeException(e);
>             }
>         });
>         IgniteInternalFuture<Object> firstNodeStopFut = 
> GridTestUtils.runAsync(() -> stopGrid(1));
>         firstNodePmeUnblockedLatch.countDown();
>         firstNodePmeFinishedLatch.await(getTestTimeout(), MILLISECONDS);
>         zeroNodePmeFinishedLatch.await(getTestTimeout(), MILLISECONDS);
>         
> grid(0).cache(DEFAULT_CACHE_NAME).put(keyForNode(grid(0).affinity(DEFAULT_CACHE_NAME),
>  new AtomicInteger(), firstNode.localNode()), "test-val");
>         firstNodeStopUnblockedLatch.countDown();
>         firstNodeStopFut.get(getTestTimeout(), MILLISECONDS);
>     }
>     /** {@inheritDoc} */
>     @Override protected long getTestTimeout() {
>         return 20_000;
>     }
> }
> {code}



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to