You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ignite.apache.org by "Pavel Pereslegin (Jira)" <ji...@apache.org> on 2021/02/24 10:49:00 UTC

[jira] [Created] (IGNITE-14227) Partition map exchange may hang if the cluster is deactivated during cache start failure handling.

Pavel Pereslegin created IGNITE-14227:
-----------------------------------------

             Summary: Partition map exchange may hang if the cluster is deactivated during cache start failure handling.
                 Key: IGNITE-14227
                 URL: https://issues.apache.org/jira/browse/IGNITE-14227
             Project: Ignite
          Issue Type: Bug
            Reporter: Pavel Pereslegin


Partition map exchange may hang if the cluster is deactivated after processing DynamicCacheChangeBatch but before sending DynamicCacheChangeFailureMessage.

The hang occurs because the current exchange (dynamic cache start) cannot be completed when the state changed to inactive and exchange future is not completed (see usage of GridCachePartitionExchangeManager#processEventInactive in DiscoveryEventListener and GridDhtPartitionsExchangeFuture#onDynamicCacheChangeFail).

Reproducer:
{code:java}
public class DynamicCacheStartFailTest extends GridCommonAbstractTest {
    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
        TcpDiscoverySpi discoSpi = new BlockingDiscoverySpi();

        discoSpi.setIpFinder(((TcpDiscoverySpi)cfg.getDiscoverySpi()).getIpFinder());

        return cfg.setDiscoverySpi(new BlockingDiscoverySpi())
            .setDataStorageConfiguration(new DataStorageConfiguration()
                .setDefaultDataRegionConfiguration(new DataRegionConfiguration()
                    .setMaxSize(100 * 1024 * 1024L)
                    .setPersistenceEnabled(true)));
    }

    @Test
    public void testRollbackCacheStartOnDeactivate() throws Exception {
        IgniteEx crd = startGrids(2);

        CacheConfiguration<Object, Object> cfg = new CacheConfiguration<>("cache1").setDataRegionName("absent");

        crd.cluster().state(ACTIVE);

        BlockingDiscoverySpi spi = (BlockingDiscoverySpi)crd.context().discovery().getInjectedDiscoverySpi();

        spi.block(msg -> msg instanceof DynamicCacheChangeFailureMessage);

        IgniteInternalFuture<IgniteCache<Object, Object>> fut = GridTestUtils.runAsync(() -> crd.createCache(cfg));

        spi.waitBlocked(5_000);

        IgniteInternalFuture<?> stateChangeFut = runAsync(() -> grid(1).cluster().state(ClusterState.INACTIVE));

        spi.unblock();

        assertThrowsAnyCause(log,
            () -> fut.get(getTestTimeout()),
            IgniteCheckedException.class, null);

        stateChangeFut.get(getTestTimeout());
    }

    /** */
    private static class BlockingDiscoverySpi extends TcpDiscoverySpi {
        private final List<DiscoverySpiCustomMessage> blocked = new CopyOnWriteArrayList<>();
        private volatile IgnitePredicate<DiscoveryCustomMessage> blockPred;

        @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
            if (msg instanceof CustomMessageWrapper) {
                DiscoveryCustomMessage msg0 = ((CustomMessageWrapper)msg).delegate();

                if (blockPred != null && blockPred.apply(msg0)) {
                    blocked.add(msg);

                    return;
                }
            }

            super.sendCustomEvent(msg);
        }

        public synchronized void block(IgnitePredicate<DiscoveryCustomMessage> pred) {
            blockPred = pred;
        }

        public synchronized void unblock() {
            blockPred = null;

            for (DiscoverySpiCustomMessage msg : blocked)
                sendCustomEvent(msg);

            blocked.clear();
        }

        public void waitBlocked(long timeout) throws IgniteInterruptedCheckedException {
            GridTestUtils.waitForCondition(() -> !blocked.isEmpty(), timeout);
        }
    }
}
{code}

Typical output:

{noformat}
[2021-02-24 13:37:11,944][WARN ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][GridCacheProcessor] Cache can not be started : cache=cache1
[2021-02-24 13:37:11,944][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][GridCacheProcessor] Cache can not be started : cache=cache1
[2021-02-24 13:37:11,945][ERROR][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][GridDhtPartitionsExchangeFuture] Failed to initialize cache(s) (will try to rollback) [exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], discoEvt=DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=af52d93d771-bd608b2d-8308-4556-9f9f-91bae6049b7c, reqs=ArrayList [DynamicCacheChangeRequest [cacheName=cache1, hasCfg=true, nodeId=b05182d2-c6e8-4b5b-bde2-576e1e700000, clientStartOnly=false, stop=false, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=[cache1], stopCaches=null, startGrps=[cache1], stopGrps=[], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163031766, loc=false, ver=2.11.0#20210224-sha1:00000000, isClient=false], topVer=2, msgTemplate=null, span=o.a.i.i.processors.tracing.NoopSpan@7e02a8f5, nodeId8=02c46b75, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1614163031926]], nodeId=b05182d2, evt=DISCOVERY_CUSTOM_EVT], caches=[o.a.i.i.processors.cache.ExchangeActions$CacheGroupActionData@50838b34]]
class org.apache.ignite.IgniteCheckedException: Requested DataRegion is not configured: absent
	at org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager.dataRegion(IgniteCacheDatabaseSharedManager.java:911)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.startCacheGroup(GridCacheProcessor.java:2467)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.getOrCreateCacheGroupContext(GridCacheProcessor.java:2205)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.prepareCacheContext(GridCacheProcessor.java:2012)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.prepareCacheStart(GridCacheProcessor.java:1946)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.lambda$prepareStartCaches$55a0e703$1(GridCacheProcessor.java:1821)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.lambda$prepareStartCachesIfPossible$14(GridCacheProcessor.java:1791)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.prepareStartCaches(GridCacheProcessor.java:1818)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.prepareStartCachesIfPossible(GridCacheProcessor.java:1789)
	at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.processCacheStartRequests(CacheAffinitySharedManager.java:996)
	at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.onCacheChangeRequest(CacheAffinitySharedManager.java:882)
	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.onCacheChangeRequest(GridDhtPartitionsExchangeFuture.java:1450)
	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:948)
	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body0(GridCachePartitionExchangeManager.java:3387)
	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:3209)
	at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
	at java.lang.Thread.run(Thread.java:748)
[2021-02-24 13:37:11,945][ERROR][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][GridDhtPartitionsExchangeFuture] Failed to initialize cache(s) (will try to rollback) [exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], discoEvt=DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=af52d93d771-bd608b2d-8308-4556-9f9f-91bae6049b7c, reqs=ArrayList [DynamicCacheChangeRequest [cacheName=cache1, hasCfg=true, nodeId=b05182d2-c6e8-4b5b-bde2-576e1e700000, clientStartOnly=false, stop=false, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=[cache1], stopCaches=null, startGrps=[cache1], stopGrps=[], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163031926, loc=true, ver=2.11.0#20210224-sha1:00000000, isClient=false], topVer=2, msgTemplate=null, span=o.a.i.i.processors.tracing.NoopSpan@7e02a8f5, nodeId8=b05182d2, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1614163031926]], nodeId=b05182d2, evt=DISCOVERY_CUSTOM_EVT], caches=[o.a.i.i.processors.cache.ExchangeActions$CacheGroupActionData@cc19771]]
class org.apache.ignite.IgniteCheckedException: Requested DataRegion is not configured: absent
	at org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager.dataRegion(IgniteCacheDatabaseSharedManager.java:911)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.startCacheGroup(GridCacheProcessor.java:2467)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.getOrCreateCacheGroupContext(GridCacheProcessor.java:2205)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.prepareCacheContext(GridCacheProcessor.java:2012)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.prepareCacheStart(GridCacheProcessor.java:1946)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.lambda$prepareStartCaches$55a0e703$1(GridCacheProcessor.java:1821)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.lambda$prepareStartCachesIfPossible$14(GridCacheProcessor.java:1791)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.prepareStartCaches(GridCacheProcessor.java:1818)
	at org.apache.ignite.internal.processors.cache.GridCacheProcessor.prepareStartCachesIfPossible(GridCacheProcessor.java:1789)
	at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.processCacheStartRequests(CacheAffinitySharedManager.java:996)
	at org.apache.ignite.internal.processors.cache.CacheAffinitySharedManager.onCacheChangeRequest(CacheAffinitySharedManager.java:882)
	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.onCacheChangeRequest(GridDhtPartitionsExchangeFuture.java:1450)
	at org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.init(GridDhtPartitionsExchangeFuture.java:948)
	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body0(GridCachePartitionExchangeManager.java:3387)
	at org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager$ExchangeWorker.body(GridCachePartitionExchangeManager.java:3209)
	at org.apache.ignite.internal.util.worker.GridWorker.run(GridWorker.java:120)
	at java.lang.Thread.run(Thread.java:748)
[2021-02-24 13:37:11,949][INFO ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][GridDhtPartitionsExchangeFuture] Finished waiting for partition release future [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], waitTime=0ms, futInfo=NA, mode=DISTRIBUTED]
[2021-02-24 13:37:11,950][INFO ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][GridDhtPartitionsExchangeFuture] Finished waiting for partition release future [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], waitTime=0ms, futInfo=NA, mode=DISTRIBUTED]
[2021-02-24 13:37:11,952][INFO ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][GridDhtPartitionsExchangeFuture] Finished waiting for partitions release latch: ServerLatch [permits=0, pendingAcks=HashSet [], super=CompletableLatch [id=CompletableLatchUid [id=exchange, topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2]]]]
[2021-02-24 13:37:11,952][INFO ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][GridDhtPartitionsExchangeFuture] Finished waiting for partitions release latch: ClientLatch [coordinator=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163031766, loc=false, ver=2.11.0#20210224-sha1:00000000, isClient=false], ackSent=true, super=CompletableLatch [id=CompletableLatchUid [id=exchange, topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2]]]]
[2021-02-24 13:37:11,953][INFO ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][GridDhtPartitionsExchangeFuture] Finished waiting for partition release future [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], waitTime=0ms, futInfo=NA, mode=LOCAL]
[2021-02-24 13:37:11,953][INFO ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][GridDhtPartitionsExchangeFuture] Finished waiting for partition release future [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], waitTime=0ms, futInfo=NA, mode=LOCAL]
[2021-02-24 13:37:11,953][INFO ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][time] Finished exchange init [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], crd=true]
[2021-02-24 13:37:11,954][INFO ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][time] Finished exchange init [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], crd=false]
[2021-02-24 13:37:11,955][INFO ][sys-#53%cache.DynamicCacheStartFailTest0%][GridDeploymentLocalStore] Class locally deployed: class org.apache.ignite.IgniteCheckedException
[2021-02-24 13:37:11,955][INFO ][sys-#53%cache.DynamicCacheStartFailTest0%][GridDeploymentLocalStore] Class locally deployed: class [Ljava.lang.StackTraceElement;
[2021-02-24 13:37:11,955][INFO ][sys-#53%cache.DynamicCacheStartFailTest0%][GridDhtPartitionsExchangeFuture] Coordinator received single message [ver=AffinityTopologyVersion [topVer=2, minorTopVer=2], node=02c46b75-6648-4796-88b0-7ce6df700001, allReceived=true]
[2021-02-24 13:37:12,133][INFO ][async-runnable-runner-1][GridClusterStateProcessor] Sending deactivate cluster request with BaselineTopology null
[2021-02-24 13:37:12,137][INFO ][disco-notifier-worker-#54%cache.DynamicCacheStartFailTest0%][GridClusterStateProcessor] Received deactivate cluster request with BaselineTopology: null initiator node ID: 02c46b75-6648-4796-88b0-7ce6df700001
[2021-02-24 13:37:12,138][INFO ][disco-notifier-worker-#54%cache.DynamicCacheStartFailTest0%][GridClusterStateProcessor] Started state transition: deactivate cluster
[2021-02-24 13:37:12,139][INFO ][disco-notifier-worker-#116%cache.DynamicCacheStartFailTest1%][GridClusterStateProcessor] Received deactivate cluster request with BaselineTopology: null initiator node ID: 02c46b75-6648-4796-88b0-7ce6df700001
[2021-02-24 13:37:12,140][INFO ][disco-notifier-worker-#116%cache.DynamicCacheStartFailTest1%][GridClusterStateProcessor] Started state transition: deactivate cluster
[2021-02-24 13:37:12,156][INFO ][disco-event-worker-#125%cache.DynamicCacheStartFailTest1%][GridCachePartitionExchangeManager] Ignore event, cluster is inactive: DiscoveryCustomEvent [customMsg=DynamicCacheChangeFailureMessage [cacheNames=ArrayList [cache1], id=b062d93d771-b05182d2-c6e8-4b5b-bde2-576e1e700000, exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], discoEvt=null, nodeId=b05182d2, evt=DISCOVERY_CUSTOM_EVT], cause=class org.apache.ignite.IgniteCheckedException: Failed to complete exchange process., exchangeActions=ExchangeActions [startCaches=null, stopCaches=[cache1], startGrps=[], stopGrps=[cache1, destroy=true], resetParts=null, stateChangeRequest=null]], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=3], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163031766, loc=false, ver=2.11.0#20210224-sha1:00000000, isClient=false], topVer=2, msgTemplate=null, span=org.apache.ignite.internal.processors.tracing.NoopSpan@7e02a8f5, nodeId8=02c46b75, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1614163032148]]
[2021-02-24 13:37:12,157][INFO ][disco-event-worker-#59%cache.DynamicCacheStartFailTest0%][GridCachePartitionExchangeManager] Ignore event, cluster is inactive: DiscoveryCustomEvent [customMsg=DynamicCacheChangeFailureMessage [cacheNames=ArrayList [cache1], id=b062d93d771-b05182d2-c6e8-4b5b-bde2-576e1e700000, exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], discoEvt=DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=af52d93d771-bd608b2d-8308-4556-9f9f-91bae6049b7c, reqs=ArrayList [DynamicCacheChangeRequest [cacheName=cache1, hasCfg=true, nodeId=b05182d2-c6e8-4b5b-bde2-576e1e700000, clientStartOnly=false, stop=false, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=[cache1], stopCaches=null, startGrps=[cache1], stopGrps=[], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163032148, loc=true, ver=2.11.0#20210224-sha1:00000000, isClient=false], topVer=2, msgTemplate=null, span=org.apache.ignite.internal.processors.tracing.NoopSpan@7e02a8f5, nodeId8=b05182d2, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1614163031926]], nodeId=b05182d2, evt=DISCOVERY_CUSTOM_EVT], cause=class org.apache.ignite.IgniteCheckedException: Failed to complete exchange process., exchangeActions=ExchangeActions [startCaches=null, stopCaches=[cache1], startGrps=[], stopGrps=[cache1, destroy=true], resetParts=null, stateChangeRequest=null]], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=3], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163032148, loc=true, ver=2.11.0#20210224-sha1:00000000, isClient=false], topVer=2, msgTemplate=null, span=org.apache.ignite.internal.processors.tracing.NoopSpan@7e02a8f5, nodeId8=b05182d2, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1614163032148]]
[2021-02-24 13:37:13,453][INFO ][sys-#176%cache.DynamicCacheStartFailTest1%][GridCachePartitionExchangeManager] Delay process full message without exchange id (there is exchange in progress) [nodeId=b05182d2-c6e8-4b5b-bde2-576e1e700000]
[2021-02-24 13:37:31,954][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][diagnostic] Failed to wait for partition map exchange [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], node=b05182d2-c6e8-4b5b-bde2-576e1e700000]. Dumping pending objects that might be the cause: 
[2021-02-24 13:37:31,955][WARN ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][diagnostic] Failed to wait for partition map exchange [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], node=02c46b75-6648-4796-88b0-7ce6df700001]. Dumping pending objects that might be the cause: 
[2021-02-24 13:37:31,956][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][diagnostic] Ready affinity version: AffinityTopologyVersion [topVer=2, minorTopVer=1]
[2021-02-24 13:37:31,956][WARN ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][diagnostic] Ready affinity version: AffinityTopologyVersion [topVer=2, minorTopVer=1]
[2021-02-24 13:37:31,964][WARN ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][diagnostic] Last exchange future: GridDhtPartitionsExchangeFuture [firstDiscoEvt=DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=af52d93d771-bd608b2d-8308-4556-9f9f-91bae6049b7c, reqs=ArrayList [DynamicCacheChangeRequest [cacheName=cache1, hasCfg=true, nodeId=b05182d2-c6e8-4b5b-bde2-576e1e700000, clientStartOnly=false, stop=false, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=[cache1], stopCaches=null, startGrps=[cache1], stopGrps=[], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163031766, loc=false, ver=2.11.0#20210224-sha1:00000000, isClient=false], topVer=2, msgTemplate=null, span=o.a.i.i.processors.tracing.NoopSpan@7e02a8f5, nodeId8=02c46b75, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1614163031926]], crd=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163031766, loc=false, ver=2.11.0#20210224-sha1:00000000, isClient=false], exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], discoEvt=DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=af52d93d771-bd608b2d-8308-4556-9f9f-91bae6049b7c, reqs=ArrayList [DynamicCacheChangeRequest [cacheName=cache1, hasCfg=true, nodeId=b05182d2-c6e8-4b5b-bde2-576e1e700000, clientStartOnly=false, stop=false, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=[cache1], stopCaches=null, startGrps=[cache1], stopGrps=[], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163031766, loc=false, ver=2.11.0#20210224-sha1:00000000, isClient=false], topVer=2, msgTemplate=null, span=o.a.i.i.processors.tracing.NoopSpan@7e02a8f5, nodeId8=02c46b75, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1614163031926]], nodeId=b05182d2, evt=DISCOVERY_CUSTOM_EVT], added=true, exchangeType=ALL, initFut=GridFutureAdapter [ignoreInterrupts=false, state=DONE, res=true, hash=425645765], init=true, lastVer=null, partReleaseFut=PartitionReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[ExplicitLockReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]], AtomicUpdateReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]], DataStreamerReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]], LocalTxReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]], AllTxReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[RemoteTxReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]]]]]], exchActions=ExchangeActions [startCaches=[cache1], stopCaches=null, startGrps=[cache1], stopGrps=[], resetParts=null, stateChangeRequest=null], affChangeMsg=null, centralizedAff=false, forceAffReassignment=false, exchangeLocE=class o.a.i.IgniteCheckedException: Failed to initialize exchange locally [locNodeId=02c46b75-6648-4796-88b0-7ce6df700001], cacheChangeFailureMsgSent=false, done=false, state=SRV, registerCachesFuture=null, startTime=1614163031933, initTime=1614163031938, rebalancedInfo=null, affinityReassign=false, span=o.a.i.i.processors.tracing.NoopSpan@7e02a8f5, evtLatch=0, remaining=HashSet [b05182d2-c6e8-4b5b-bde2-576e1e700000], mergedJoinExchMsgs=null, awaitMergedMsgs=0, super=GridFutureAdapter [ignoreInterrupts=false, state=INIT, res=null, hash=1833752790]]
[2021-02-24 13:37:31,964][WARN ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][GridCachePartitionExchangeManager] First 10 pending exchange futures [total=1]
[2021-02-24 13:37:31,964][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][diagnostic] Last exchange future: GridDhtPartitionsExchangeFuture [firstDiscoEvt=DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=af52d93d771-bd608b2d-8308-4556-9f9f-91bae6049b7c, reqs=ArrayList [DynamicCacheChangeRequest [cacheName=cache1, hasCfg=true, nodeId=b05182d2-c6e8-4b5b-bde2-576e1e700000, clientStartOnly=false, stop=false, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=[cache1], stopCaches=null, startGrps=[cache1], stopGrps=[], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163051747, loc=true, ver=2.11.0#20210224-sha1:00000000, isClient=false], topVer=2, msgTemplate=null, span=o.a.i.i.processors.tracing.NoopSpan@7e02a8f5, nodeId8=b05182d2, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1614163031926]], crd=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163051747, loc=true, ver=2.11.0#20210224-sha1:00000000, isClient=false], exchId=GridDhtPartitionExchangeId [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], discoEvt=DiscoveryCustomEvent [customMsg=DynamicCacheChangeBatch [id=af52d93d771-bd608b2d-8308-4556-9f9f-91bae6049b7c, reqs=ArrayList [DynamicCacheChangeRequest [cacheName=cache1, hasCfg=true, nodeId=b05182d2-c6e8-4b5b-bde2-576e1e700000, clientStartOnly=false, stop=false, destroy=false, disabledAfterStartfalse]], exchangeActions=ExchangeActions [startCaches=[cache1], stopCaches=null, startGrps=[cache1], stopGrps=[], resetParts=null, stateChangeRequest=null], startCaches=false], affTopVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], super=DiscoveryEvent [evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163051747, loc=true, ver=2.11.0#20210224-sha1:00000000, isClient=false], topVer=2, msgTemplate=null, span=o.a.i.i.processors.tracing.NoopSpan@7e02a8f5, nodeId8=b05182d2, msg=null, type=DISCOVERY_CUSTOM_EVT, tstamp=1614163031926]], nodeId=b05182d2, evt=DISCOVERY_CUSTOM_EVT], added=true, exchangeType=ALL, initFut=GridFutureAdapter [ignoreInterrupts=false, state=DONE, res=true, hash=1191663263], init=true, lastVer=GridCacheVersion [topVer=0, order=1614163030229, nodeOrder=0], partReleaseFut=PartitionReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[ExplicitLockReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]], AtomicUpdateReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]], DataStreamerReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]], LocalTxReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]], AllTxReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[RemoteTxReleaseFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], futures=[]]]]]], exchActions=ExchangeActions [startCaches=[cache1], stopCaches=null, startGrps=[cache1], stopGrps=[], resetParts=null, stateChangeRequest=null], affChangeMsg=null, centralizedAff=false, forceAffReassignment=false, exchangeLocE=class o.a.i.IgniteCheckedException: Failed to initialize exchange locally [locNodeId=b05182d2-c6e8-4b5b-bde2-576e1e700000], cacheChangeFailureMsgSent=true, done=false, state=CRD, registerCachesFuture=null, startTime=1614163031932, initTime=1614163031938, rebalancedInfo=null, affinityReassign=false, span=o.a.i.i.processors.tracing.NoopSpan@7e02a8f5, evtLatch=0, remaining=HashSet [], mergedJoinExchMsgs=null, awaitMergedMsgs=0, super=GridFutureAdapter [ignoreInterrupts=false, state=INIT, res=null, hash=2101726312]]
[2021-02-24 13:37:31,966][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][GridCachePartitionExchangeManager] First 10 pending exchange futures [total=1]
[2021-02-24 13:37:31,966][WARN ][exchange-worker-#127%cache.DynamicCacheStartFailTest1%][GridCachePartitionExchangeManager] >>> GridDhtPartitionsExchangeFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=3], evt=DISCOVERY_CUSTOM_EVT, evtNode=TcpDiscoveryNode [id=02c46b75-6648-4796-88b0-7ce6df700001, consistentId=be23b7a5-f2c4-4ac8-9f71-bc3d0dbbdaf7, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47501], discPort=47501, order=2, intOrder=2, lastExchangeTime=1614163051747, loc=true, ver=2.11.0#20210224-sha1:00000000, isClient=false], rebalanced=false, done=false, newCrdFut=null]
[2021-02-24 13:37:31,968][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][GridCachePartitionExchangeManager] >>> GridDhtPartitionsExchangeFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=3], evt=DISCOVERY_CUSTOM_EVT, evtNode=TcpDiscoveryNode [id=02c46b75-6648-4796-88b0-7ce6df700001, consistentId=be23b7a5-f2c4-4ac8-9f71-bc3d0dbbdaf7, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47501], discPort=47501, order=2, intOrder=2, lastExchangeTime=1614163031715, loc=false, ver=2.11.0#20210224-sha1:00000000, isClient=false], rebalanced=false, done=false, newCrdFut=null]
[2021-02-24 13:37:31,972][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][diagnostic] Last 10 exchange futures (total: 3):
[2021-02-24 13:37:31,972][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][diagnostic] >>> GridDhtPartitionsExchangeFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=3], evt=DISCOVERY_CUSTOM_EVT, evtNode=TcpDiscoveryNode [id=02c46b75-6648-4796-88b0-7ce6df700001, consistentId=be23b7a5-f2c4-4ac8-9f71-bc3d0dbbdaf7, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47501], discPort=47501, order=2, intOrder=2, lastExchangeTime=1614163031715, loc=false, ver=2.11.0#20210224-sha1:00000000, isClient=false], rebalanced=false, done=false, newCrdFut=null]
[2021-02-24 13:37:31,973][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][diagnostic] >>> GridDhtPartitionsExchangeFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=2], evt=DISCOVERY_CUSTOM_EVT, evtNode=TcpDiscoveryNode [id=b05182d2-c6e8-4b5b-bde2-576e1e700000, consistentId=9bc35ede-4ab7-44cb-9d15-0d7215350ed3, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47500], discPort=47500, order=1, intOrder=1, lastExchangeTime=1614163051747, loc=true, ver=2.11.0#20210224-sha1:00000000, isClient=false], rebalanced=false, done=false, newCrdFut=null]
[2021-02-24 13:37:31,973][WARN ][exchange-worker-#61%cache.DynamicCacheStartFailTest0%][diagnostic] >>> GridDhtPartitionsExchangeFuture [topVer=AffinityTopologyVersion [topVer=2, minorTopVer=1], evt=DISCOVERY_CUSTOM_EVT, evtNode=TcpDiscoveryNode [id=02c46b75-6648-4796-88b0-7ce6df700001, consistentId=be23b7a5-f2c4-4ac8-9f71-bc3d0dbbdaf7, addrs=ArrayList [127.0.0.1], sockAddrs=HashSet [/127.0.0.1:47501], discPort=47501, order=2, intOrder=2, lastExchangeTime=1614163031715, loc=false, ver=2.11.0#20210224-sha1:00000000, isClient=false], rebalanced=true, done=true, newCrdFut=null]
{noformat}





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