You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2021/01/13 11:00:21 UTC

[GitHub] [ignite] sk0x50 commented on a change in pull request #8645: IGNITE-12950

sk0x50 commented on a change in pull request #8645:
URL: https://github.com/apache/ignite/pull/8645#discussion_r556416281



##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionsStateValidator.java
##########
@@ -49,6 +54,13 @@
     /** Cache shared context. */
     private final GridCacheSharedContext<?, ?> cctx;
 
+    /**
+     * Collection of partitions that did not pass validation.
+     * This collection is supported and updated by coordinator node only.
+     * Represents the following mapping: group id -> set of partitions.
+     */
+    private Map<Integer, Set<Integer>> invalidParts = new ConcurrentHashMap<>();

Review comment:
       It looks like, this field is not used anywhere and can be removed.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionsStateValidator.java
##########
@@ -59,15 +71,19 @@ public GridDhtPartitionsStateValidator(GridCacheSharedContext<?, ?> cctx) {
     }
 
     /**
-     * Validates partition states - update counters and cache sizes for all nodes.
-     * If update counter value or cache size for the same partitions are different on some nodes
-     * method throws exception with full information about inconsistent partitions.
+     * Validates partition states - update counters and cache sizes

Review comment:
       What is the reason for formatting this Javadoc?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionsStateValidator.java
##########
@@ -99,15 +112,40 @@ public void validatePartitionCountersAndSizes(
 
         if (!cctx.cache().cacheGroup(top.groupId()).mvccEnabled()) { // TODO: Remove "if" clause in IGNITE-9451.
             // Validate cache sizes.
-            result = validatePartitionsSizes(top, messages, ignoringNodes);
+            resSize = validatePartitionsSizes(top, messages, ignoringNodes);
+        }
+
+        AffinityTopologyVersion topVer = fut.context().events().topologyVersion();
 
-            if (!result.isEmpty())
-                throw new IgniteCheckedException("Partitions cache sizes are inconsistent for " + fold(topVer, result));
+        if (!resUpdCnt.isEmpty() && !resSize.isEmpty()) {
+            error.append("Partitions cache size and update counters are inconsistent for ")
+                .append(fold(topVer, resUpdCnt, resSize));
         }
+        else if (!resUpdCnt.isEmpty() && resSize.isEmpty())
+            error.append("Partitions update counters are inconsistent for ").append(fold(topVer, resUpdCnt));
+        else if (resUpdCnt.isEmpty() && !resSize.isEmpty())
+            error.append("Partitions cache sizes are inconsistent for ").append(fold(topVer, resSize));
+
+        if (error.length() > 0) {
+            Set<Integer> parts = new HashSet<>(resUpdCnt.keySet());
+            parts.addAll(resSize.keySet());
+
+            invalidParts.putIfAbsent(top.groupId(), parts);
+
+            throw new IgniteCheckedException(error.toString());
+        }
+    }
+
+    /**
+     * Cleans up resources to avoid excessive memory usage.
+     */
+    public void cleanUp() {

Review comment:
       Please remove the unused method.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionsStateValidator.java
##########
@@ -329,4 +372,64 @@ private String fold(AffinityTopologyVersion topVer, Map<Integer, Map<UUID, Long>
 
         return sb.toString();
     }
+
+    /**
+     * Folds given map of invalid partition states to string representation
+     * in the following format:
+     * Part [id]: [consistentId=value meta=[updCnt=value, size=value]]
+     * @param topVer Topology version.
+     * @param invalidPartitionsCounters Invalid partitions counters map.
+     * @param invalidPartitionsSize Invalid partitions size map.
+     * @return value is String in the following format: Part [id]:
+     * [consistentId=value meta=[updCnt=value, size=value]]
+     */
+    private String fold(AffinityTopologyVersion topVer, Map<Integer, Map<UUID, Long>> invalidPartitionsCounters,
+        Map<Integer, Map<UUID, Long>> invalidPartitionsSize) {
+        SB sb = new SB();
+
+        NavigableMap<Integer, Map<UUID, IgnitePair<Long>>> sortedAllPartitions = new TreeMap<>();
+
+        Set<Integer> allKeys = new HashSet<>(invalidPartitionsCounters.keySet());
+
+        allKeys.addAll(invalidPartitionsSize.keySet());
+
+        for (Integer p : allKeys) {
+            Map<UUID, IgnitePair<Long>> map = new HashMap<>();
+
+            fillMapForPartition(invalidPartitionsCounters.get(p), map, true);
+            fillMapForPartition(invalidPartitionsSize.get(p), map, false);
+
+            sortedAllPartitions.put(p, map);
+        }
+
+        for (Map.Entry<Integer, Map<UUID, IgnitePair<Long>>> p : sortedAllPartitions.entrySet()) {
+            sb.a("Part ").a(p.getKey()).a(": [");
+            for (Map.Entry<UUID, IgnitePair<Long>> e : p.getValue().entrySet()) {
+                Object consistentId = cctx.discovery().node(topVer, e.getKey()).consistentId();
+                sb.a("consistentId=").a(consistentId).a(" meta=[updCnt=").a(e.getValue().get1())
+                    .a(", size=").a(e.getValue().get2()).a("] ");
+            }
+            sb.a("] ");
+        }
+
+        return sb.toString();
+    }
+
+    /**
+     * Add pair of counters and size in result map.

Review comment:
       `Adds`

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionsStateValidator.java
##########
@@ -329,4 +372,64 @@ private String fold(AffinityTopologyVersion topVer, Map<Integer, Map<UUID, Long>
 
         return sb.toString();
     }
+
+    /**
+     * Folds given map of invalid partition states to string representation
+     * in the following format:
+     * Part [id]: [consistentId=value meta=[updCnt=value, size=value]]
+     * @param topVer Topology version.
+     * @param invalidPartitionsCounters Invalid partitions counters map.
+     * @param invalidPartitionsSize Invalid partitions size map.
+     * @return value is String in the following format: Part [id]:
+     * [consistentId=value meta=[updCnt=value, size=value]]
+     */
+    private String fold(AffinityTopologyVersion topVer, Map<Integer, Map<UUID, Long>> invalidPartitionsCounters,

Review comment:
       It seems to me, it would be nice to rearrange parameters as follows:
   ```
       private String fold(
           AffinityTopologyVersion topVer,
           Map<Integer, Map<UUID, Long>> invalidPartitionsCounters,
           Map<Integer, Map<UUID, Long>> invalidPartitionsSize
       ) {
   ```

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsUpdateCountersAndSizeTest.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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;
+
+import java.util.HashSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+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.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Test correct behaviour of class to validate partitions update counters and
+ * cache sizes during exchange process
+ * {@link org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionsStateValidator}.
+ */
+public class GridCachePartitionsUpdateCountersAndSizeTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "cacheTest";
+
+    /** Listener for parsing patterns in log. */
+    private static final ListeningTestLogger testLog = new ListeningTestLogger(false, log);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        cfg.setCacheConfiguration(new CacheConfiguration(CACHE_NAME)
+            .setBackups(2)
+            .setAffinity(new RendezvousAffinityFunction(false, 32))
+        );
+
+        if (igniteInstanceName.endsWith("0"))
+            cfg.setGridLogger(testLog);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        testLog.clearListeners();
+
+        super.afterTest();
+    }
+
+    /**
+     * Four tests that partitions state validation works correctly and
+     * show partition size always:
+     * Start three-nodes grid,

Review comment:
       `Start three-node grid,` - it doesn't look like a complete sentence.

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionsStateValidator.java
##########
@@ -329,4 +372,64 @@ private String fold(AffinityTopologyVersion topVer, Map<Integer, Map<UUID, Long>
 
         return sb.toString();
     }
+
+    /**
+     * Folds given map of invalid partition states to string representation
+     * in the following format:
+     * Part [id]: [consistentId=value meta=[updCnt=value, size=value]]
+     * @param topVer Topology version.
+     * @param invalidPartitionsCounters Invalid partitions counters map.
+     * @param invalidPartitionsSize Invalid partitions size map.
+     * @return value is String in the following format: Part [id]:

Review comment:
       I think that the string representing a format should be placed on one line for better understanding.
   ```
        * @return value is String in the following format:
        *     Part [id]: [consistentId=value meta=[updCnt=value, size=value]]
   
   ```

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsUpdateCountersAndSizeTest.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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;
+
+import java.util.HashSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+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.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Test correct behaviour of class to validate partitions update counters and
+ * cache sizes during exchange process
+ * {@link org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionsStateValidator}.
+ */
+public class GridCachePartitionsUpdateCountersAndSizeTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "cacheTest";
+
+    /** Listener for parsing patterns in log. */
+    private static final ListeningTestLogger testLog = new ListeningTestLogger(false, log);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        cfg.setCacheConfiguration(new CacheConfiguration(CACHE_NAME)
+            .setBackups(2)
+            .setAffinity(new RendezvousAffinityFunction(false, 32))
+        );
+
+        if (igniteInstanceName.endsWith("0"))
+            cfg.setGridLogger(testLog);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        testLog.clearListeners();
+
+        super.afterTest();
+    }
+
+    /**
+     * Four tests that partitions state validation works correctly and
+     * show partition size always:
+     * Start three-nodes grid,
+     * @param cnt - partition counters are inconsistent(boolean)
+     * @param size - partition size are inconsistent(boolean)
+     * @throws Exception If failed.
+     */
+    private void startThreeNodesGrid(boolean cnt, boolean size) throws Exception {
+        SizeCounterLogListener lsnr = new SizeCounterLogListener();
+
+        IgniteEx ignite = startGrids(3);
+        ignite.cluster().active(true);

Review comment:
       The `active()` method is deprecated. Please use `ignite.cluster().state(ClusterState.ACTIVE);` instead.

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsUpdateCountersAndSizeTest.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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;
+
+import java.util.HashSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+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.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Test correct behaviour of class to validate partitions update counters and
+ * cache sizes during exchange process
+ * {@link org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionsStateValidator}.
+ */
+public class GridCachePartitionsUpdateCountersAndSizeTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "cacheTest";
+
+    /** Listener for parsing patterns in log. */
+    private static final ListeningTestLogger testLog = new ListeningTestLogger(false, log);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        cfg.setCacheConfiguration(new CacheConfiguration(CACHE_NAME)
+            .setBackups(2)
+            .setAffinity(new RendezvousAffinityFunction(false, 32))
+        );
+
+        if (igniteInstanceName.endsWith("0"))
+            cfg.setGridLogger(testLog);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        testLog.clearListeners();
+
+        super.afterTest();
+    }
+
+    /**
+     * Four tests that partitions state validation works correctly and
+     * show partition size always:
+     * Start three-nodes grid,
+     * @param cnt - partition counters are inconsistent(boolean)

Review comment:
       I think that `cnt` and `size` are not good enough. Perhaps, it can be changed to:
   `@param inconsistentCnt - {@code true} if it is expected that partition counters are inconsistent.`
   `@param inconsistentSize - {@code true} if it is expected that partition sizes are inconsistent.`
   What do you think?

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionsStateValidator.java
##########
@@ -329,4 +372,64 @@ private String fold(AffinityTopologyVersion topVer, Map<Integer, Map<UUID, Long>
 
         return sb.toString();
     }
+
+    /**
+     * Folds given map of invalid partition states to string representation
+     * in the following format:
+     * Part [id]: [consistentId=value meta=[updCnt=value, size=value]]
+     * @param topVer Topology version.
+     * @param invalidPartitionsCounters Invalid partitions counters map.
+     * @param invalidPartitionsSize Invalid partitions size map.
+     * @return value is String in the following format: Part [id]:
+     * [consistentId=value meta=[updCnt=value, size=value]]
+     */
+    private String fold(AffinityTopologyVersion topVer, Map<Integer, Map<UUID, Long>> invalidPartitionsCounters,
+        Map<Integer, Map<UUID, Long>> invalidPartitionsSize) {
+        SB sb = new SB();
+
+        NavigableMap<Integer, Map<UUID, IgnitePair<Long>>> sortedAllPartitions = new TreeMap<>();
+
+        Set<Integer> allKeys = new HashSet<>(invalidPartitionsCounters.keySet());
+
+        allKeys.addAll(invalidPartitionsSize.keySet());
+
+        for (Integer p : allKeys) {
+            Map<UUID, IgnitePair<Long>> map = new HashMap<>();
+
+            fillMapForPartition(invalidPartitionsCounters.get(p), map, true);
+            fillMapForPartition(invalidPartitionsSize.get(p), map, false);
+
+            sortedAllPartitions.put(p, map);
+        }
+
+        for (Map.Entry<Integer, Map<UUID, IgnitePair<Long>>> p : sortedAllPartitions.entrySet()) {
+            sb.a("Part ").a(p.getKey()).a(": [");
+            for (Map.Entry<UUID, IgnitePair<Long>> e : p.getValue().entrySet()) {
+                Object consistentId = cctx.discovery().node(topVer, e.getKey()).consistentId();
+                sb.a("consistentId=").a(consistentId).a(" meta=[updCnt=").a(e.getValue().get1())
+                    .a(", size=").a(e.getValue().get2()).a("] ");
+            }
+            sb.a("] ");
+        }
+
+        return sb.toString();
+    }
+
+    /**
+     * Add pair of counters and size in result map.
+     * @param sourceMap PartitionCounters or PartitionSize
+     * @param resultMap  result map with pair of values
+     */
+    private void fillMapForPartition(Map<UUID, Long> sourceMap,

Review comment:
       In accordance with https://cwiki.apache.org/confluence/display/IGNITE/Abbreviation+Rules, `sourceMap` should be changed to `srcMap`.
   Also, please put each parameter on a separate line:
   ```
       private void fillMapForPartition(
           Map<UUID, Long> sourceMap,
           Map<UUID, IgnitePair<Long>> resultMap,
           boolean isFirst
       ) {
   ```
   https://cwiki.apache.org/confluence/display/IGNITE/Coding+Guidelines#CodingGuidelines-MethodArguments

##########
File path: modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionsStateValidator.java
##########
@@ -82,15 +98,12 @@ public void validatePartitionCountersAndSizes(
                 ignoringNodes.add(evt.eventNode().id());
         }
 
-        AffinityTopologyVersion topVer = fut.context().events().topologyVersion();
-
-        // Validate update counters.
-        Map<Integer, Map<UUID, Long>> result = validatePartitionsUpdateCounters(top, messages, ignoringNodes);
+        StringBuilder error = new StringBuilder();
 
-        if (!result.isEmpty())
-            throw new IgniteCheckedException("Partitions update counters are inconsistent for " + fold(topVer, result));
+        Map<Integer, Map<UUID, Long>> resUpdCnt = validatePartitionsUpdateCounters(top, messages, ignoringNodes);
+        Map<Integer, Map<UUID, Long>> resSize = Collections.emptyMap();
 
-        // For sizes validation ignore also nodes which are not able to send cache sizes.
+        //For sizes validation ignore also nodes which are not able to send cache sizes.

Review comment:
       Please add whitespace: `// For sizes...`

##########
File path: modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionsUpdateCountersAndSizeTest.java
##########
@@ -0,0 +1,245 @@
+/*
+ * 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;
+
+import java.util.HashSet;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+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.testframework.ListeningTestLogger;
+import org.apache.ignite.testframework.LogListener;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Test correct behaviour of class to validate partitions update counters and
+ * cache sizes during exchange process
+ * {@link org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionsStateValidator}.
+ */
+public class GridCachePartitionsUpdateCountersAndSizeTest extends GridCommonAbstractTest {
+    /** Cache name. */
+    private static final String CACHE_NAME = "cacheTest";
+
+    /** Listener for parsing patterns in log. */
+    private static final ListeningTestLogger testLog = new ListeningTestLogger(false, log);

Review comment:
       Please do not use the deprecated constructor. It can be changed to `ListeningTestLogger(log);` I think.
   Moreover, it does not seem good to use non-static field `log` for the initialization of the static field `testLog`.




----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
users@infra.apache.org