IGNITE-12618 Affinity cache for version of last server event can be wiped from history - Fixes #7359.

Signed-off-by: Ivan Rakov <irakov@apache.org>
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 9297ae8..52a9342 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -44,6 +44,7 @@
 import org.apache.ignite.internal.cluster.NodeOrderComparator;
 import org.apache.ignite.internal.managers.discovery.DiscoCache;
 import org.apache.ignite.internal.processors.cache.ExchangeDiscoveryEvents;
+import org.apache.ignite.internal.processors.cache.GridCachePartitionExchangeManager;
 import org.apache.ignite.internal.processors.cluster.BaselineTopology;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.F;
@@ -65,11 +66,27 @@
  * Affinity cached function.
  */
 public class GridAffinityAssignmentCache {
-    /** Cleanup history size. */
-    private final int MAX_HIST_SIZE = getInteger(IGNITE_AFFINITY_HISTORY_SIZE, 50);
+    /**
+     * Affinity cache will shrink when total number of non-shallow (see {@link HistoryAffinityAssignmentImpl})
+     * historical instances will be greater than value of this constant.
+     */
+    private final int MAX_NON_SHALLOW_HIST_SIZE = getInteger(IGNITE_AFFINITY_HISTORY_SIZE, 25);
 
-    /** Cleanup history links size (calculated by both real entries and shallow copies). */
-    private final int MAX_HIST_LINKS_SIZE = MAX_HIST_SIZE * 10;
+    /**
+     * Affinity cache will also shrink when total number of both shallow ({@link HistoryAffinityAssignmentShallowCopy})
+     * and non-shallow (see {@link HistoryAffinityAssignmentImpl}) historical instances will be greater than
+     * value of this constant.
+     */
+    private final int MAX_TOTAL_HIST_SIZE = MAX_NON_SHALLOW_HIST_SIZE * 10;
+
+    /**
+     * Independent of {@link #MAX_NON_SHALLOW_HIST_SIZE} and {@link #MAX_TOTAL_HIST_SIZE}, affinity cache will always
+     * keep this number of non-shallow (see {@link HistoryAffinityAssignmentImpl}) instances.
+     * We need at least one real instance, otherwise we won't be able to get affinity cache for
+     * {@link GridCachePartitionExchangeManager#lastAffinityChangedTopologyVersion} in case cluster has experienced
+     * too many client joins / client leaves / local cache starts.
+     */
+    private final int MIN_NON_SHALLOW_HIST_SIZE = 2;
 
     /** Partition distribution. */
     private final float partDistribution = getFloat(IGNITE_PART_DISTRIBUTION_WARN_THRESHOLD, 50f);
@@ -122,8 +139,8 @@
     /** Node stop flag. */
     private volatile IgniteCheckedException stopErr;
 
-    /** Full history size. */
-    private final AtomicInteger fullHistSize = new AtomicInteger();
+    /** Number of non-shallow (see {@link HistoryAffinityAssignmentImpl}) affinity cache instances. */
+    private final AtomicInteger nonShallowHistSize = new AtomicInteger();
 
     /** */
     private final Object similarAffKey;
@@ -278,7 +295,7 @@
 
         affCache.clear();
 
-        fullHistSize.set(0);
+        nonShallowHistSize.set(0);
 
         head.set(new GridAffinityAssignmentV2(AffinityTopologyVersion.NONE));
 
@@ -780,8 +797,8 @@
                 cache = e.getValue();
 
             if (cache == null) {
-                throw new IllegalStateException("Getting affinity for too old topology version that is already " +
-                    "out of history [locNode=" + ctx.discovery().localNode() +
+                throw new IllegalStateException("Getting affinity for topology version earlier than affinity is " +
+                    "calculated [locNode=" + ctx.discovery().localNode() +
                     ", grp=" + cacheOrGrpName +
                     ", topVer=" + topVer +
                     ", lastAffChangeTopVer=" + lastAffChangeTopVer +
@@ -791,8 +808,8 @@
             }
 
             if (cache.topologyVersion().compareTo(topVer) > 0) {
-                throw new IllegalStateException("Getting affinity for topology version earlier than affinity is " +
-                    "calculated [locNode=" + ctx.discovery().localNode() +
+                throw new IllegalStateException("Getting affinity for too old topology version that is already " +
+                    "out of history [locNode=" + ctx.discovery().localNode() +
                     ", grp=" + cacheOrGrpName +
                     ", topVer=" + topVer +
                     ", lastAffChangeTopVer=" + lastAffChangeTopVer +
@@ -910,59 +927,74 @@
             cleanupNeeded = true;
 
             if (added.requiresHistoryCleanup())
-                fullHistSize.incrementAndGet();
+                nonShallowHistSize.incrementAndGet();
         }
         else {
             if (replaced.requiresHistoryCleanup() != added.requiresHistoryCleanup()) {
                 if (added.requiresHistoryCleanup()) {
                     cleanupNeeded = true;
 
-                    fullHistSize.incrementAndGet();
+                    nonShallowHistSize.incrementAndGet();
                 }
                 else
-                    fullHistSize.decrementAndGet();
+                    nonShallowHistSize.decrementAndGet();
             }
         }
 
         if (!cleanupNeeded)
             return;
 
-        int fullSize = fullHistSize.get();
+        int nonShallowSize = nonShallowHistSize.get();
 
-        int linksSize = affCache.size();
+        int totalSize = affCache.size();
 
-        int fullRmvCnt = fullSize > MAX_HIST_SIZE ? (MAX_HIST_SIZE / 2) : 0;
+        if (shouldContinueCleanup(nonShallowSize, totalSize)) {
+            int initNonShallowSize = nonShallowSize;
 
-        int linksRmvCnt = linksSize > MAX_HIST_LINKS_SIZE ? (MAX_HIST_LINKS_SIZE / 2) : 0;
-
-        if (fullRmvCnt > 0 || linksRmvCnt > 0) {
             Iterator<HistoryAffinityAssignment> it = affCache.values().iterator();
 
-            AffinityTopologyVersion topVerRmv = null;
-
-            while (it.hasNext() && (fullRmvCnt > 0 || linksRmvCnt > 0)) {
+            while (it.hasNext()) {
                 HistoryAffinityAssignment aff0 = it.next();
 
-                if (aff0.requiresHistoryCleanup()) { // Don't decrement counter in case of fullHistoryCleanupRequired copy remove.
-                    fullRmvCnt--;
+                if (aff0.requiresHistoryCleanup()) {
+                    // We can stop cleanup only on non-shallow item.
+                    // Keeping part of shallow items chain if corresponding real item is missing makes no sense.
+                    if (!shouldContinueCleanup(nonShallowSize, totalSize)) {
+                        nonShallowHistSize.getAndAdd(nonShallowSize - initNonShallowSize);
 
-                    fullHistSize.decrementAndGet();
+                        // GridAffinityProcessor#affMap has the same size and instance set as #affCache.
+                        ctx.affinity().removeCachedAffinity(aff0.topologyVersion());
+
+                        return;
+                    }
+
+                    nonShallowSize--;
                 }
 
-                linksRmvCnt--;
+                totalSize--;
 
                 it.remove();
-
-                topVerRmv = aff0.topologyVersion();
             }
 
-            topVerRmv = it.hasNext() ? it.next().topologyVersion() : topVerRmv;
-
-            ctx.affinity().removeCachedAffinity(topVerRmv);
+            assert false : "All elements have been removed from affinity cache during cleanup";
         }
     }
 
     /**
+     * Checks whether affinity cache size conditions are still unsatisfied.
+     *
+     * @param nonShallowSize Non shallow size.
+     * @param totalSize Total size.
+     * @return <code>true</code> if affinity cache cleanup is not finished yet.
+     */
+    private boolean shouldContinueCleanup(int nonShallowSize, int totalSize) {
+        if (nonShallowSize <= MIN_NON_SHALLOW_HIST_SIZE)
+            return false;
+
+        return nonShallowSize > MAX_NON_SHALLOW_HIST_SIZE || totalSize > MAX_TOTAL_HIST_SIZE;
+    }
+
+    /**
      * @return All initialized versions.
      */
     public Collection<AffinityTopologyVersion> cachedVersions() {
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
index cdee5eb..44de7d3 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java
@@ -102,8 +102,8 @@
             topVer(2, 1), // FullHistSize = 3.
             topVer(3, 0), // FullHistSize = 4.
             topVer(3, 1), // FullHistSize = 5.
-            topVer(4, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4.
-            topVer(4, 1)), // FullHistSize = 5.
+            topVer(4, 0), // FullHistSize = 6 - 1 = 5.
+            topVer(4, 1)), // FullHistSize = 6 - 1 = 5.
             5);
 
         startClientGrid(4);
@@ -114,10 +114,10 @@
             topVer(2, 1), // FullHistSize = 3.
             topVer(3, 0), // FullHistSize = 4.
             topVer(3, 1), // FullHistSize = 5.
-            topVer(4, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4.
-            topVer(4, 1), // FullHistSize = 5.
-            topVer(5, 0), // Client event ->FullHistSize = 5.
-            topVer(6, 0)), // Client event ->FullHistSize = 5.
+            topVer(4, 0), // FullHistSize = 6 - 1 = 5.
+            topVer(4, 1), // FullHistSize = 6 - 1 = 5.
+            topVer(5, 0), // Client event -> FullHistSize = 5.
+            topVer(6, 0)), // Client event -> FullHistSize = 5.
             5);
 
         startClientGrid(4);
@@ -126,14 +126,14 @@
 
         checkHistory(ignite, F.asList(
             topVer(2, 1), // FullHistSize = 3.
-            topVer(3, 0), // FullHistSize =4.
+            topVer(3, 0), // FullHistSize = 4.
             topVer(3, 1), // FullHistSize = 5.
-            topVer(4, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4.
-            topVer(4, 1), // FullHistSize = 5.
+            topVer(4, 0), // FullHistSize = 6 - 1 = 5.
+            topVer(4, 1), // FullHistSize = 6 - 1 = 5.
             topVer(5, 0), // Client event -> FullHistSize = 5.
             topVer(6, 0), // Client event -> FullHistSize = 5.
             topVer(7, 0), // Client event -> FullHistSize = 5.
-            topVer(8, 0)), // Client event ->FullHistSize = 5.
+            topVer(8, 0)), // Client event -> FullHistSize = 5.
             5);
 
         startClientGrid(4);
@@ -143,31 +143,31 @@
         checkHistory(ignite, F.asList(
             topVer(2, 1), // FullHistSize = 3.
             topVer(3, 0), // FullHistSize = 4.
-            topVer(3, 1), // FullHistSize =5.
-            topVer(4, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4.
-            topVer(4, 1), // FullHistSize = 5.
+            topVer(3, 1), // FullHistSize = 5.
+            topVer(4, 0), // FullHistSize = 6 - 1 = 5.
+            topVer(4, 1), // FullHistSize = 6 - 1 = 5.
             topVer(5, 0), // Client event -> FullHistSize = 5.
             topVer(6, 0), // Client event -> FullHistSize = 5.
             topVer(7, 0), // Client event -> FullHistSize = 5.
             topVer(8, 0), // Client event -> FullHistSize = 5.
             topVer(9, 0), // Client event -> FullHistSize = 5.
-            topVer(10, 0)), // Client event ->FullHistSize = 5.
+            topVer(10, 0)), // Client event -> FullHistSize = 5.
             5);
 
         startGrid(4);
 
         checkHistory(ignite, F.asList(
             topVer(3, 1), // FullHistSize = 5.
-            topVer(4, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4.
-            topVer(4, 1), // FullHistSize = 5.
+            topVer(4, 0), // FullHistSize = 6 - 1 = 5.
+            topVer(4, 1), // FullHistSize = 6 - 1 = 5.
             topVer(5, 0), // Client event -> FullHistSize = 5.
             topVer(6, 0), // Client event -> FullHistSize = 5.
             topVer(7, 0), // Client event -> FullHistSize = 5.
             topVer(8, 0), // Client event -> FullHistSize = 5.
             topVer(9, 0), // Client event -> FullHistSize = 5.
             topVer(10, 0), // Client event -> FullHistSize = 5.
-            topVer(11, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4.
-            topVer(11, 1)), // FullHistSize = 5.
+            topVer(11, 0), // FullHistSize = 6 - 1 = 5.
+            topVer(11, 1)), // FullHistSize = 6 - 1 = 5.
             5);
 
         stopGrid(4);
@@ -175,11 +175,12 @@
         startGrid(4);
 
         checkHistory(ignite, F.asList(
+            topVer(11, 0), // FullHistSize = 5.
             topVer(11, 1), // FullHistSize = 5.
-            topVer(12, 0), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4.
+            topVer(12, 0), // FullHistSize = 6 - 1 = 5.
             topVer(13, 0), // FullHistSize = 5.
-            topVer(13, 1)), // FullHistSize = (6 - IGNITE_AFFINITY_HISTORY_SIZE(5)/2) = 4.
-            4);
+            topVer(13, 1)), // FullHistSize = 6 - 1 = 5.
+            5);
     }
 
     /**
@@ -198,7 +199,7 @@
         for (GridCacheContext cctx : proc.context().cacheContexts()) {
             GridAffinityAssignmentCache aff = GridTestUtils.getFieldValue(cctx.affinity(), "aff");
 
-            AtomicInteger fullHistSize = GridTestUtils.getFieldValue(aff, "fullHistSize");
+            AtomicInteger fullHistSize = GridTestUtils.getFieldValue(aff, "nonShallowHistSize");
 
             assertEquals(expSize, fullHistSize.get());
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorMemoryLeakTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorMemoryLeakTest.java
deleted file mode 100644
index 1bb309f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorMemoryLeakTest.java
+++ /dev/null
@@ -1,194 +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.affinity;
-
-import java.util.concurrent.ConcurrentSkipListMap;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCache;
-import org.apache.ignite.IgniteDataStreamer;
-import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.cache.CacheMode;
-import org.apache.ignite.configuration.CacheConfiguration;
-import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.internal.IgniteKernal;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.testframework.GridTestUtils;
-import org.apache.ignite.testframework.junits.WithSystemProperty;
-import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
-import org.apache.ignite.testframework.junits.common.GridCommonTest;
-import org.junit.Test;
-
-/**
- * Tests for {@link GridAffinityProcessor}.
- */
-@GridCommonTest(group = "Affinity Processor")
-public class GridAffinityProcessorMemoryLeakTest extends GridCommonAbstractTest {
-    /** Cache name. */
-    private static final String CACHE_NAME = "cache";
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
-
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
-
-        CacheConfiguration cacheCfg = defaultCacheConfiguration();
-
-        cacheCfg.setName(CACHE_NAME);
-
-        cacheCfg.setStoreKeepBinary(true);
-
-        cacheCfg.setCacheMode(CacheMode.LOCAL);
-
-        cfg.setCacheConfiguration(cacheCfg);
-
-        return cfg;
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void afterTestsStopped() throws Exception {
-        stopAllGrids();
-    }
-
-    /**
-     * Test affinity functions caching and clean up.
-     *
-     * @throws Exception In case of any exception.
-     */
-    @Test
-    @WithSystemProperty(key = IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, value = "10")
-    public void testAffinityProcessor() throws Exception {
-        int maxHistSize = 10;
-
-        Ignite ignite = startGrid(0);
-
-        IgniteKernal grid = (IgniteKernal)grid(0);
-
-        IgniteCache<String, String> cache;
-
-        IgniteCache<String, String> globalCache = getOrCreateGlobalCache(ignite);
-
-        IgniteDataStreamer<String, String> globalStreamer;
-
-        int cnt = maxHistSize * 30;
-
-        int expLimit = cnt / 2;
-
-        int size;
-
-        do {
-            try {
-                cache = createLocalCache(ignite, cnt);
-
-                cache.put("Key" + cnt, "Value" + cnt);
-
-                cache.destroy();
-
-                globalStreamer = createGlobalStreamer(ignite, globalCache);
-
-                globalStreamer.addData("GlobalKey" + cnt, "GlobalValue" + cnt);
-
-                globalStreamer.flush();
-
-                globalStreamer.close();
-
-                size = ((ConcurrentSkipListMap)GridTestUtils.getFieldValue(grid.context().affinity(), "affMap")).size();
-
-                assertTrue("Cache has size that bigger then expected [size=" + size +
-                    ", expLimit=" + expLimit + "]", size < expLimit);
-            }
-            catch (Exception e) {
-                fail("Error was handled [" + e.getMessage() + "]");
-            }
-        }
-        while (cnt-- > 0);
-    }
-
-    /**
-     * Creates global cache.
-     *
-     * @param ignite instance of {@code Ignite}.
-     * @param id unique id for local cache.
-     * @return local cache instance.
-     */
-    private static IgniteCache<String, String> createLocalCache(Ignite ignite, long id) {
-        final String cacheName = "localCache" + id;
-
-        final CacheConfiguration<String, String> cCfg = new CacheConfiguration<>();
-
-        cCfg.setName(cacheName);
-
-        cCfg.setCacheMode(CacheMode.LOCAL);
-
-        cCfg.setGroupName("some group");
-
-        ignite.destroyCache(cacheName); // Local cache is not really local - reference can be kept by other nodes if restart during the load happens.
-
-        return ignite.createCache(cCfg).withKeepBinary();
-    }
-
-    /**
-     * Gets or creates global cache.
-     *
-     * @param ignite instance of {@code Ignite}.
-     * @return global cache instance.
-     */
-    private static IgniteCache<String, String> getOrCreateGlobalCache(Ignite ignite) {
-        final String cacheName = "GlobalCache";
-
-        final CacheConfiguration<String, String> cCfg = new CacheConfiguration<>();
-
-        cCfg.setName(cacheName);
-
-        cCfg.setStoreKeepBinary(true);
-
-        cCfg.setCacheMode(CacheMode.PARTITIONED);
-
-        cCfg.setOnheapCacheEnabled(false);
-
-        cCfg.setCopyOnRead(false);
-
-        cCfg.setBackups(0);
-
-        cCfg.setWriteBehindEnabled(false);
-
-        cCfg.setReadThrough(false);
-
-        return ignite.getOrCreateCache(cCfg).withKeepBinary();
-    }
-
-    /**
-     * Creates streamer for global cache.
-     *
-     * @param ignite instance of {@code Ignite}.
-     * @param cache instance of global cache.
-     * @return instance of {@code IgniteDataStreamer}.
-     */
-    private static IgniteDataStreamer<String, String> createGlobalStreamer(Ignite ignite,
-        IgniteCache<String, String> cache) {
-        IgniteDataStreamer<String, String> streamer = ignite.dataStreamer(cache.getName());
-
-        streamer.allowOverwrite(true);
-
-        streamer.skipStore(true);
-
-        streamer.keepBinary(false);
-
-        return streamer;
-    }
-}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNoAffinityExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNoAffinityExchangeTest.java
index bd3fead..879e02b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNoAffinityExchangeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheNoAffinityExchangeTest.java
@@ -310,13 +310,33 @@
     }
 
     /**
-     * Tests that multiple client events won't fail transactions due to affinity assignment history expiration.
+     * Checks case when number of client events is greater than affinity history size.
      *
      * @throws Exception If failed.
      */
     @Test
     @WithSystemProperty(key = IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, value = "10")
     public void testMulipleClientLeaveJoin() throws Exception {
+        doTestMulipleClientLeaveJoin();
+    }
+
+    /**
+     * Checks case when number of client events is so big that history consists only from client event versions.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    @WithSystemProperty(key = IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, value = "2")
+    public void testMulipleClientLeaveJoinLinksLimitOverflow() throws Exception {
+        doTestMulipleClientLeaveJoin();
+    }
+
+    /**
+     * Tests that multiple client events won't fail transactions due to affinity assignment history expiration.
+     *
+     * @throws Exception If failed.
+     */
+    public void doTestMulipleClientLeaveJoin() throws Exception {
         Ignite ig = startGrids(2);
 
         ig.cluster().active(true);
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 6d3a685..5dbdea7 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -19,8 +19,6 @@
 
 import org.apache.ignite.ClassPathContentLoggingTest;
 import org.apache.ignite.GridSuppressedExceptionSelfTest;
-import org.apache.ignite.internal.processors.cache.distributed.dht.topology.EvictPartitionInLogTest;
-import org.apache.ignite.internal.processors.failure.FailureProcessorLoggingTest;
 import org.apache.ignite.failure.FailureHandlerTriggeredTest;
 import org.apache.ignite.failure.OomFailureHandlerTest;
 import org.apache.ignite.failure.StopNodeFailureHandlerTest;
@@ -56,7 +54,6 @@
 import org.apache.ignite.internal.processors.DeadLockOnNodeLeftExchangeTest;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentV2Test;
 import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentV2TestNoOptimizations;
-import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorMemoryLeakTest;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessorRendezvousSelfTest;
 import org.apache.ignite.internal.processors.affinity.GridHistoryAffinityAssignmentTest;
 import org.apache.ignite.internal.processors.affinity.GridHistoryAffinityAssignmentTestNoOptimization;
@@ -72,6 +69,7 @@
 import org.apache.ignite.internal.processors.cache.SetTxTimeoutOnPartitionMapExchangeTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteRejectConnectOnNodeStopTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.DropCacheContextDuringEvictionTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.topology.EvictPartitionInLogTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionsEvictionTaskFailureHandlerTest;
 import org.apache.ignite.internal.processors.cache.persistence.pagemem.PagePoolTest;
 import org.apache.ignite.internal.processors.cache.query.continuous.DiscoveryDataDeserializationFailureHanderTest;
@@ -91,6 +89,7 @@
 import org.apache.ignite.internal.processors.database.DataRegionMetricsSelfTest;
 import org.apache.ignite.internal.processors.database.IndexStorageSelfTest;
 import org.apache.ignite.internal.processors.database.SwapPathConstructionSelfTest;
+import org.apache.ignite.internal.processors.failure.FailureProcessorLoggingTest;
 import org.apache.ignite.internal.processors.failure.FailureProcessorThreadDumpThrottlingTest;
 import org.apache.ignite.internal.processors.metastorage.DistributedMetaStorageTest;
 import org.apache.ignite.internal.processors.metastorage.persistence.DistributedMetaStorageHistoryCacheTest;
@@ -99,9 +98,9 @@
 import org.apache.ignite.internal.processors.odbc.OdbcConfigurationValidationSelfTest;
 import org.apache.ignite.internal.processors.odbc.OdbcEscapeSequenceSelfTest;
 import org.apache.ignite.internal.product.GridProductVersionSelfTest;
+import org.apache.ignite.internal.util.GridCleanerTest;
 import org.apache.ignite.internal.util.collection.BitSetIntSetTest;
 import org.apache.ignite.internal.util.collection.ImmutableIntSetTest;
-import org.apache.ignite.internal.util.GridCleanerTest;
 import org.apache.ignite.internal.util.collection.IntHashMapTest;
 import org.apache.ignite.internal.util.collection.IntRWHashMapTest;
 import org.apache.ignite.internal.util.nio.IgniteExceptionInNioWorkerSelfTest;
@@ -160,7 +159,6 @@
     GridHistoryAffinityAssignmentTest.class,
     GridHistoryAffinityAssignmentTestNoOptimization.class,
     GridAffinityProcessorRendezvousSelfTest.class,
-    GridAffinityProcessorMemoryLeakTest.class,
     GridClosureProcessorSelfTest.class,
     GridClosureProcessorRemoteTest.class,
     GridClosureSerializationTest.class,