You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ir...@apache.org on 2020/01/22 09:24:46 UTC

[ignite] branch master updated: IGNITE-12497 PartitionsEvictManager should log all partitions which will be evicted - Fixes #7202.

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

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


The following commit(s) were added to refs/heads/master by this push:
     new f7c954c  IGNITE-12497 PartitionsEvictManager should log all partitions which will be evicted - Fixes #7202.
f7c954c is described below

commit f7c954c2739351dd31a96721545237068eb9e696
Author: Kirill Tkalenko <tk...@yandex.ru>
AuthorDate: Wed Jan 22 12:24:25 2020 +0300

    IGNITE-12497 PartitionsEvictManager should log all partitions which will be evicted - Fixes #7202.
    
    Signed-off-by: Ivan Rakov <ir...@apache.org>
---
 .../dht/topology/GridDhtLocalPartition.java        |   6 +-
 .../dht/topology/PartitionsEvictManager.java       | 125 ++++++++--
 .../dht/topology/EvictPartitionInLogTest.java      | 274 +++++++++++++++++++++
 .../testframework/junits/GridAbstractTest.java     |  40 ++-
 .../ignite/testsuites/IgniteBasicTestSuite.java    |   2 +
 5 files changed, 429 insertions(+), 18 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
index f6eb74a..ff0b0f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java
@@ -80,6 +80,8 @@ import static org.apache.ignite.internal.processors.cache.distributed.dht.topolo
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.RENTING;
+import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionsEvictManager.EvictReason.CLEARING;
+import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.PartitionsEvictManager.EvictReason.EVICTION;
 import static org.apache.ignite.internal.processors.cache.persistence.CheckpointState.FINISHED;
 
 /**
@@ -712,7 +714,7 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements
                             return;
                         }
 
-                        ctx.evict().evictPartitionAsync(grp, GridDhtLocalPartition.this);
+                        ctx.evict().evictPartitionAsync(grp, this, CLEARING);
                     }
                 });
 
@@ -739,7 +741,7 @@ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements
             }
         }
 
-        ctx.evict().evictPartitionAsync(grp, this);
+        ctx.evict().evictPartitionAsync(grp, this, EVICTION);
     }
 
     /**
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionsEvictManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionsEvictManager.java
index 3c4b1e9..444b630 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionsEvictManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/PartitionsEvictManager.java
@@ -17,12 +17,17 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht.topology;
 
+import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Comparator;
+import java.util.EnumMap;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Map;
+import java.util.Map.Entry;
 import java.util.Queue;
 import java.util.Set;
+import java.util.StringJoiner;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.PriorityBlockingQueue;
@@ -35,8 +40,10 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedManagerAdapter;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.typedef.internal.LT;
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 
+import static java.util.Objects.nonNull;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_EVICTION_PERMITS;
 import static org.apache.ignite.IgniteSystemProperties.getInteger;
 import static org.apache.ignite.IgniteSystemProperties.getLong;
@@ -57,7 +64,8 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
     private static final byte EVICT_POOL_PLC = GridIoPolicy.SYSTEM_POOL;
 
     /** Eviction progress frequency in ms. */
-    private final long evictionProgressFreqMs = getLong(SHOW_EVICTION_PROGRESS_FREQ, DEFAULT_SHOW_EVICTION_PROGRESS_FREQ_MS);
+    private final long evictionProgressFreqMs =
+        getLong(SHOW_EVICTION_PROGRESS_FREQ, DEFAULT_SHOW_EVICTION_PROGRESS_FREQ_MS);
 
     /** */
     private final int confPermits = getInteger(IGNITE_EVICTION_PERMITS, -1);
@@ -68,6 +76,11 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
     /** */
     private final Map<Integer, GroupEvictionContext> evictionGroupsMap = new ConcurrentHashMap<>();
 
+    /**
+     * Evicted partitions for printing to log. It works under {@link #mux}.
+     */
+    private final Map<Integer, Map<Integer, EvictReason>> logEvictPartByGrps = new HashMap<>();
+
     /** Flag indicates that eviction process has stopped. */
     private volatile boolean stop;
 
@@ -80,9 +93,10 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
     /** How many eviction task may execute concurrent. */
     private volatile int permits;
 
-    /** Bucket queue for load balance partitions to the threads via count of partition size.
-     *  Is not thread-safe.
-     *  All method should be called under mux synchronization.
+    /**
+     * Bucket queue for load balance partitions to the threads via count of
+     * partition size. Is not thread-safe. All method should be called under
+     * mux synchronization.
      */
     volatile BucketQueue evictionQueue;
 
@@ -107,14 +121,26 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
-     * Adds partition to eviction queue and starts eviction process if permit available.
+     * Adds partition to eviction queue and starts eviction process if permit
+     * available.
      *
      * @param grp Group context.
      * @param part Partition to evict.
+     * @param reason Reason for eviction.
      */
-    public void evictPartitionAsync(CacheGroupContext grp, GridDhtLocalPartition part) {
+    public void evictPartitionAsync(
+        CacheGroupContext grp,
+        GridDhtLocalPartition part,
+        EvictReason reason
+    ) {
+        assert nonNull(grp);
+        assert nonNull(part);
+        assert nonNull(reason);
+
+        int grpId = grp.groupId();
+
         GroupEvictionContext grpEvictionCtx = evictionGroupsMap.computeIfAbsent(
-            grp.groupId(), (k) -> new GroupEvictionContext(grp));
+            grpId, (k) -> new GroupEvictionContext(grp));
 
         // Check node stop.
         if (grpEvictionCtx.shouldStop())
@@ -123,10 +149,14 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
         int bucket;
 
         synchronized (mux) {
-            if (!grpEvictionCtx.partIds.add(part.id()))
+            int partId = part.id();
+
+            if (!grpEvictionCtx.partIds.add(partId))
                 return;
 
-            bucket = evictionQueue.offer(new PartitionEvictionTask(part, grpEvictionCtx));
+            bucket = evictionQueue.offer(new PartitionEvictionTask(part, grpEvictionCtx, reason));
+
+            logEvictPartByGrps.computeIfAbsent(grpId, i -> new HashMap<>()).put(partId, reason);
         }
 
         grpEvictionCtx.totalTasks.incrementAndGet();
@@ -218,13 +248,31 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
         if (U.millisSinceNanos(lastShowProgressTimeNanos) >= evictionProgressFreqMs) {
             int size = evictionQueue.size() + 1; // Queue size plus current partition.
 
-            if (log.isInfoEnabled())
-                log.info("Eviction in progress [permits=" + permits+
+            if (log.isInfoEnabled()) {
+                log.info("Eviction in progress [permits=" + permits +
                     ", threads=" + threads +
                     ", groups=" + evictionGroupsMap.keySet().size() +
                     ", remainingPartsToEvict=" + size + "]");
 
-            evictionGroupsMap.values().forEach(GroupEvictionContext::showProgress);
+                evictionGroupsMap.values().forEach(GroupEvictionContext::showProgress);
+
+                if (!logEvictPartByGrps.isEmpty()) {
+                    StringJoiner evictPartJoiner = new StringJoiner(", ");
+
+                    logEvictPartByGrps.forEach((grpId, evictParts) -> {
+                        CacheGroupContext grpCtx = cctx.cache().cacheGroup(grpId);
+                        String grpName = (nonNull(grpCtx) ? grpCtx.cacheOrGroupName() : null);
+
+                        String partByReasonStr = partByReasonStr(evictParts);
+
+                        evictPartJoiner.add("[grpId=" + grpId + ", grpName=" + grpName + ", " + partByReasonStr + ']');
+                    });
+
+                    log.info("Partitions have been scheduled for eviction: " + evictPartJoiner);
+
+                    logEvictPartByGrps.clear();
+                }
+            }
 
             lastShowProgressTimeNanos = System.nanoTime();
         }
@@ -266,6 +314,27 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
     }
 
     /**
+     * Creating a group partitions for reasons of eviction as a string.
+     *
+     * @param evictParts Partitions with a reason for eviction.
+     * @return String with group partitions for reasons of eviction.
+     */
+    private String partByReasonStr(Map<Integer, EvictReason> evictParts) {
+        assert nonNull(evictParts);
+
+        Map<EvictReason, Collection<Integer>> partByReason = new EnumMap<>(EvictReason.class);
+
+        for (Entry<Integer, EvictReason> entry : evictParts.entrySet())
+            partByReason.computeIfAbsent(entry.getValue(), b -> new ArrayList<>()).add(entry.getKey());
+
+        StringJoiner joiner = new StringJoiner(", ");
+
+        partByReason.forEach((reason, partIds) -> joiner.add(reason.toString() + '=' + S.compact(partIds)));
+
+        return joiner.toString();
+    }
+
+    /**
      *
      */
     private class GroupEvictionContext implements EvictionContext {
@@ -385,6 +454,9 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
         /** */
         private final long size;
 
+        /** Reason for eviction. */
+        private final EvictReason reason;
+
         /** Eviction context. */
         private final GroupEvictionContext grpEvictionCtx;
 
@@ -394,13 +466,16 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
         /**
          * @param part Partition.
          * @param grpEvictionCtx Eviction context.
+         * @param reason Reason for eviction.
          */
         private PartitionEvictionTask(
             GridDhtLocalPartition part,
-            GroupEvictionContext grpEvictionCtx
+            GroupEvictionContext grpEvictionCtx,
+            EvictReason reason
         ) {
             this.part = part;
             this.grpEvictionCtx = grpEvictionCtx;
+            this.reason = reason;
 
             size = part.fullSize();
         }
@@ -425,7 +500,7 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
 
                 // Re-offer partition if clear was unsuccessful due to partition reservation.
                 if (!success)
-                    evictPartitionAsync(grpEvictionCtx.grp, part);
+                    evictPartitionAsync(grpEvictionCtx.grp, part, reason);
             }
             catch (Throwable ex) {
                 finishFut.onDone(ex);
@@ -565,4 +640,26 @@ public class PartitionsEvictManager extends GridCacheSharedManagerAdapter {
             }
         }
     }
+
+    /**
+     * Reason for eviction of partition.
+     */
+    enum EvictReason {
+        /**
+         * Partition evicted after changing to
+         * {@link GridDhtPartitionState#RENTING RENTING} state.
+         */
+        EVICTION,
+
+        /**
+         * Partition evicted after changing to
+         * {@link GridDhtPartitionState#MOVING MOVING} state.
+         */
+        CLEARING;
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return name().toLowerCase();
+        }
+    }
 }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/EvictPartitionInLogTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/EvictPartitionInLogTest.java
new file mode 100644
index 0000000..15d4eba
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/EvictPartitionInLogTest.java
@@ -0,0 +1,274 @@
+/*
+ * 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.distributed.dht.topology;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.stream.IntStream;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionDemander.RebalanceFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.WithSystemProperty;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static java.lang.Integer.parseInt;
+import static java.util.stream.Collectors.groupingBy;
+import static java.util.stream.Collectors.mapping;
+import static java.util.stream.Collectors.toList;
+import static java.util.stream.Stream.of;
+import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.MOVING;
+import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.RENTING;
+
+/**
+ * Class checks the presence of evicted partitions in log.
+ */
+@WithSystemProperty(key = "SHOW_EVICTION_PROGRESS_FREQ", value = "10")
+public class EvictPartitionInLogTest extends GridCommonAbstractTest {
+    /** Listener log messages. */
+    private static ListeningTestLogger testLog;
+
+    /** Cache names. */
+    private static final String[] DEFAULT_CACHE_NAMES = {DEFAULT_CACHE_NAME + "0", DEFAULT_CACHE_NAME + "1"};
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        clearStaticLog(GridDhtLocalPartition.class);
+
+        testLog = new ListeningTestLogger(false, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        testLog.clearListeners();
+
+        super.afterTest();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setGridLogger(testLog)
+            .setCacheConfiguration(
+                of(DEFAULT_CACHE_NAMES)
+                    .map(cacheName ->
+                        new CacheConfiguration<>(cacheName)
+                            .setGroupName(cacheName)
+                            .setBackups(0)
+                            .setAffinity(new RendezvousAffinityFunction(false, 12))
+                            .setIndexedTypes(Integer.class, Integer.class)
+                    ).toArray(CacheConfiguration[]::new)
+            );
+    }
+
+    /**
+     * Test checks the presence of evicted partitions (RENTING state) in log
+     * without duplicate partitions.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEvictPartByRentingState() throws Exception {
+        IgniteEx node = startGrid(0);
+        awaitPartitionMapExchange();
+
+        Map<Integer, Collection<Integer>> parseParts = new ConcurrentHashMap<>();
+
+        LogListener logLsnr = logListener("eviction", parseParts, DEFAULT_CACHE_NAMES);
+        testLog.registerListener(logLsnr);
+
+        List<GridDhtLocalPartition> parts = of(DEFAULT_CACHE_NAMES)
+            .map(node::cache)
+            .map(GridCommonAbstractTest::internalCache0)
+            .flatMap(internalCache -> internalCache.context().topology().localPartitions().stream())
+            .peek(p -> p.setState(RENTING))
+            .collect(toList());
+
+        parts.subList(0, parts.size() - 1).forEach(GridDhtLocalPartition::clearAsync);
+
+        doSleep(100);
+
+        parts.get(parts.size() - 1).clearAsync();
+
+        check(logLsnr, parts, parseParts);
+    }
+
+    /**
+     * Test checks the presence of evicted partitions (MOVING state) in log
+     * without duplicate partitions.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testEvictPartByMovingState() throws Exception {
+        IgniteEx node = startGrid(0);
+        awaitPartitionMapExchange();
+
+        Map<Integer, Collection<Integer>> parseParts = new ConcurrentHashMap<>();
+
+        LogListener logLsnr = logListener("clearing", parseParts, DEFAULT_CACHE_NAMES);
+        testLog.registerListener(logLsnr);
+
+        List<GridCacheAdapter<Object, Object>> internalCaches = of(DEFAULT_CACHE_NAMES)
+            .map(node::cache)
+            .map(GridCommonAbstractTest::internalCache0)
+            .collect(toList());
+
+        List<RebalanceFuture> rebFuts = internalCaches.stream()
+            .map(internalCache -> (RebalanceFuture)internalCache.context().preloader().rebalanceFuture())
+            .collect(toList());
+
+        rebFuts.forEach(GridFutureAdapter::reset);
+
+        List<GridDhtLocalPartition> parts = of(DEFAULT_CACHE_NAMES)
+            .map(node::cache)
+            .map(GridCommonAbstractTest::internalCache0)
+            .flatMap(internalCache -> internalCache.context().topology().localPartitions().stream())
+            .peek(p -> p.setState(MOVING))
+            .collect(toList());
+
+        parts.subList(0, parts.size() - 1).forEach(GridDhtLocalPartition::clearAsync);
+        rebFuts.forEach(rebFut -> rebFut.onDone(Boolean.TRUE));
+
+        doSleep(100);
+        rebFuts.forEach(GridFutureAdapter::reset);
+
+        parts.get(parts.size() - 1).clearAsync();
+        rebFuts.forEach(rebFut -> rebFut.onDone(Boolean.TRUE));
+
+        check(logLsnr, parts, parseParts);
+    }
+
+    /**
+     * Checking for logs without duplicate partitions.
+     *
+     * @param logLsnr Log listener.
+     * @param parts Partitions.
+     * @param parseParts Parsed partitions from the logs.
+     */
+    private void check(
+        LogListener logLsnr,
+        Collection<GridDhtLocalPartition> parts,
+        Map<Integer, Collection<Integer>> parseParts
+    ) {
+        assertNotNull(logLsnr);
+        assertNotNull(parts);
+        assertNotNull(parseParts);
+
+        assertTrue(logLsnr.check());
+
+        Map<Integer, List<Integer>> partsByGrpId = parts.stream()
+            .collect(groupingBy(p -> p.group().groupId(), mapping(GridDhtLocalPartition::id, toList())));
+
+        partsByGrpId.forEach((grpId, partIds) -> {
+            assertTrue(parseParts.containsKey(grpId));
+
+            List<Integer> parsePartIds = new ArrayList<>(parseParts.get(grpId));
+
+            Collections.sort(parsePartIds);
+            Collections.sort(partIds);
+
+            assertEqualsCollections(partIds, parsePartIds);
+        });
+    }
+
+    /**
+     * Creating a listener for logs with parsing of partitions.
+     *
+     * @param reason Reason to eviction.
+     * @param evictParts To collect parsed partitions.
+     * @param cacheNames Cache names.
+     * @return Log Listener.
+     */
+    private LogListener logListener(
+        String reason,
+        Map<Integer, Collection<Integer>> evictParts,
+        String... cacheNames
+    ) {
+        assertNotNull(reason);
+        assertNotNull(evictParts);
+        assertNotNull(cacheNames);
+
+        List<String> cacheInfos = of(cacheNames)
+            .map(cacheName -> "grpId=" + CU.cacheId(cacheName) + ", grpName=" + cacheName)
+            .collect(toList());
+
+        Pattern extractParts = Pattern.compile(reason + "=\\[([0-9\\-,]*)]]");
+        Pattern extractGrpId = Pattern.compile("grpId=([0-9]*)");
+
+        LogListener.Builder builder = LogListener.matches(logStr -> {
+            if (logStr.contains("Partitions have been scheduled for eviction:")) {
+                Matcher grpIdMatcher = extractGrpId.matcher(logStr);
+                Matcher partsMatcher = extractParts.matcher(logStr);
+
+                //find and parsing grpId and partitions
+                while (grpIdMatcher.find() && partsMatcher.find()) {
+                    evictParts.computeIfAbsent(parseInt(grpIdMatcher.group(1)), i -> new ConcurrentLinkedQueue<>())
+                        .addAll(parseContentCompactStr(partsMatcher.group(1)));
+                }
+
+                return cacheInfos.stream().allMatch(logStr::contains);
+            }
+            else
+                return false;
+        });
+
+        return builder.build();
+    }
+
+    /**
+     * Parse contents of compact string after a {@link S#compact(Collection)}.
+     *
+     * @param str Compact string content.
+     * @return Parsed numbers.
+     */
+    private Collection<Integer> parseContentCompactStr(String str) {
+        assertNotNull(str);
+
+        return of(str.split(","))
+            .map(String::trim)
+            .flatMap(s -> {
+                if (s.contains("-")) {
+                    String[] range = s.split("-");
+
+                    return IntStream.rangeClosed(parseInt(range[0]), parseInt(range[1])).boxed();
+                }
+                else
+                    return of(parseInt(s));
+            }).collect(toList());
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 54e810a..259aa99 100755
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -144,6 +144,7 @@ import org.springframework.beans.BeansException;
 import org.springframework.context.ApplicationContext;
 import org.springframework.context.support.FileSystemXmlApplicationContext;
 
+import static java.util.Collections.newSetFromMap;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ALLOW_ATOMIC_OPS_IN_TX;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_ATOMIC_CACHE_DELETE_HISTORY_SIZE;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CLIENT_CACHE_CHANGE_MESSAGE_TIMEOUT;
@@ -156,6 +157,8 @@ import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL;
 import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL_SNAPSHOT;
 import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
 import static org.apache.ignite.internal.GridKernalState.DISCONNECTED;
+import static org.apache.ignite.testframework.GridTestUtils.getFieldValue;
+import static org.apache.ignite.testframework.GridTestUtils.setFieldValue;
 import static org.apache.ignite.testframework.GridTestUtils.getFieldValueHierarchy;
 import static org.apache.ignite.testframework.config.GridTestProperties.BINARY_MARSHALLER_USE_SIMPLE_NAME_MAPPER;
 import static org.apache.ignite.testframework.config.GridTestProperties.IGNITE_CFG_PREPROCESSOR_CLS;
@@ -207,6 +210,9 @@ public abstract class GridAbstractTest extends JUnitAssertAware {
         }
     };
 
+    /** Classes for which you want to clear the static log. */
+    private static final Collection<Class<?>> clearStaticLogClasses = newSetFromMap(new ConcurrentHashMap<>());
+
     /** Allows easy repeating for test. */
     @Rule public transient RepeatRule repeatRule = new RepeatRule();
 
@@ -344,7 +350,8 @@ public abstract class GridAbstractTest extends JUnitAssertAware {
      * @throws Exception If failed.
      */
     protected void afterTestsStopped() throws Exception {
-        // No-op.
+        clearStaticLogClasses.forEach(this::clearStaticClassLog);
+        clearStaticLogClasses.clear();
     }
 
     /**
@@ -887,7 +894,7 @@ public abstract class GridAbstractTest extends JUnitAssertAware {
     protected IgniteEx startGrid(int idx) throws Exception {
         return (IgniteEx)startGrid(getTestIgniteInstanceName(idx));
     }
-    
+
     /**
      * Starts new client grid with given index.
      *
@@ -2680,6 +2687,35 @@ public abstract class GridAbstractTest extends JUnitAssertAware {
         }
     }
 
+    /**
+     * Clearing the static log for the class. <br/>
+     * There is a situation when class logs cannot be listened to although they
+     * are visible, for example, in a file. This happens when the test is in
+     * one of the suites and the static log was installed earlier and is not
+     * reset when the next test class is launched. To prevent this from
+     * happening, before starting all the tests in the test class, you need to
+     * reset the static class log.
+     *
+     * @param cls Class.
+     */
+    protected void clearStaticLog(Class<?> cls) {
+        assertNotNull(cls);
+
+        clearStaticLogClasses.add(cls);
+        clearStaticClassLog(cls);
+    }
+
+    /**
+     * Clearing the static log for the class.
+     *
+     * @param cls Class.
+     */
+    private void clearStaticClassLog(Class<?> cls) {
+        assertNotNull(cls);
+
+        ((AtomicReference<IgniteLogger>)getFieldValue(cls, "logRef")).set(null);
+        setFieldValue(cls, "log", null);
+    }
 
     /**
      * Returns metric set.
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 5c6be99..6d3a685 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,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 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;
@@ -187,6 +188,7 @@ import org.junit.runners.Suite;
 
     PartitionsEvictionTaskFailureHandlerTest.class,
     DropCacheContextDuringEvictionTest.class,
+    EvictPartitionInLogTest.class,
 
     IgniteExceptionInNioWorkerSelfTest.class,
     IgniteLocalNodeMapBeforeStartTest.class,