You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2020/12/09 11:56:17 UTC

[ignite] branch master updated: IGNITE-13814 restorePartitionStates moved to sys pool instead of striped pool. - Fixes #8542.

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

sergeychugunov 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 f852ba7  IGNITE-13814 restorePartitionStates moved to sys pool instead of striped pool. - Fixes #8542.
f852ba7 is described below

commit f852ba7c9c0cc1eb3c02d59d2ce0ba5e69dc8566
Author: ibessonov <be...@gmail.com>
AuthorDate: Wed Dec 9 14:55:11 2020 +0300

    IGNITE-13814 restorePartitionStates moved to sys pool instead of striped pool. - Fixes #8542.
    
    Signed-off-by: Sergey Chugunov <se...@gmail.com>
---
 .../internal/processors/cache/GridCacheProcessor.java    | 16 ++++++++--------
 1 file changed, 8 insertions(+), 8 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index 4a1aceb..ae33bf8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -33,6 +33,7 @@ import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
@@ -150,7 +151,6 @@ import org.apache.ignite.internal.suggestions.GridPerformanceSuggestions;
 import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.IgniteCollectors;
 import org.apache.ignite.internal.util.InitializationProtector;
-import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -192,7 +192,6 @@ import static java.lang.String.format;
 import static java.util.Arrays.asList;
 import static java.util.Objects.isNull;
 import static java.util.Objects.nonNull;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_ALLOW_START_CACHES_IN_PARALLEL;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CACHE_REMOVED_ENTRIES_TTL;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
@@ -5465,12 +5464,12 @@ public class GridCacheProcessor extends GridProcessorAdapter {
 
             AtomicReference<IgniteCheckedException> restoreStateError = new AtomicReference<>();
 
-            StripedExecutor stripedExec = ctx.getStripedExecutorService();
+            ExecutorService sysPool = ctx.getSystemExecutorService();
 
-            int roundRobin = 0;
+            CountDownLatch completionLatch = new CountDownLatch(forGroups.size());
 
             for (CacheGroupContext grp : forGroups) {
-                stripedExec.execute(roundRobin % stripedExec.stripesCount(), () -> {
+                sysPool.execute(() -> {
                     try {
                         long processed = grp.offheap().restorePartitionStates(partitionStates);
 
@@ -5487,14 +5486,15 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                                 : new IgniteCheckedException(e)
                         );
                     }
+                    finally {
+                        completionLatch.countDown();
+                    }
                 });
-
-                roundRobin++;
             }
 
             try {
                 // Await completion restore state tasks in all stripes.
-                stripedExec.awaitComplete();
+                completionLatch.await();
             }
             catch (InterruptedException e) {
                 throw new IgniteInterruptedException(e);