You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/07/07 09:37:08 UTC

[07/50] [abbrv] ignite git commit: Reworked cluster activation/deactivation.

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteActiveClusterTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteActiveClusterTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteActiveClusterTest.java
deleted file mode 100644
index cf68767..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteActiveClusterTest.java
+++ /dev/null
@@ -1,182 +0,0 @@
-/*
- * 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;
-
-import java.util.concurrent.Callable;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.ThreadLocalRandom;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-
-import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
-
-/**
- *
- */
-public class IgniteActiveClusterTest extends GridCommonAbstractTest {
-    /** */
-    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
-
-    /** */
-    private boolean client;
-
-    /** */
-    private boolean active = true;
-
-    /** */
-    private CacheConfiguration ccfg;
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
-
-        cfg.setClientMode(client);
-
-        cfg.setActiveOnStart(active);
-
-        if (ccfg != null) {
-            cfg.setCacheConfiguration(ccfg);
-
-            ccfg = null;
-        }
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testActivate() throws Exception {
-        active = false;
-
-        for (int i = 0; i < 3; i++) {
-            ccfg = cacheConfiguration(DEFAULT_CACHE_NAME);
-
-            startGrid(i);
-        }
-
-        ignite(0).active(true);
-
-        startGrid(3);
-
-        for (int i  = 0; i < 4; i++) {
-            IgniteCache<Integer, Integer> cache = ignite(i).cache(DEFAULT_CACHE_NAME);
-
-            for (int j = 0; j < 10; j++) {
-                ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                Integer key = rnd.nextInt(1000);
-
-                cache.put(key, j);
-
-                assertEquals((Integer)j, cache.get(key));
-            }
-        }
-    }
-
-    /**
-     * @throws Exception If failed.
-     */
-    public void testJoinAndActivate() throws Exception {
-        for (int iter = 0; iter < 3; iter++) {
-            log.info("Iteration: " + iter);
-
-            active = false;
-
-            for (int i = 0; i < 3; i++) {
-                ccfg = cacheConfiguration(DEFAULT_CACHE_NAME);
-
-                startGrid(i);
-            }
-
-            final int START_NODES = 3;
-
-            final CyclicBarrier b = new CyclicBarrier(START_NODES + 1);
-
-            IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    b.await();
-
-                    Thread.sleep(ThreadLocalRandom.current().nextLong(100) + 1);
-
-                    ignite(0).active(true);
-
-                    return null;
-                }
-            });
-
-            final AtomicInteger nodeIdx = new AtomicInteger(3);
-
-            IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
-                @Override public Void call() throws Exception {
-                    int idx = nodeIdx.getAndIncrement();
-
-                    b.await();
-
-                    startGrid(idx);
-
-                    return null;
-                }
-            }, START_NODES, "start-node");
-
-            fut1.get();
-            fut2.get();
-
-            for (int i  = 0; i < 6; i++) {
-                IgniteCache<Integer, Integer> cache = ignite(i).cache(DEFAULT_CACHE_NAME);
-
-                for (int j = 0; j < 10; j++) {
-                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
-
-                    Integer key = rnd.nextInt(1000);
-
-                    cache.put(key, j);
-
-                    assertEquals((Integer)j, cache.get(key));
-                }
-            }
-
-            stopAllGrids();
-        }
-    }
-
-    /**
-     * @param name Cache name.
-     * @return Cache configuration.
-     */
-    private CacheConfiguration cacheConfiguration(String name) {
-        CacheConfiguration ccfg = new CacheConfiguration(name);
-
-        ccfg.setWriteSynchronizationMode(FULL_SYNC);
-        ccfg.setAtomicityMode(TRANSACTIONAL);
-        ccfg.setBackups(3);
-
-        return ccfg;
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
new file mode 100644
index 0000000..8a604be
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTest.java
@@ -0,0 +1,1284 @@
+/*
+ * 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;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.MemoryConfiguration;
+import org.apache.ignite.configuration.PersistentStoreConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.IgniteClientReconnectAbstractTest;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteBiPredicate;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.TestTcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class IgniteClusterActivateDeactivateTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    static final String CACHE_NAME_PREFIX = "cache-";
+
+    /** */
+    boolean client;
+
+    /** */
+    private boolean active = true;
+
+    /** */
+    CacheConfiguration[] ccfgs;
+
+    /** */
+    private boolean testSpi;
+
+    /** */
+    private boolean testDiscoSpi;
+
+    /** */
+    private boolean testReconnectSpi;
+
+    /** */
+    private Class[] testSpiRecord;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        if (testReconnectSpi) {
+            TcpDiscoverySpi spi = new IgniteClientReconnectAbstractTest.TestTcpDiscoverySpi();
+
+            cfg.setDiscoverySpi(spi);
+
+            spi.setJoinTimeout(2 * 60_000);
+        }
+        else if (testDiscoSpi)
+            cfg.setDiscoverySpi(new TestTcpDiscoverySpi());
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        cfg.setClientMode(client);
+
+        cfg.setActiveOnStart(active);
+
+        if (ccfgs != null) {
+            cfg.setCacheConfiguration(ccfgs);
+
+            ccfgs = null;
+        }
+
+        MemoryConfiguration memCfg = new MemoryConfiguration();
+        memCfg.setPageSize(1024);
+        memCfg.setDefaultMemoryPolicySize(10 * 1024 * 1024);
+
+        cfg.setMemoryConfiguration(memCfg);
+
+        if (persistenceEnabled()) {
+            PersistentStoreConfiguration pCfg = new PersistentStoreConfiguration();
+
+            pCfg.setWalMode(WALMode.LOG_ONLY);
+
+            cfg.setPersistentStoreConfiguration(pCfg);
+        }
+
+        if (testSpi) {
+            TestRecordingCommunicationSpi spi = new TestRecordingCommunicationSpi();
+
+            if (testSpiRecord != null)
+                spi.record(testSpiRecord);
+
+            cfg.setCommunicationSpi(spi);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @return {@code True} if test with persistence.
+     */
+    protected boolean persistenceEnabled() {
+        return false;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_SingleNode() throws Exception {
+        activateSimple(1, 0, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_5_Servers() throws Exception {
+        activateSimple(5, 0, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_5_Servers2() throws Exception {
+        activateSimple(5, 0, 4);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_5_Servers_5_Clients() throws Exception {
+        activateSimple(5, 4, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
+        activateSimple(5, 4, 6);
+    }
+
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @param activateFrom Index of node stating activation.
+     * @throws Exception If failed.
+     */
+    private void activateSimple(int srvs, int clients, int activateFrom) throws Exception {
+        active = false;
+
+        final int CACHES = 2;
+
+        for (int i = 0; i < srvs + clients; i++) {
+            client = i >= srvs;
+
+            ccfgs = cacheConfigurations1();
+
+            startGrid(i);
+
+            checkNoCaches(i);
+        }
+
+        for (int i = 0; i < srvs + clients; i++)
+            assertFalse(ignite(i).active());
+
+        ignite(activateFrom).active(false); // Should be no-op.
+
+        ignite(activateFrom).active(true);
+
+        for (int i = 0; i < srvs + clients; i++)
+            assertTrue(ignite(i).active());
+
+        for (int i = 0; i < srvs + clients; i++) {
+            for (int c = 0; c < 2; c++)
+                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+
+            checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
+        }
+
+        checkCaches(srvs + clients, CACHES);
+
+        client = false;
+
+        startGrid(srvs + clients);
+
+        for (int c = 0; c < 2; c++)
+            checkCache(ignite(srvs + clients), CACHE_NAME_PREFIX + c, true);
+
+        checkCaches(srvs + clients + 1, CACHES);
+
+        client = true;
+
+        startGrid(srvs + clients + 1);
+
+        for (int c = 0; c < 2; c++)
+            checkCache(ignite(srvs + clients + 1), CACHE_NAME_PREFIX + c, false);
+
+        checkCaches(srvs + clients + 2, CACHES);
+    }
+
+    /**
+     * @param nodes Number of nodes.
+     * @param caches Number of caches.
+     */
+    final void checkCaches(int nodes, int caches) {
+        for (int i  = 0; i < nodes; i++) {
+            for (int c = 0; c < caches; c++) {
+                IgniteCache<Integer, Integer> cache = ignite(i).cache(CACHE_NAME_PREFIX + c);
+
+                for (int j = 0; j < 10; j++) {
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    Integer key = rnd.nextInt(1000);
+
+                    cache.put(key, j);
+
+                    assertEquals((Integer)j, cache.get(key));
+                }
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileActivate1_Server() throws Exception {
+        joinWhileActivate1(false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileActivate1_WithCache_Server() throws Exception {
+        joinWhileActivate1(false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileActivate1_Client() throws Exception {
+        joinWhileActivate1(true, false);
+    }
+
+    /**
+     * @param startClient If {@code true} joins client node, otherwise server.
+     * @param withNewCache If {@code true} joining node has new cache in configuration.
+     * @throws Exception If failed.
+     */
+    private void joinWhileActivate1(final boolean startClient, final boolean withNewCache) throws Exception {
+        IgniteInternalFuture<?> activeFut = startNodesAndBlockStatusChange(2, 0, 0, false);
+
+        IgniteInternalFuture<?> startFut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                client = startClient;
+
+                ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1();
+
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(ignite(1));
+
+        spi1.stopBlock();
+
+        activeFut.get();
+        startFut.get();
+
+        for (int c = 0; c < 2; c++)
+            checkCache(ignite(2), CACHE_NAME_PREFIX + c, true);
+
+        if (withNewCache) {
+            for (int i = 0; i < 3; i++) {
+                for (int c = 0; c < 4; c++)
+                    checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+            }
+        }
+
+        awaitPartitionMapExchange();
+
+        checkCaches(3, withNewCache ? 4 : 2);
+
+        client = false;
+
+        startGrid(3);
+
+        checkCaches(4, withNewCache ? 4 : 2);
+
+        client = true;
+
+        startGrid(4);
+
+        checkCaches(5, withNewCache ? 4 : 2);
+    }
+
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @param stateChangeFrom Index of node initiating changes.
+     * @param initiallyActive If {@code true} start cluster in active state (otherwise in inactive).
+     * @param blockMsgNodes Nodes whcis block exchange messages.
+     * @return State change future.
+     * @throws Exception If failed.
+     */
+    private IgniteInternalFuture<?> startNodesAndBlockStatusChange(int srvs,
+        int clients,
+        final int stateChangeFrom,
+        final boolean initiallyActive,
+        int... blockMsgNodes) throws Exception {
+        active = initiallyActive;
+        testSpi = true;
+
+        startWithCaches1(srvs, clients);
+
+        if (initiallyActive && persistenceEnabled())
+            ignite(0).active(true);
+
+        if (blockMsgNodes.length == 0)
+            blockMsgNodes = new int[]{1};
+
+        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(srvs + clients, 1);
+
+        List<TestRecordingCommunicationSpi> spis = new ArrayList<>();
+
+        for (int idx : blockMsgNodes) {
+            TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(ignite(idx));
+
+            spis.add(spi);
+
+            blockExchangeSingleMessage(spi, STATE_CHANGE_TOP_VER);
+        }
+
+        IgniteInternalFuture<?> stateChangeFut = GridTestUtils.runAsync(new Runnable() {
+            @Override public void run() {
+                ignite(stateChangeFrom).active(!initiallyActive);
+            }
+        });
+
+        for (TestRecordingCommunicationSpi spi : spis)
+            spi.waitForBlocked();
+
+        U.sleep(500);
+
+        assertFalse(stateChangeFut.isDone());
+
+        return stateChangeFut;
+    }
+
+    /**
+     * @param spi SPI.
+     * @param topVer Exchange topology version.
+     */
+    private void blockExchangeSingleMessage(TestRecordingCommunicationSpi spi, final AffinityTopologyVersion topVer) {
+        spi.blockMessages(new IgniteBiPredicate<ClusterNode, Message>() {
+            @Override public boolean apply(ClusterNode clusterNode, Message msg) {
+                if (msg instanceof GridDhtPartitionsSingleMessage) {
+                    GridDhtPartitionsSingleMessage pMsg = (GridDhtPartitionsSingleMessage)msg;
+
+                    if (pMsg.exchangeId() != null && pMsg.exchangeId().topologyVersion().equals(topVer))
+                        return true;
+                }
+
+                return false;
+            }
+        });
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileDeactivate1_Server() throws Exception {
+        joinWhileDeactivate1(false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileDeactivate1_WithCache_Server() throws Exception {
+        joinWhileDeactivate1(false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testJoinWhileDeactivate1_Client() throws Exception {
+        joinWhileDeactivate1(true, false);
+    }
+
+    /**
+     * @param startClient If {@code true} joins client node, otherwise server.
+     * @param withNewCache If {@code true} joining node has new cache in configuration.
+     * @throws Exception If failed.
+     */
+    private void joinWhileDeactivate1(final boolean startClient, final boolean withNewCache) throws Exception {
+        IgniteInternalFuture<?> activeFut = startNodesAndBlockStatusChange(2, 0, 0, true);
+
+        IgniteInternalFuture<?> startFut = GridTestUtils.runAsync(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                client = startClient;
+
+                ccfgs = withNewCache ? cacheConfigurations2() : cacheConfigurations1();
+
+                startGrid(2);
+
+                return null;
+            }
+        });
+
+        TestRecordingCommunicationSpi spi1 = TestRecordingCommunicationSpi.spi(ignite(1));
+
+        spi1.stopBlock();
+
+        activeFut.get();
+        startFut.get();
+
+        checkNoCaches(3);
+
+        ignite(2).active(true);
+
+        for (int c = 0; c < 2; c++)
+            checkCache(ignite(2), CACHE_NAME_PREFIX + c, true);
+
+        if (withNewCache) {
+            for (int i = 0; i < 3; i++) {
+                for (int c = 0; c < 4; c++)
+                    checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+            }
+        }
+
+        awaitPartitionMapExchange();
+
+        checkCaches(3, withNewCache ? 4 : 2);
+
+        client = false;
+
+        startGrid(3);
+
+        checkCaches(4, withNewCache ? 4 : 2);
+
+        client = true;
+
+        startGrid(4);
+
+        checkCaches(5, withNewCache ? 4 : 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentJoinAndActivate() throws Exception {
+        for (int iter = 0; iter < 3; iter++) {
+            log.info("Iteration: " + iter);
+
+            active = false;
+
+            for (int i = 0; i < 3; i++) {
+                ccfgs = cacheConfigurations1();
+
+                startGrid(i);
+            }
+
+            final int START_NODES = 3;
+
+            final CyclicBarrier b = new CyclicBarrier(START_NODES + 1);
+
+            IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    b.await();
+
+                    Thread.sleep(ThreadLocalRandom.current().nextLong(100) + 1);
+
+                    ignite(0).active(true);
+
+                    return null;
+                }
+            });
+
+            final AtomicInteger nodeIdx = new AtomicInteger(3);
+
+            IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    int idx = nodeIdx.getAndIncrement();
+
+                    b.await();
+
+                    startGrid(idx);
+
+                    return null;
+                }
+            }, START_NODES, "start-node");
+
+            fut1.get();
+            fut2.get();
+
+            checkCaches(6, 2);
+
+            afterTest();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_SingleNode() throws Exception {
+        deactivateSimple(1, 0, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_5_Servers() throws Exception {
+        deactivateSimple(5, 0, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_5_Servers2() throws Exception {
+        deactivateSimple(5, 0, 4);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_5_Servers_5_Clients() throws Exception {
+        deactivateSimple(5, 4, 0);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateSimple_5_Servers_5_Clients_FromClient() throws Exception {
+        deactivateSimple(5, 4, 6);
+    }
+
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @param deactivateFrom Index of node stating deactivation.
+     * @throws Exception If failed.
+     */
+    private void deactivateSimple(int srvs, int clients, int deactivateFrom) throws Exception {
+        active = true;
+
+        final int CACHES = 2;
+
+        for (int i = 0; i < srvs + clients; i++) {
+            client = i >= srvs;
+
+            ccfgs = cacheConfigurations1();
+
+            startGrid(i);
+        }
+
+        if (persistenceEnabled())
+            ignite(deactivateFrom).active(true);
+
+        ignite(deactivateFrom).active(true); // Should be no-op.
+
+        checkCaches(srvs + clients, CACHES);
+
+        for (int i = 0; i < srvs + clients; i++)
+            assertTrue(ignite(i).active());
+
+        ignite(deactivateFrom).active(false);
+
+        for (int i = 0; i < srvs + clients; i++)
+            assertFalse(ignite(i).active());
+
+        checkNoCaches(srvs + clients);
+
+        client = false;
+
+        startGrid(srvs + clients);
+
+        checkNoCaches(srvs + clients + 1);
+
+        client = true;
+
+        startGrid(srvs + clients + 1);
+
+        checkNoCaches(srvs + clients + 2);
+
+        for (int i = 0; i < srvs + clients + 2; i++)
+            assertFalse(ignite(i).active());
+
+        ignite(deactivateFrom).active(true);
+
+        for (int i = 0; i < srvs + clients + 2; i++) {
+            assertTrue(ignite(i).active());
+
+            checkCache(ignite(i), CU.UTILITY_CACHE_NAME, true);
+        }
+
+        for (int i = 0; i < srvs; i++) {
+            for (int c = 0; c < 2; c++)
+                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+        }
+
+        checkCaches1(srvs + clients + 2);
+    }
+
+    /**
+     * @param srvs Number of servers.
+     * @param clients Number of clients.
+     * @throws Exception If failed.
+     */
+    private void startWithCaches1(int srvs, int clients) throws Exception {
+        for (int i = 0; i < srvs + clients; i++) {
+            ccfgs = cacheConfigurations1();
+
+            client = i >= srvs;
+
+            startGrid(i);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterActive() throws Exception {
+        testReconnectSpi = true;
+
+        ccfgs = cacheConfigurations1();
+
+        final int SRVS = 3;
+        final int CLIENTS = 3;
+
+        startWithCaches1(SRVS, CLIENTS);
+
+        if (persistenceEnabled())
+            ignite(0).active(true);
+
+        Ignite srv = ignite(0);
+        Ignite client = ignite(SRVS);
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, null);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        this.client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        this.client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkCaches1(SRVS + CLIENTS + 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterInactive() throws Exception {
+        testReconnectSpi = true;
+
+        active = false;
+
+        final int SRVS = 3;
+        final int CLIENTS = 3;
+
+        startWithCaches1(SRVS, CLIENTS);
+
+        Ignite srv = ignite(0);
+        Ignite client = ignite(SRVS);
+
+        checkNoCaches(SRVS + CLIENTS);
+
+        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, null);
+
+        checkNoCaches(SRVS + CLIENTS);
+
+        ignite(0).active(true);
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        this.client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        this.client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkCaches1(SRVS + CLIENTS);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterDeactivated() throws Exception {
+        clientReconnectClusterDeactivated(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterDeactivateInProgress() throws Exception {
+        clientReconnectClusterDeactivated(true);
+    }
+
+    /**
+     * @param transition If {@code true} client reconnects while cluster state transition is in progress.
+     * @throws Exception If failed.
+     */
+    private void clientReconnectClusterDeactivated(final boolean transition) throws Exception {
+        testReconnectSpi = true;
+        testSpi = transition;
+
+        final int SRVS = 3;
+        final int CLIENTS = 3;
+
+        startWithCaches1(SRVS, CLIENTS);
+
+        final Ignite srv = ignite(0);
+        Ignite client = ignite(SRVS);
+
+        if (persistenceEnabled())
+            ignite(0).active(true);
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(SRVS + CLIENTS + 1, 1);
+
+        final TestRecordingCommunicationSpi spi1 = transition ? TestRecordingCommunicationSpi.spi(ignite(1)) : null;
+
+        final AtomicReference<IgniteInternalFuture> stateFut = new AtomicReference<>();
+
+        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, new Runnable() {
+            @Override public void run() {
+                if (transition) {
+                    blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER);
+
+                    stateFut.set(GridTestUtils.runAsync(new Runnable() {
+                        @Override public void run() {
+                            srv.active(false);
+                        }
+                    }, "deactivate"));
+
+                    try {
+                        U.sleep(500);
+                    }
+                    catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+                else
+                    srv.active(false);
+            }
+        });
+
+        checkCache(client, CACHE_NAME_PREFIX + 0, false);
+
+        if (transition) {
+            assertFalse(stateFut.get().isDone());
+
+            assertFalse(client.active());
+
+            spi1.waitForBlocked();
+
+            spi1.stopBlock();
+
+            stateFut.get().get();
+        }
+
+        checkNoCaches(SRVS + CLIENTS);
+
+        ignite(0).active(true);
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        assertTrue(client.active());
+
+        checkCaches1(SRVS + CLIENTS);
+
+        checkCache(client, CACHE_NAME_PREFIX + 0, true);
+
+        this.client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        this.client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkCaches1(SRVS + CLIENTS + 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterActivated() throws Exception {
+        clientReconnectClusterActivated(false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testClientReconnectClusterActivateInProgress() throws Exception {
+        clientReconnectClusterActivated(true);
+    }
+
+    /**
+     * @param transition If {@code true} client reconnects while cluster state transition is in progress.
+     * @throws Exception If failed.
+     */
+    private void clientReconnectClusterActivated(final boolean transition) throws Exception {
+        testReconnectSpi = true;
+        testSpi = transition;
+
+        active = false;
+
+        final int SRVS = 3;
+        final int CLIENTS = 3;
+
+        startWithCaches1(SRVS, CLIENTS);
+
+        final Ignite srv = ignite(0);
+        Ignite client = ignite(SRVS);
+
+        checkNoCaches(SRVS + CLIENTS);
+
+        final AffinityTopologyVersion STATE_CHANGE_TOP_VER = new AffinityTopologyVersion(SRVS + CLIENTS + 1, 1);
+
+        final TestRecordingCommunicationSpi spi1 = transition ? TestRecordingCommunicationSpi.spi(ignite(1)) : null;
+
+        final AtomicReference<IgniteInternalFuture> stateFut = new AtomicReference<>();
+
+        IgniteClientReconnectAbstractTest.reconnectClientNode(log, client, srv, new Runnable() {
+            @Override public void run() {
+                if (transition) {
+                    blockExchangeSingleMessage(spi1, STATE_CHANGE_TOP_VER);
+
+                    stateFut.set(GridTestUtils.runAsync(new Runnable() {
+                        @Override public void run() {
+                            srv.active(true);
+                        }
+                    }, "activate"));
+
+                    try {
+                        U.sleep(500);
+                    }
+                    catch (Exception e) {
+                        e.printStackTrace();
+                    }
+                }
+                else
+                    srv.active(true);
+            }
+        });
+
+        checkCache(client, CACHE_NAME_PREFIX + 0, !transition);
+
+        if (transition) {
+            assertFalse(stateFut.get().isDone());
+
+            assertFalse(client.active());
+
+            spi1.waitForBlocked();
+
+            spi1.stopBlock();
+
+            stateFut.get().get();
+        }
+
+        checkCache(client, CU.UTILITY_CACHE_NAME, true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        checkCache(client, CACHE_NAME_PREFIX + 0, true);
+
+        this.client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        this.client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkCaches1(SRVS + CLIENTS + 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInactiveTopologyChanges() throws Exception {
+        testSpi = true;
+
+        testSpiRecord = new Class[]{GridDhtPartitionsSingleMessage.class, GridDhtPartitionsFullMessage.class};
+
+        active = false;
+
+        final int SRVS = 4;
+        final int CLIENTS = 4;
+
+        startWithCaches1(SRVS, CLIENTS);
+
+        checkRecordedMessages(false);
+
+        for (int i = 0; i < 2; i++) {
+            stopGrid(i);
+
+            client = false;
+
+            startGrid(i);
+        }
+
+        checkRecordedMessages(false);
+
+        for (int i = 0; i < 2; i++) {
+            stopGrid(SRVS + i);
+
+            client = true;
+
+            startGrid(SRVS + i);
+        }
+
+        checkRecordedMessages(false);
+
+        ignite(0).active(true);
+
+        checkCaches1(SRVS + CLIENTS);
+
+        checkRecordedMessages(true);
+
+        client = false;
+
+        startGrid(SRVS + CLIENTS);
+
+        client = true;
+
+        startGrid(SRVS + CLIENTS + 1);
+
+        checkRecordedMessages(true);
+
+        checkCaches1(SRVS + CLIENTS + 2);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateFailover1() throws Exception {
+        stateChangeFailover1(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateFailover1() throws Exception {
+        stateChangeFailover1(false);
+    }
+
+    /**
+     * @param activate If {@code true} tests activation, otherwise deactivation.
+     * @throws Exception If failed.
+     */
+    private void stateChangeFailover1(boolean activate) throws Exception {
+        // Nodes 1 and 4 do not reply to coordinator.
+        IgniteInternalFuture<?> fut = startNodesAndBlockStatusChange(4, 4, 3, !activate, 1, 4);
+
+        client = false;
+
+        // Start one more node while transition is in progress.
+        IgniteInternalFuture startFut = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                startGrid(8);
+
+                return null;
+            }
+        }, "start-node");
+
+        U.sleep(500);
+
+        stopGrid(getTestIgniteInstanceName(1), true, false);
+        stopGrid(getTestIgniteInstanceName(4), true, false);
+
+        fut.get();
+
+        startFut.get();
+
+        client = false;
+
+        startGrid(1);
+
+        client = true;
+
+        startGrid(4);
+
+        if (!activate) {
+            checkNoCaches(9);
+
+            ignite(0).active(true);
+        }
+
+        checkCaches1(9);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateFailover2() throws Exception {
+        stateChangeFailover2(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateFailover2() throws Exception {
+        stateChangeFailover2(false);
+    }
+
+    /**
+     * @param activate If {@code true} tests activation, otherwise deactivation.
+     * @throws Exception If failed.
+     */
+    private void stateChangeFailover2(boolean activate) throws Exception {
+        // Nodes 1 and 4 do not reply to coordinator.
+        IgniteInternalFuture<?> fut = startNodesAndBlockStatusChange(4, 4, 3, !activate, 1, 4);
+
+        client = false;
+
+        // Start one more nodes while transition is in progress.
+        IgniteInternalFuture startFut1 = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                startGrid(8);
+
+                return null;
+            }
+        }, "start-node1");
+        IgniteInternalFuture startFut2 = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                startGrid(9);
+
+                return null;
+            }
+        }, "start-node2");
+
+        U.sleep(500);
+
+        // Stop coordinator.
+        stopGrid(0);
+
+        stopGrid(getTestIgniteInstanceName(1), true, false);
+        stopGrid(getTestIgniteInstanceName(4), true, false);
+
+        fut.get();
+
+        startFut1.get();
+        startFut2.get();
+
+        client = false;
+
+        startGrid(0);
+        startGrid(1);
+
+        client = true;
+
+        startGrid(4);
+
+        if (!activate) {
+            checkNoCaches(10);
+
+            ignite(0).active(true);
+        }
+
+        checkCaches1(10);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateFailover3() throws Exception {
+        stateChangeFailover3(true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeactivateFailover3() throws Exception {
+        stateChangeFailover3(false);
+    }
+
+    /**
+     * @param activate If {@code true} tests activation, otherwise deactivation.
+     * @throws Exception If failed.
+     */
+    private void stateChangeFailover3(boolean activate) throws Exception {
+        testDiscoSpi = true;
+
+        startNodesAndBlockStatusChange(4, 0, 0, !activate);
+
+        client = false;
+
+        IgniteInternalFuture startFut1 = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                startGrid(4);
+
+                return null;
+            }
+        }, "start-node1");
+
+        IgniteInternalFuture startFut2 = GridTestUtils.runAsync(new Callable() {
+            @Override public Object call() throws Exception {
+                startGrid(5);
+
+                return null;
+            }
+        }, "start-node2");
+
+        U.sleep(1000);
+
+        // Stop all nodes participating in state change and not allow last node to finish exchange.
+        for (int i = 0; i < 4; i++)
+            ((TestTcpDiscoverySpi)ignite(i).configuration().getDiscoverySpi()).simulateNodeFailure();
+
+        for (int i = 0; i < 4; i++)
+            stopGrid(getTestIgniteInstanceName(i), true, false);
+
+        startFut1.get();
+        startFut2.get();
+
+        assertFalse(ignite(4).active());
+        assertFalse(ignite(5).active());
+
+        ignite(4).active(true);
+
+        for (int i = 0; i < 4; i++)
+            startGrid(i);
+
+        checkCaches1(6);
+    }
+
+    /**
+     * @param exp If {@code true} there should be recorded messages.
+     */
+    private void checkRecordedMessages(boolean exp) {
+        for (Ignite node : G.allGrids()) {
+            List<Object> recorded =
+                TestRecordingCommunicationSpi.spi(node).recordedMessages(false);
+
+            if (exp)
+                assertFalse(F.isEmpty(recorded));
+            else
+                assertTrue(F.isEmpty(recorded));
+        }
+    }
+
+    /**
+     * @param nodes Expected nodes number.
+     */
+    private void checkCaches1(int nodes) {
+        checkCaches(nodes, 2);
+    }
+
+    /**
+     * @return Cache configurations.
+     */
+    final CacheConfiguration[] cacheConfigurations1() {
+        CacheConfiguration[] ccfgs = new CacheConfiguration[2];
+
+        ccfgs[0] = cacheConfiguration(CACHE_NAME_PREFIX + 0, ATOMIC);
+        ccfgs[1] = cacheConfiguration(CACHE_NAME_PREFIX + 1, TRANSACTIONAL);
+
+        return ccfgs;
+    }
+
+    /**
+     * @return Cache configurations.
+     */
+    final CacheConfiguration[] cacheConfigurations2() {
+        CacheConfiguration[] ccfgs = new CacheConfiguration[4];
+
+        ccfgs[0] = cacheConfiguration(CACHE_NAME_PREFIX + 0, ATOMIC);
+        ccfgs[1] = cacheConfiguration(CACHE_NAME_PREFIX + 1, TRANSACTIONAL);
+        ccfgs[2] = cacheConfiguration(CACHE_NAME_PREFIX + 2, ATOMIC);
+        ccfgs[3] = cacheConfiguration(CACHE_NAME_PREFIX + 3, TRANSACTIONAL);
+
+        return ccfgs;
+    }
+
+    /**
+     * @param name Cache name.
+     * @param atomicityMode Atomicity mode.
+     * @return Cache configuration.
+     */
+    protected final CacheConfiguration cacheConfiguration(String name, CacheAtomicityMode atomicityMode) {
+        CacheConfiguration ccfg = new CacheConfiguration(name);
+
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setAtomicityMode(atomicityMode);
+        ccfg.setBackups(1);
+
+        return ccfg;
+    }
+
+    /**
+     * @param cacheName Cache name.
+     * @param node Node.
+     * @param exp {@code True} if expect that cache is started on node.
+     */
+    void checkCache(Ignite node, String cacheName, boolean exp) {
+        GridCacheAdapter cache = ((IgniteKernal)node).context().cache().internalCache(cacheName);
+
+        if (exp)
+            assertNotNull("Cache not found [cache=" + cacheName + ", node=" + node.name() + ']', cache);
+        else
+            assertNull("Unexpected cache found [cache=" + cacheName + ", node=" + node.name() + ']', cache);
+    }
+
+    /**
+     * @param nodes Number of nodes.
+     */
+    final void checkNoCaches(int nodes) {
+        for (int i = 0; i < nodes; i++) {
+            GridCacheProcessor cache = ((IgniteKernal)ignite(i)).context().cache();
+
+            assertTrue(cache.caches().isEmpty());
+            assertTrue(cache.internalCaches().isEmpty());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
new file mode 100644
index 0000000..4a19aa8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteClusterActivateDeactivateTestWithPersistence.java
@@ -0,0 +1,197 @@
+/*
+ * 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;
+
+import java.util.Arrays;
+import java.util.LinkedHashMap;
+import java.util.Map;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.testframework.GridTestUtils;
+
+/**
+ *
+ */
+public class IgniteClusterActivateDeactivateTestWithPersistence extends IgniteClusterActivateDeactivateTest {
+    /** {@inheritDoc} */
+    @Override protected boolean persistenceEnabled() {
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        GridTestUtils.deleteDbFiles();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        GridTestUtils.deleteDbFiles();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCachesRestore_SingleNode() throws Exception {
+        activateCachesRestore(1, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCachesRestore_SingleNode_WithNewCaches() throws Exception {
+        activateCachesRestore(1, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCachesRestore_5_Servers() throws Exception {
+        activateCachesRestore(5, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCachesRestore_5_Servers_WithNewCaches() throws Exception {
+        activateCachesRestore(5, false);
+    }
+
+    /**
+     * @param srvs Number of server nodes.
+     * @param withNewCaches If {@code true} then after restart has new caches in configuration.
+     * @throws Exception If failed.
+     */
+    private void activateCachesRestore(int srvs, boolean withNewCaches) throws Exception {
+        Ignite srv = startGrids(srvs);
+
+        srv.active(true);
+
+        srv.createCaches(Arrays.asList(cacheConfigurations1()));
+
+        Map<Integer, Integer> cacheData = new LinkedHashMap<>();
+
+        for (int i = 1; i <= 100; i++) {
+            for (CacheConfiguration ccfg : cacheConfigurations1()) {
+                srv.cache(ccfg.getName()).put(-i, i);
+
+                cacheData.put(-i, i);
+            }
+        }
+
+        stopAllGrids();
+
+        for (int i = 0; i < srvs; i++) {
+            if (withNewCaches)
+                ccfgs = cacheConfigurations2();
+
+            startGrid(i);
+        }
+
+        srv = ignite(0);
+
+        checkNoCaches(srvs);
+
+        srv.active(true);
+
+        final int CACHES = withNewCaches ? 4 : 2;
+
+        for (int i = 0; i < srvs; i++) {
+            for (int c = 0; c < CACHES; c++)
+                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+        }
+
+        for (CacheConfiguration ccfg : cacheConfigurations1())
+            checkCacheData(cacheData, ccfg.getName());
+
+        checkCaches(srvs, CACHES);
+
+        int nodes = srvs;
+
+        client = false;
+
+        startGrid(nodes++);
+
+        for (int i = 0; i < nodes; i++) {
+            for (int c = 0; c < CACHES; c++)
+                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+        }
+
+        checkCaches(nodes, CACHES);
+
+        client = true;
+
+        startGrid(nodes++);
+
+        for (int c = 0; c < CACHES; c++)
+            checkCache(ignite(nodes - 1), CACHE_NAME_PREFIX + c, false);
+
+        checkCaches(nodes, CACHES);
+
+        for (int i = 0; i < nodes; i++) {
+            for (int c = 0; c < CACHES; c++)
+                checkCache(ignite(i), CACHE_NAME_PREFIX + c, true);
+        }
+
+        for (CacheConfiguration ccfg : cacheConfigurations1())
+            checkCacheData(cacheData, ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testActivateCacheRestoreConfigurationConflict() throws Exception {
+        final int SRVS = 3;
+
+        Ignite srv = startGrids(SRVS);
+
+        srv.active(true);
+
+        CacheConfiguration ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME);
+
+        srv.createCache(ccfg);
+
+        stopAllGrids();
+
+        ccfg = new CacheConfiguration(DEFAULT_CACHE_NAME + 1);
+
+        ccfg.setGroupName(DEFAULT_CACHE_NAME);
+
+        ccfgs = new CacheConfiguration[]{ccfg};
+
+        startGrids(SRVS);
+
+        try {
+            ignite(0).active(true);
+
+            fail();
+        }
+        catch (IgniteException e) {
+            // Expected error.
+        }
+
+        for (int i = 0; i < SRVS; i++)
+            assertFalse(ignite(i).active());
+
+        checkNoCaches(SRVS);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
index fdf5350..566860d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDaemonNodeMarshallerCacheTest.java
@@ -22,9 +22,6 @@ import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteInternalFuture;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.lang.IgniteCallable;
 import org.apache.ignite.resources.IgniteInstanceResource;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -107,13 +104,6 @@ public class IgniteDaemonNodeMarshallerCacheTest extends GridCommonAbstractTest
 
         awaitPartitionMapExchange();
 
-        // Workaround for IGNITE-1365.
-        IgniteInternalFuture<?> fut = ((IgniteKernal) daemonNode).context().cache().context().exchange().
-            affinityReadyFuture(new AffinityTopologyVersion(2, 0));
-
-        if (fut != null)
-            fut.get();
-
         TestClass1 res1 = daemonNode.compute(daemonNode.cluster().forRemotes()).call(new TestCallable1());
 
         assertNotNull(res1);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
index 4dfe69b..665bb56 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpPageStoreManager.java
@@ -152,11 +152,6 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart(boolean reconnect) throws IgniteCheckedException {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void onKernalStop(boolean cancel) {
         // No-op.
     }
@@ -177,8 +172,7 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
     }
 
     /** {@inheritDoc} */
-    @Override public void storeCacheData(CacheGroupDescriptor grpDesc,
-        StoredCacheData cacheData) throws IgniteCheckedException {
+    @Override public void storeCacheData(StoredCacheData cacheData) throws IgniteCheckedException {
         // No-op.
     }
 
@@ -189,11 +183,11 @@ public class NoOpPageStoreManager implements IgnitePageStoreManager {
 
     /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext kctx) {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void onDeActivate(GridKernalContext kctx) {
-
+        // No-op.
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
index 0ef593f..72450b8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/pagemem/NoOpWALManager.java
@@ -43,7 +43,7 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager {
 
     /** {@inheritDoc} */
     @Override public void resumeLogging(WALPointer ptr) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
@@ -83,42 +83,37 @@ public class NoOpWALManager implements IgniteWriteAheadLogManager {
 
     /** {@inheritDoc} */
     @Override public void start(GridCacheSharedContext cctx) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void stop(boolean cancel) {
-
-    }
-
-    /** {@inheritDoc} */
-    @Override public void onKernalStart(boolean reconnect) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void onKernalStop(boolean cancel) {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void onDisconnected(IgniteFuture reconnectFut) {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void printMemoryStats() {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
     @Override public void onActivate(GridKernalContext kctx) throws IgniteCheckedException {
-
+        // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public void onDeActivate(GridKernalContext kctx) throws IgniteCheckedException {
-
+    @Override public void onDeActivate(GridKernalContext kctx) {
+        // No-op.
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
index 9fa6f7c..675aca5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/AbstractNodeJoinTemplate.java
@@ -53,123 +53,160 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
     /** Cache 2. */
     protected static final String cache2 = "cache2";
 
-    //Todo Cache with node filter.
+    /** */
     protected static final String cache3 = "cache3";
 
+    /** */
     protected static final String cache4 = "cache4";
 
-    protected static final String cache5 = "cache5";
+    /** */
+    private static final String cache5 = "cache5";
 
     /** Caches info. */
-    public static final String CACHES_INFO = "cachesInfo";
+    private static final String CACHES_INFO = "cachesInfo";
 
     /** Registered caches. */
-    public static final String REGISTERED_CACHES = "registeredCaches";
+    private static final String REGISTERED_CACHES = "registeredCaches";
 
     /** Caches. */
     public static final String CACHES = "caches";
 
     /**
-     * @param ig Ig.
+     * @param ig Node.
+     * @return Cache descriptors.
      */
     protected static Map<String, DynamicCacheDescriptor> cacheDescriptors(IgniteEx ig) {
         return field((Object)field(ig.context().cache(), CACHES_INFO), REGISTERED_CACHES);
     }
 
     /**
-     * @param ig Ig.
+     * @param ig Node.
+     * @return Node caches.
      */
     protected static Map<String, GridCacheAdapter> caches(IgniteEx ig){
         return field(ig.context().cache(), CACHES);
     }
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder withOutConfigurationTemplate() throws Exception;
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder staticCacheConfigurationOnJoinTemplate() throws Exception;
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder staticCacheConfigurationInClusterTemplate() throws Exception;
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder staticCacheConfigurationSameOnBothTemplate() throws Exception;
 
     /**
-     *
+     * @return Test builder.
+     * @throws Exception If failed.
      */
     public abstract JoinNodeTestPlanBuilder staticCacheConfigurationDifferentOnBothTemplate() throws Exception;
 
     // Client node join.
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientWithOutConfigurationTemplate() throws Exception;
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationOnJoinTemplate() throws Exception;
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationInClusterTemplate() throws Exception;
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationSameOnBothTemplate() throws Exception;
 
+    /**
+     * @return Test builder.
+     * @throws Exception If failed.
+     */
     public abstract JoinNodeTestPlanBuilder joinClientStaticCacheConfigurationDifferentOnBothTemplate() throws Exception;
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public abstract void testJoinWithOutConfiguration() throws Exception;
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public abstract void testStaticCacheConfigurationOnJoin() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testStaticCacheConfigurationInCluster() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testStaticCacheConfigurationSameOnBoth() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testStaticCacheConfigurationDifferentOnBoth() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testJoinClientWithOutConfiguration() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testJoinClientStaticCacheConfigurationOnJoin() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testJoinClientStaticCacheConfigurationInCluster() throws Exception;
 
     /**
+     * @throws Exception If failed.
      *
      */
     public abstract void testJoinClientStaticCacheConfigurationSameOnBoth() throws Exception;
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public abstract void testJoinClientStaticCacheConfigurationDifferentOnBoth() throws Exception;
 
+    /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
         super.beforeTest();
 
@@ -178,6 +215,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         deleteRecursively(U.resolveWorkDirectory(U.defaultWorkDirectory(), "db", false));
     }
 
+    /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
 
@@ -188,6 +226,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
     /**
      * @param idx Index.
+     * @return Ignite instance name.
      */
     protected String name(int idx) {
         return getTestIgniteInstanceName(idx);
@@ -195,6 +234,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
     /**
      * @param name Name.
+     * @return Igntie configuration.
+     * @throws Exception If failed.
      */
     protected IgniteConfiguration cfg(String name) throws Exception {
         try {
@@ -206,7 +247,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
     }
 
     /**
-     *
+     * @return Test builder.
      */
     protected JoinNodeTestPlanBuilder builder() {
         return JoinNodeTestPlanBuilder.builder();
@@ -214,13 +255,14 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
     /**
      * @param cfgs Cfgs.
+     * @return Configurations.
      */
-    protected static <T> T[] buildConfiguration(T... cfgs) {
+    private static <T> T[] buildConfiguration(T... cfgs) {
         return cfgs;
     }
 
     /**
-     *
+     * @return Cache configuration.
      */
     protected CacheConfiguration atomicCfg() {
         return new CacheConfiguration(cache1)
@@ -228,6 +270,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
     }
 
     /**
+     * @return Cache configuration.
      *
      */
     protected CacheConfiguration transactionCfg() {
@@ -236,7 +279,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
     }
 
     /**
-     *
+     * @return Cache configurations.
      */
     protected CacheConfiguration[] allCacheConfigurations() {
         return buildConfiguration(atomicCfg(), transactionCfg());
@@ -309,6 +352,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         /** After de activate. */
         private Runnable afterDeActivate = Noop;
 
+        /** */
         private IgniteCallable<List<CacheConfiguration>> dynamicCacheStart =
             new IgniteCallable<List<CacheConfiguration>>() {
                 @Override public List<CacheConfiguration> call() throws Exception {
@@ -316,6 +360,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
                 }
             };
 
+        /** */
         private IgniteCallable<List<String>> dynamicCacheStop =
             new IgniteCallable<List<String>>() {
                 @Override public List<String> call() throws Exception {
@@ -323,15 +368,19 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
                 }
             };
 
+        /** */
         private Runnable afterDynamicCacheStarted = Noop;
 
+        /** */
         private Runnable afterDynamicCacheStopped = Noop;
 
         /** End. */
         private Runnable end = Noop;
 
         /**
-         *
+         * @param cfgs Configurations.
+         * @return Test builder.
+         * @throws Exception If failed.
          */
         public JoinNodeTestPlanBuilder clusterConfiguration(IgniteConfiguration... cfgs) throws Exception {
             clusterCfg = cfgs;
@@ -359,7 +408,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param cfg Configuration.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder nodeConfiguration(IgniteConfiguration cfg) {
             nodeCfg = cfg;
@@ -382,6 +432,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
         /**
          * @param func Func.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder nodeConfiguration(
             IgniteClosure<IgniteConfiguration, IgniteConfiguration> func
@@ -393,7 +444,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param r Cluster start callback.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder afterClusterStarted(Runnable r) {
             strPlanBuilder.append("Check after cluster start\n");
@@ -404,7 +456,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param r Node join callback.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder afterNodeJoin(Runnable r) {
             strPlanBuilder.append("Check after node join")
@@ -416,7 +469,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param state State after join.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder stateAfterJoin(boolean state) {
             strPlanBuilder.append("Check state on all nodes after join, must be ")
@@ -429,7 +483,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param r Activate callback.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder afterActivate(Runnable r) {
             strPlanBuilder.append("Check after activate")
@@ -441,7 +496,8 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @param r Deactivate callback.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder afterDeActivate(Runnable r) {
             strPlanBuilder.append("Check after deActivate")
@@ -452,6 +508,10 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             return this;
         }
 
+        /**
+         * @param caches Callback.
+         * @return Test builder.
+         */
         public JoinNodeTestPlanBuilder dynamicCacheStart(IgniteCallable<List<CacheConfiguration>> caches){
             strPlanBuilder.append("Dynamic caches start")
                 .append("\n");
@@ -461,6 +521,10 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             return this;
         }
 
+        /**
+         * @param r Cache start callback.
+         * @return Test builder.
+         */
         public JoinNodeTestPlanBuilder afterDynamicCacheStarted(Runnable r){
             strPlanBuilder.append("Check after dynamic caches start")
                 .append("\n");
@@ -470,6 +534,10 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             return this;
         }
 
+        /**
+         * @param caches Callback.
+         * @return Test builder.
+         */
         public JoinNodeTestPlanBuilder dynamicCacheStop(IgniteCallable<List<String>> caches){
             strPlanBuilder.append("Dynamic caches stop")
                 .append("\n");
@@ -479,6 +547,10 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             return this;
         }
 
+        /**
+         * @param r Callback.
+         * @return Test builder.
+         */
         public JoinNodeTestPlanBuilder afterDynamicCacheStopped(Runnable r){
             strPlanBuilder.append("Check after dynamic caches stop")
                 .append("\n");
@@ -490,6 +562,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
         /**
          * @param end End.
+         * @return Test builder.
          */
         public JoinNodeTestPlanBuilder setEnd(Runnable end) {
             strPlanBuilder.append("Check before stop")
@@ -501,7 +574,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @throws Exception If failed.
          */
         public void execute() throws Exception {
             try {
@@ -611,12 +684,19 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
             }
         }
 
+        /**
+         * @param ig Node.
+         * @return Next minor version.
+         */
         private AffinityTopologyVersion nextMinorVersion(IgniteEx ig){
             AffinityTopologyVersion cur = ig.context().discovery().topologyVersionEx();
 
-           return new AffinityTopologyVersion(cur.topologyVersion(), cur.minorTopologyVersion() + 1);
+           return cur.nextMinorVersion();
         }
 
+        /**
+         * @param ver Version.
+         */
         private void awaitTopologyVersion(final AffinityTopologyVersion ver){
             onAllNode(new CI1<IgniteEx>() {
                 @Override public void apply(IgniteEx ig) {
@@ -643,7 +723,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @return Started nodes.
          */
         protected List<IgniteEx> grids() {
             List<IgniteEx> res = new ArrayList<>();
@@ -655,14 +735,14 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @return Test builder.
          */
         public static JoinNodeTestPlanBuilder builder() {
             return new JoinNodeTestPlanBuilder();
         }
 
         /**
-         *
+         * @return Callback.
          */
         public Runnable checkCacheOnlySystem() {
             return onAllNode(new IgniteInClosure<IgniteEx>() {
@@ -686,15 +766,11 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @return Callback.
          */
         public Runnable checkCacheEmpty() {
             return onAllNode(new IgniteInClosure<IgniteEx>() {
                 @Override public void apply(IgniteEx ig) {
-                    Map<String, DynamicCacheDescriptor> desc = cacheDescriptors(ig);
-
-                    Assert.assertTrue(desc.isEmpty());
-
                     Assert.assertNull(ig.context().cache().cache(cache1));
                     Assert.assertNull(ig.context().cache().cache(cache2));
 
@@ -706,7 +782,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
         }
 
         /**
-         *
+         * @return Callback.
          */
         public Runnable checkCacheNotEmpty() {
             return onAllNode(new IgniteInClosure<IgniteEx>() {
@@ -735,6 +811,7 @@ public abstract class AbstractNodeJoinTemplate extends GridCommonAbstractTest {
 
         /**
          * @param cls Closure.
+         * @return Callback.
          */
         private Runnable onAllNode(final IgniteInClosure<IgniteEx> cls) {
             return new Runnable() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java
index a06e0ce..4e575cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateAbstractTest.java
@@ -47,13 +47,10 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     private static final String clientSuffix = "-client";
 
     /** Primary ip finder. */
-    protected final TcpDiscoveryIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true);
+    private final TcpDiscoveryIpFinder primaryIpFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** Back up ip finder. */
-    protected final TcpDiscoveryIpFinder backUpIpFinder = new TcpDiscoveryVmIpFinder(true);
-
-    /** Consistent id count. */
-    private int consistentIdCnt;
+    private final TcpDiscoveryIpFinder backUpIpFinder = new TcpDiscoveryVmIpFinder(true);
 
     /** Nodes. */
     protected Map<String, Ignite> nodes = new ConcurrentHashMap<>();
@@ -91,28 +88,28 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     }
 
     /**
-     *
+     * @return Number of server nodes in primary cluster.
      */
     protected int primaryNodes() {
         return 3;
     }
 
     /**
-     *
+     * @return Number of client nodes in primary cluster.
      */
     protected int primaryClientNodes() {
         return 3;
     }
 
     /**
-     *
+     * @return Number of server nodes in backup cluster.
      */
     protected int backUpNodes() {
         return 3;
     }
 
     /**
-     *
+     * @return Number of client nodes in backup cluster.
      */
     protected int backUpClientNodes() {
         return 3;
@@ -120,6 +117,7 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param idx idx.
+     * @return Primary cluster node.
      */
     protected Ignite primary(int idx) {
         return nodes.get("node" + idx + primarySuffix);
@@ -127,29 +125,33 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param idx idx.
+     * @return Primary cluster client node.
      */
-    protected Ignite primaryClient(int idx) {
+    Ignite primaryClient(int idx) {
         return nodes.get("node" + idx + primarySuffix + clientSuffix);
     }
 
     /**
      * @param idx idx.
+     * @return Backup cluster node.
      */
-    protected Ignite backUp(int idx) {
+    Ignite backUp(int idx) {
         return nodes.get("node" + idx + backUpSuffix);
     }
 
     /**
      * @param idx idx.
+     * @return Backup cluster client node.
      */
-    protected Ignite backUpClient(int idx) {
+    Ignite backUpClient(int idx) {
         return nodes.get("node" + idx + backUpSuffix + clientSuffix);
     }
 
     /**
      * @param cnt Count.
+     * @throws Exception If failed.
      */
-    protected void startPrimaryNodes(int cnt) throws Exception {
+    private void startPrimaryNodes(int cnt) throws Exception {
         for (int i = 0; i < cnt; i++)
             startPrimary(i);
 
@@ -159,8 +161,9 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param idx Index.
+     * @throws Exception If failed.
      */
-    protected void startPrimary(int idx) throws Exception {
+    private void startPrimary(int idx) throws Exception {
         String node = "node" + idx;
 
         String name = node + primarySuffix;
@@ -176,22 +179,26 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param cnt Count.
+     * @throws Exception If failed.
      */
-    protected void startBackUpNodes(int cnt) throws Exception {
+    private void startBackUpNodes(int cnt) throws Exception {
         for (int i = 0; i < cnt; i++)
             startBackUp(i);
     }
 
     /**
      * @param idx Index.
+     * @throws Exception If failed.
      */
-    protected void startBackUp(int idx) throws Exception {
+    void startBackUp(int idx) throws Exception {
         String node = "node" + idx;
 
         String name = node + backUpSuffix;
 
         IgniteConfiguration cfg = getConfiguration(name);
+
         cfg.setConsistentId(node);
+
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(backUpIpFinder);
 
         Ignite ig = startGrid(name, cfg);
@@ -201,16 +208,19 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param cnt Count.
+     * @throws Exception If failed.
      */
-    protected void startPrimaryClientNodes(int cnt) throws Exception {
+    void startPrimaryClientNodes(int cnt) throws Exception {
         for (int i = 0; i < cnt; i++) {
             String node = "node" + i;
 
             String name = node + primarySuffix + clientSuffix;
 
             IgniteConfiguration cfg = getConfiguration(name);
+
             cfg.setConsistentId(node);
             cfg.setClientMode(true);
+
             ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(primaryIpFinder);
 
             Ignite ig = startGrid(name, cfg);
@@ -221,8 +231,9 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
 
     /**
      * @param cnt Count.
+     * @throws Exception If failed.
      */
-    protected void startBackUpClientNodes(int cnt) throws Exception {
+    private void startBackUpClientNodes(int cnt) throws Exception {
         for (int i = 0; i < cnt; i++) {
             String node = "node" + i;
 
@@ -241,9 +252,9 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     }
 
     /**
-     *
+     * @return All nodes from backup cluster.
      */
-    protected Iterable<Ignite> allBackUpNodes() {
+    Iterable<Ignite> allBackUpNodes() {
         List<Ignite> r = new ArrayList<>();
 
         for (String name : this.nodes.keySet())
@@ -254,11 +265,10 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     }
 
     /**
-     *
+     * @param includeClient If {@code true} then allow to return client.
+     * @return Random node from backup topology.
      */
-    protected Ignite randomBackUp(boolean includeClient) {
-        int nodes = 0;
-
+    Ignite randomBackUp(boolean includeClient) {
         List<Ignite> igs = new ArrayList<>();
 
         for (String name : this.nodes.keySet())
@@ -281,7 +291,7 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     /**
      * @param i Idx.
      */
-    protected void stopPrimary(int i) {
+    void stopPrimary(int i) {
         String name = "node" + i + primarySuffix;
 
         nodes.get(name).close();
@@ -292,7 +302,7 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     /**
      *
      */
-    protected void stopAllPrimary() {
+    void stopAllPrimary() {
         stopAll(primarySuffix);
     }
 
@@ -357,10 +367,9 @@ public abstract class IgniteChangeGlobalStateAbstractTest extends GridCommonAbst
     }
 
     /**
-     *
+     * @return Test class name.
      */
     protected String testName() {
         return getClass().getSimpleName();
     }
-
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java
index 2f2385d..938b3c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateCacheTest.java
@@ -62,7 +62,7 @@ public class IgniteChangeGlobalStateCacheTest extends IgniteChangeGlobalStateAbs
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testMoreKeyValueAfterActivate() throws Exception {
         String cacheName = "my-cache";

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java
index d3e8298..16be316 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStreamerTest.java
@@ -36,10 +36,9 @@ public class IgniteChangeGlobalStateDataStreamerTest extends IgniteChangeGlobalS
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
-    public void testDeActivateAndActivateDataStreamer() throws InterruptedException {
-
+    public void testDeActivateAndActivateDataStreamer() throws Exception {
         Ignite ig1 = primary(0);
         Ignite ig2 = primary(1);
         Ignite ig3 = primary(2);

http://git-wip-us.apache.org/repos/asf/ignite/blob/1337901f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java
index 9013226..8902a36 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/standbycluster/IgniteChangeGlobalStateDataStructureTest.java
@@ -35,9 +35,9 @@ import static org.apache.ignite.testframework.GridTestUtils.runAsync;
  */
 public class IgniteChangeGlobalStateDataStructureTest extends IgniteChangeGlobalStateAbstractTest {
     /**
-     *
+     * @throws Exception If failed.
      */
-    public void testDeActivateAndActivateAtomicLong() throws Exception{
+    public void testDeActivateAndActivateAtomicLong() throws Exception {
         String lName = "myLong";
 
         Ignite ig1 = primary(0);
@@ -106,7 +106,7 @@ public class IgniteChangeGlobalStateDataStructureTest extends IgniteChangeGlobal
     }
 
     /**
-     *
+     * @throws Exception If failed.
      */
     public void testDeActivateAndActivateCountDownLatch() throws Exception {
         final AtomicInteger cnt = new AtomicInteger();