You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by jo...@apache.org on 2019/03/28 07:26:15 UTC

[ignite] branch master updated: IGNITE-11011 Initialize grid disco data components at the end of node join process - Fixes #6009.

This is an automated email from the ASF dual-hosted git repository.

jokser pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new f2229b1  IGNITE-11011 Initialize grid disco data components at the end of node join process - Fixes #6009.
f2229b1 is described below

commit f2229b1b014643cf97fded543c5776f34f356b55
Author: Sergey Chugunov <se...@gmail.com>
AuthorDate: Thu Mar 28 10:25:12 2019 +0300

    IGNITE-11011 Initialize grid disco data components at the end of node join process - Fixes #6009.
    
    Signed-off-by: Pavel Kovalenko <jo...@gmail.com>
---
 .../ignite/spi/discovery/tcp/ServerImpl.java       |  50 ++++--
 ...> IgniteDiscoDataHandlingInNewClusterTest.java} |  75 ++++++--
 ...IgnitePdsDiscoDataHandlingInNewClusterTest.java | 198 +++++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite4.java   |   4 +-
 .../ignite/testsuites/IgnitePdsTestSuite.java      |   4 +
 5 files changed, 303 insertions(+), 28 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index eefecbc..0304b0f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -1005,6 +1005,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                     throw new IgniteSpiException(e);
                 }
 
+                msgWorker.nullifyDiscoData();
+
                 break;
             }
 
@@ -2781,6 +2783,12 @@ class ServerImpl extends TcpDiscoveryImpl {
         /** */
         private long lastRingMsgTime;
 
+        /** */
+        private List<DiscoveryDataPacket> joiningNodesDiscoDataList;
+
+        /** */
+        private DiscoveryDataPacket gridDiscoveryData;
+
         /**
          * @param log Logger.
          */
@@ -2907,6 +2915,12 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
         }
 
+        /** */
+        private void nullifyDiscoData() {
+            gridDiscoveryData = null;
+            joiningNodesDiscoDataList = null;
+        }
+
         /**
          * Initializes connection check frequency. Used only when failure detection timeout is enabled.
          */
@@ -4674,11 +4688,19 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     dataPacket.joiningNodeClient(msg.client());
 
-                    if (dataPacket.hasJoiningNodeData())
-                        spi.onExchange(dataPacket, U.resolveClassLoader(spi.ignite().configuration()));
+                    if (dataPacket.hasJoiningNodeData()) {
+                        if (spiState == CONNECTED) {
+                            // Node already connected to the cluster can apply joining nodes' disco data immediately
+                            spi.onExchange(dataPacket, U.resolveClassLoader(spi.ignite().configuration()));
 
-                    if (!node.isDaemon())
-                        spi.collectExchangeData(dataPacket);
+                            if (!node.isDaemon())
+                                spi.collectExchangeData(dataPacket);
+                        }
+                        else if (spiState == CONNECTING)
+                            // Node joining to the cluster should postpone applying disco data of other joiners till
+                            // receiving gridDiscoData (when NodeAddFinished message arrives)
+                            joiningNodesDiscoDataList.add(dataPacket);
+                    }
 
                     processMessageFailedNodes(msg);
                 }
@@ -4689,8 +4711,6 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
 
             if (msg.verified() && locNodeId.equals(node.id())) {
-                DiscoveryDataPacket dataPacket;
-
                 synchronized (mux) {
                     if (spiState == CONNECTING && locNode.internalOrder() != node.internalOrder()) {
                         // Initialize topology.
@@ -4772,7 +4792,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                             if (log.isDebugEnabled())
                                 log.debug("Restored topology from node added message: " + ring);
 
-                            dataPacket = msg.gridDiscoveryData();
+                            gridDiscoveryData = msg.gridDiscoveryData();
+
+                            joiningNodesDiscoDataList = new ArrayList<>();
 
                             topHist.clear();
                             topHist.putAll(msg.topologyHistory());
@@ -4804,10 +4826,6 @@ class ServerImpl extends TcpDiscoveryImpl {
                     }
                 }
 
-                // Notify outside of synchronized block.
-                if (dataPacket != null)
-                    spi.onExchange(dataPacket, U.resolveClassLoader(spi.ignite().configuration()));
-
                 processMessageFailedNodes(msg);
             }
 
@@ -4948,6 +4966,16 @@ class ServerImpl extends TcpDiscoveryImpl {
                     mux.notifyAll();
                 }
 
+                if (gridDiscoveryData != null)
+                    spi.onExchange(gridDiscoveryData, U.resolveClassLoader(spi.ignite().configuration()));
+
+                if (joiningNodesDiscoDataList != null) {
+                    for (DiscoveryDataPacket dataPacket : joiningNodesDiscoDataList)
+                        spi.onExchange(dataPacket, U.resolveClassLoader(spi.ignite().configuration()));
+                }
+
+                nullifyDiscoData();
+
                 // Discovery manager must create local joined event before spiStart completes.
                 notifyDiscovery(EVT_NODE_JOINED, topVer, locNode);
             }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDiscoveryDataHandlingInNewClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDiscoDataHandlingInNewClusterTest.java
similarity index 75%
rename from modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDiscoveryDataHandlingInNewClusterTest.java
rename to modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDiscoDataHandlingInNewClusterTest.java
index 351d660..3348554 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDiscoveryDataHandlingInNewClusterTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDiscoDataHandlingInNewClusterTest.java
@@ -16,6 +16,8 @@
  */
 package org.apache.ignite.internal.processors.cache;
 
+import java.util.Arrays;
+import java.util.Collection;
 import java.util.Map;
 import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
@@ -38,7 +40,7 @@ import org.junit.Test;
 /**
  *
  */
-public class IgniteDiscoveryDataHandlingInNewClusterTest extends GridCommonAbstractTest {
+public class IgniteDiscoDataHandlingInNewClusterTest extends GridCommonAbstractTest {
     /** */
     private static final String NODE_1_CONS_ID = "node01";
 
@@ -49,7 +51,19 @@ public class IgniteDiscoveryDataHandlingInNewClusterTest extends GridCommonAbstr
     private static final String NODE_3_CONS_ID = "node03";
 
     /** */
-    private static final String STATIC_CACHE_NAME = "staticCache";
+    private static final String NODE_4_DIFF_STATIC_CONFIG = "node04_diff_config";
+
+    /** */
+    private static final String CLIENT_WITH_DIFF_CONFIG = "clientWithDiffConfig";
+
+    /** */
+    private static final String STATIC_CACHE_NAME_1 = "staticCache1";
+
+    /** */
+    private static final String STATIC_CACHE_NAME_2 = "staticCache2";
+
+    /** */
+    private static final String STATIC_CACHE_NAME_3 = "staticCache3";
 
     /** */
     private static final String DYNAMIC_CACHE_NAME_1 = "dynamicCache1";
@@ -116,14 +130,26 @@ public class IgniteDiscoveryDataHandlingInNewClusterTest extends GridCommonAbstr
             );
         }
 
-        CacheConfiguration staticCacheCfg = new CacheConfiguration(STATIC_CACHE_NAME)
+        if (igniteInstanceName.equals(NODE_4_DIFF_STATIC_CONFIG) || igniteInstanceName.equals(CLIENT_WITH_DIFF_CONFIG))
+            cfg.setCacheConfiguration(
+                prepareStaticCacheCfg(STATIC_CACHE_NAME_1),
+                prepareStaticCacheCfg(STATIC_CACHE_NAME_2)
+            );
+        else
+            cfg.setCacheConfiguration(
+                prepareStaticCacheCfg(STATIC_CACHE_NAME_1),
+                prepareStaticCacheCfg(STATIC_CACHE_NAME_3)
+            );
+
+        return cfg;
+    }
+
+    /** */
+    private CacheConfiguration prepareStaticCacheCfg(String cacheName) {
+        return new CacheConfiguration(cacheName)
             .setGroupName(GROUP_WITH_STATIC_CACHES)
             .setAffinity(new RendezvousAffinityFunction(false, 32))
             .setNodeFilter(nodeFilter);
-
-        cfg.setCacheConfiguration(staticCacheCfg);
-
-        return cfg;
     }
 
     /**
@@ -135,31 +161,50 @@ public class IgniteDiscoveryDataHandlingInNewClusterTest extends GridCommonAbstr
      * See related ticket <a href="https://issues.apache.org/jira/browse/IGNITE-10878">IGNITE-10878</a>.
      */
     @Test
-    public void testNewClusterFiltersDiscoveryDataReceivedFromStoppedCluster() throws Exception {
+    public void testNewClusterIgnoresDiscoDataFromStopped_SameStaticConfig() throws Exception {
         IgniteEx ig0 = startGrid(NODE_1_CONS_ID);
 
         prepareDynamicCaches(ig0);
 
         IgniteEx ig1 = startGrid(NODE_2_CONS_ID);
 
-        verifyCachesAndGroups(ig1);
+        verifyCachesAndGroups(ig1, Arrays.asList(new String[] {STATIC_CACHE_NAME_1, STATIC_CACHE_NAME_3}));
 
         IgniteEx ig2 = startGrid(NODE_3_CONS_ID);
 
-        verifyCachesAndGroups(ig2);
+        verifyCachesAndGroups(ig2, Arrays.asList(new String[] {STATIC_CACHE_NAME_1, STATIC_CACHE_NAME_3}));
 
         IgniteEx client = startGrid("client01");
 
-        verifyCachesAndGroups(client);
+        verifyCachesAndGroups(client, Arrays.asList(new String[] {STATIC_CACHE_NAME_1, STATIC_CACHE_NAME_3}));
+    }
+
+    /**
+     *
+     * @throws Exception
+     */
+    @Test
+    public void testNewClusterIgnoresDiscoDataFromStopped_DifferentStaticConfig() throws Exception {
+        IgniteEx ig0 = startGrid(NODE_1_CONS_ID);
+
+        prepareDynamicCaches(ig0);
+
+        IgniteEx ig1 = startGrid(NODE_4_DIFF_STATIC_CONFIG);
+
+        verifyCachesAndGroups(ig1, Arrays.asList(new String[] {STATIC_CACHE_NAME_1, STATIC_CACHE_NAME_2}));
+
+        IgniteEx client = startGrid(CLIENT_WITH_DIFF_CONFIG);
+
+        verifyCachesAndGroups(client, Arrays.asList(new String[] {STATIC_CACHE_NAME_1, STATIC_CACHE_NAME_2}));
     }
 
     /** */
-    private void verifyCachesAndGroups(IgniteEx ignite) {
+    private void verifyCachesAndGroups(IgniteEx ignite, Collection<String> cacheNames) {
         Map<String, DynamicCacheDescriptor> caches = ignite.context().cache().cacheDescriptors();
 
-        assertEquals(2, caches.size());
-        caches.keySet().contains(GridCacheUtils.UTILITY_CACHE_NAME);
-        caches.keySet().contains(STATIC_CACHE_NAME);
+        assertEquals(cacheNames.size() + 1, caches.size());
+        assertTrue(caches.keySet().contains(GridCacheUtils.UTILITY_CACHE_NAME));
+        assertTrue(caches.keySet().containsAll(cacheNames));
 
         Map<Integer, CacheGroupDescriptor> groups = ignite.context().cache().cacheGroupDescriptors();
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDiscoDataHandlingInNewClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDiscoDataHandlingInNewClusterTest.java
new file mode 100644
index 0000000..1663952
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsDiscoDataHandlingInNewClusterTest.java
@@ -0,0 +1,198 @@
+/*
+ * 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.internal.processors.cache.persistence;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cache.affinity.AffinityFunction;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.CacheGroupDescriptor;
+import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.processors.cache.GridCacheUtils;
+import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class IgnitePdsDiscoDataHandlingInNewClusterTest extends GridCommonAbstractTest {
+    /** */
+    private static final String NODE_CONS_ID_0 = "node0";
+
+    /** */
+    private static final String NODE_CONS_ID_1 = "node1";
+
+    /** */
+    private static final String CLIENT_CONS_ID = "client0";
+
+    /** */
+    private static final String STATIC_CACHE_NAME_0 = "staticCache0";
+
+    /** */
+    private static final String DYNAMIC_CACHE_NAME_0 = "dynaCache0";
+
+    /** */
+    private static final String DYNAMIC_CACHE_NAME_1 = "dynaCache1";
+
+    /** */
+    private static final String DYNAMIC_CACHE_NAME_2 = "dynaCache2";
+
+    /** */
+    private static final String MIXED_CACHES_GROUP_NAME_0 = "mixedCachesGroup0";
+
+    /** */
+    private static final String DYNAMIC_CACHES_GROUP_NAME_1 = "dynaCachesGroup1";
+
+    /** */
+    private static final AffinityFunction AFFINITY = new RendezvousAffinityFunction(false, 16);
+
+    /** Node filter to pin dynamic caches to a specific node. */
+    private static final IgnitePredicate<ClusterNode> nodeFilter = new IgnitePredicate<ClusterNode>() {
+        @Override public boolean apply(ClusterNode node) {
+            return node.consistentId().toString().contains(NODE_CONS_ID_1);
+        }
+    };
+
+    /** */
+    private static final AtomicBoolean SHOULD_FAIL = new AtomicBoolean(false);
+
+    /** Discovery SPI aimed to fail node with it when another server node joins the topology. */
+    private TcpDiscoverySpi failingOnNodeJoinSpi = new TcpDiscoverySpi() {
+        @Override protected void startMessageProcess(TcpDiscoveryAbstractMessage msg) {
+            if (SHOULD_FAIL.get()) {
+                if (msg instanceof TcpDiscoveryNodeAddedMessage) {
+                    super.startMessageProcess(msg);
+
+                    throw new RuntimeException("Simulation of failure of node " + NODE_CONS_ID_0);
+                }
+            }
+
+            super.startMessageProcess(msg);
+        }
+    };
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        if (igniteInstanceName.equals(CLIENT_CONS_ID))
+            cfg.setClientMode(true);
+        else {
+            cfg.setDataStorageConfiguration(
+                new DataStorageConfiguration()
+                    .setDefaultDataRegionConfiguration(
+                        new DataRegionConfiguration()
+                            .setPersistenceEnabled(true)
+                    )
+            );
+
+            cfg.setCacheConfiguration(
+                new CacheConfiguration(STATIC_CACHE_NAME_0)
+                    .setGroupName(MIXED_CACHES_GROUP_NAME_0)
+                    .setAffinity(AFFINITY)
+                    .setNodeFilter(nodeFilter)
+            );
+        }
+
+        if (igniteInstanceName.equals(NODE_CONS_ID_0)) {
+            failingOnNodeJoinSpi.setIpFinder(sharedStaticIpFinder);
+            failingOnNodeJoinSpi.setJoinTimeout(60_000);
+
+            cfg.setDiscoverySpi(failingOnNodeJoinSpi);
+        }
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception
+     */
+    @Test
+    @WithSystemProperty(key = IgniteSystemProperties.IGNITE_DUMP_THREADS_ON_FAILURE, value = "false")
+    public void testNewDynamicCacheDoesntStartOnOldNode() throws Exception {
+        IgniteEx ig0 = startGrid(NODE_CONS_ID_0);
+
+        startGrid(NODE_CONS_ID_1);
+
+        ig0.cluster().active(true);
+
+        startDynamicCache(ig0, DYNAMIC_CACHE_NAME_0, MIXED_CACHES_GROUP_NAME_0);
+
+        stopGrid(NODE_CONS_ID_1);
+
+        startDynamicCache(ig0, DYNAMIC_CACHE_NAME_1, MIXED_CACHES_GROUP_NAME_0);
+
+        startDynamicCache(ig0, DYNAMIC_CACHE_NAME_2, DYNAMIC_CACHES_GROUP_NAME_1);
+
+        SHOULD_FAIL.set(true);
+
+        IgniteEx ig1 = startGrid(NODE_CONS_ID_1);
+
+        verifyCachesAndGroups(ig1);
+    }
+
+    /** */
+    private void startDynamicCache(Ignite ig, String cacheName, String groupName) {
+        ig.getOrCreateCache(new CacheConfiguration<>(cacheName)
+            .setGroupName(groupName)
+            .setAffinity(new RendezvousAffinityFunction(false, 16))
+            .setNodeFilter(nodeFilter)
+        );
+    }
+
+    /** */
+    private void verifyCachesAndGroups(IgniteEx ig) {
+        Map<String, DynamicCacheDescriptor> caches = ig.context().cache().cacheDescriptors();
+
+        assertEquals(3, caches.size());
+        assertTrue(caches.keySet().contains(GridCacheUtils.UTILITY_CACHE_NAME));
+        assertTrue(caches.keySet().contains(STATIC_CACHE_NAME_0));
+        assertTrue(caches.keySet().contains(DYNAMIC_CACHE_NAME_0));
+
+        Map<Integer, CacheGroupDescriptor> groups = ig.context().cache().cacheGroupDescriptors();
+
+        assertEquals(2, groups.size());
+
+        boolean defaultGroupFound = false;
+        boolean mixedCachesGroupFound = false;
+
+        for (CacheGroupDescriptor grpDesc : groups.values()) {
+            if (grpDesc.cacheOrGroupName().equals(GridCacheUtils.UTILITY_CACHE_NAME))
+                defaultGroupFound = true;
+            else if (grpDesc.cacheOrGroupName().equals(MIXED_CACHES_GROUP_NAME_0))
+                mixedCachesGroupFound = true;
+        }
+
+        assertTrue(String.format("Default group found: %b, mixed group found: %b",
+            defaultGroupFound,
+            mixedCachesGroupFound),
+            defaultGroupFound && mixedCachesGroupFound);
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
index 7c19909..cce27f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java
@@ -86,7 +86,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheTxPreloadNoWriteTe
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxReplicatedPeekModesTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheTxStoreValueTest;
 import org.apache.ignite.internal.processors.cache.IgniteClientCacheInitializationFailTest;
-import org.apache.ignite.internal.processors.cache.IgniteDiscoveryDataHandlingInNewClusterTest;
+import org.apache.ignite.internal.processors.cache.IgniteDiscoDataHandlingInNewClusterTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheFilterTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheMultinodeTest;
 import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartCoordinatorFailoverTest;
@@ -260,7 +260,7 @@ public class IgniteCacheTestSuite4 {
         GridTestUtils.addTestIfNeeded(suite, IgniteCacheCreatePutTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, CacheStartOnJoinTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, IgniteCacheStartTest.class, ignoredTests);
-        GridTestUtils.addTestIfNeeded(suite, IgniteDiscoveryDataHandlingInNewClusterTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, IgniteDiscoDataHandlingInNewClusterTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, CacheDiscoveryDataConcurrentJoinTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, IgniteClientCacheInitializationFailTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, IgniteCacheFailedUpdateResponseTest.class, ignoredTests);
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
index 5a052bb..41b7281 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite.java
@@ -26,6 +26,7 @@ import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCacheCon
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCacheObjectBinaryProcessorOnDiscoveryTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsDestroyCacheTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsDestroyCacheWithoutCheckpointsTest;
+import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsDiscoDataHandlingInNewClusterTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsDynamicCacheTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsSingleNodePutGetPersistenceTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsSporadicDataRecordsOnBackupTest;
@@ -101,6 +102,9 @@ public class IgnitePdsTestSuite {
         // Write throttling
         GridTestUtils.addTestIfNeeded(suite, PagesWriteThrottleSmokeTest.class, ignoredTests);
 
+        // Discovery data handling on node join and old cluster abnormal shutdown
+        GridTestUtils.addTestIfNeeded(suite, IgnitePdsDiscoDataHandlingInNewClusterTest.class, ignoredTests);
+
         // Metrics
         GridTestUtils.addTestIfNeeded(suite, FillFactorMetricTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, UsedPagesMetricTest.class, ignoredTests);