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/07/13 12:58:35 UTC

[ignite] branch master updated: IGNITE-13245 Rebalance future might hangs in no final state though all partitions are owned - Fixes #8025

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 d49580e  IGNITE-13245 Rebalance future might hangs in no final state though all partitions are owned - Fixes #8025
d49580e is described below

commit d49580e22fb8e7fe949bddb5c08669662eaa16eb
Author: vd_pyatkov <vl...@gmail.com>
AuthorDate: Mon Jul 13 15:58:11 2020 +0300

    IGNITE-13245 Rebalance future might hangs in no final state though all partitions are owned - Fixes #8025
    
    Signed-off-by: Ivan Rakov <iv...@gmail.com>
---
 .../internal/commandline/CommandHandler.java       |   8 +-
 .../dht/preloader/GridDhtPartitionDemander.java    |   2 +
 ...lanceIsProcessingWhenAssignmentIsEmptyTest.java | 157 +++++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite2.java   |   2 +
 4 files changed, 167 insertions(+), 2 deletions(-)

diff --git a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
index 52566f6..32875b7 100644
--- a/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
+++ b/modules/control-utility/src/main/java/org/apache/ignite/internal/commandline/CommandHandler.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.commandline;
 import java.io.File;
 import java.time.Duration;
 import java.time.LocalDateTime;
+import java.time.format.DateTimeFormatter;
 import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
@@ -124,6 +125,9 @@ public class CommandHandler {
     /** */
     private Object lastOperationRes;
 
+    /** Date format. */
+    private final DateTimeFormatter formatter = DateTimeFormatter.ofPattern("yyyy-MM-dd'T'HH:mm:ss.SSS");
+
     /**
      * @param args Arguments to parse and apply.
      */
@@ -217,7 +221,7 @@ public class CommandHandler {
         logger.info("Control utility [ver. " + ACK_VER_STR + "]");
         logger.info(COPYRIGHT);
         logger.info("User: " + System.getProperty("user.name"));
-        logger.info("Time: " + startTime);
+        logger.info("Time: " + startTime.format(formatter));
 
         String commandName = "";
 
@@ -368,7 +372,7 @@ public class CommandHandler {
             if (nonNull(err))
                 logger.info("Error stack trace:" + System.lineSeparator() + X.getFullStackTrace(err));
 
-            logger.info("Control utility has completed execution at: " + endTime);
+            logger.info("Control utility has completed execution at: " + endTime.format(formatter));
             logger.info("Execution time: " + diff.toMillis() + " ms");
 
             Arrays.stream(logger.getHandlers())
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 514dd1f..8969b31 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -314,6 +314,8 @@ public class GridDhtPartitionDemander {
 
                 if (oldFut.isInitial())
                     oldFut.onDone(true);
+                else if (!oldFut.isDone())
+                    oldFut.tryCancel();
 
                 ((GridFutureAdapter)grp.preloader().syncFuture()).onDone();
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/RebalanceIsProcessingWhenAssignmentIsEmptyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/RebalanceIsProcessingWhenAssignmentIsEmptyTest.java
new file mode 100644
index 0000000..11e7d9e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/RebalanceIsProcessingWhenAssignmentIsEmptyTest.java
@@ -0,0 +1,157 @@
+/*
+ * 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.ArrayList;
+import java.util.List;
+import org.apache.ignite.cache.affinity.AffinityFunctionContext;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+/**
+ * Checks that a rebalance future completed when all partitions are rebalanced.
+ */
+public class RebalanceIsProcessingWhenAssignmentIsEmptyTest extends GridCommonAbstractTest {
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName)
+            .setCommunicationSpi(new TestRecordingCommunicationSpi())
+            .setConsistentId(igniteInstanceName)
+            .setCacheConfiguration(new CacheConfiguration(DEFAULT_CACHE_NAME)
+                .setBackups(1)
+                .setAffinity(new TestAffinity(getTestIgniteInstanceName(0),
+                    getTestIgniteInstanceName(1),
+                    getTestIgniteInstanceName(2),
+                    getTestIgniteInstanceName(3))));
+    }
+
+    /**
+     * Test with specific affinity on a default cache.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void test() throws Exception {
+        IgniteEx ignite0 = startGrids(3);
+
+        ignite0.cachex(DEFAULT_CACHE_NAME).put(0, 0);
+        ignite0.cachex(DEFAULT_CACHE_NAME).put(1, 0);
+
+        awaitPartitionMapExchange();
+
+        TestRecordingCommunicationSpi.spi(ignite0).blockMessages((node, msg) -> {
+            if (msg instanceof GridDhtPartitionSupplyMessage) {
+                GridDhtPartitionSupplyMessage supplyMsg = (GridDhtPartitionSupplyMessage)msg;
+
+                return supplyMsg.groupId() == CU.cacheId(DEFAULT_CACHE_NAME) &&
+                    node.consistentId().equals(getTestIgniteInstanceName(1));
+            }
+
+            return false;
+        });
+
+        startGrid(3);
+
+        TestRecordingCommunicationSpi.spi(ignite0).waitForBlocked();
+
+        stopGrid(0);
+
+        awaitPartitionMapExchange();
+
+        IgniteInternalFuture fut = grid(1).context().cache().cacheGroup(CU.cacheId(DEFAULT_CACHE_NAME)).preloader().rebalanceFuture();
+
+        assertTrue("Rebalance completed but this rebalance future is not complete, fut=" + fut, fut.isDone());
+    }
+
+    /**
+     * Tets affinity function.
+     * It gives same assignment on second node in topology of three nodes and differences in fours.
+     */
+    public static class TestAffinity extends RendezvousAffinityFunction {
+        /** Nodes consistence ids. */
+        String[] nodeConsistentIds;
+
+        /**
+         * @param nodes Nodes consistence ids.
+         */
+        public TestAffinity(String... nodes) {
+            super(false,2);
+
+            this.nodeConsistentIds = nodes;
+        }
+
+        /** {@inheritDoc} */
+        @Override public List<List<ClusterNode>> assignPartitions(AffinityFunctionContext affCtx) {
+            int nodes = affCtx.currentTopologySnapshot().size();
+
+            if (nodes != 4 && nodes != 3)
+                return super.assignPartitions(affCtx);
+
+            List<List<ClusterNode>> assignment = new ArrayList<>();
+
+            assignment.add(new ArrayList<>(2));
+            assignment.add(new ArrayList<>(2));
+
+            assignment.get(1).add(null);
+            assignment.get(1).add(null);
+            assignment.get(0).add(null);
+            assignment.get(0).add(null);
+
+            if (nodes == 3) {
+                for (ClusterNode node : affCtx.currentTopologySnapshot())
+                    if (nodeConsistentIds[0].equals(node.consistentId()) ||
+                        nodeConsistentIds[3].equals(node.consistentId())) {
+                        assignment.get(0).set(0, node);
+                        assignment.get(1).set(1, node);
+                    }
+                    else if (nodeConsistentIds[1].equals(node.consistentId()))
+                        assignment.get(1).set(0, node);
+                    else if (nodeConsistentIds[2].equals(node.consistentId()))
+                        assignment.get(0).set(1, node);
+                    else
+                        throw new AssertionError("Unexpected node consistent id is " + node.consistentId());
+            }
+            else {
+                for (ClusterNode node : affCtx.currentTopologySnapshot()) {
+                    if (nodeConsistentIds[0].equals(node.consistentId()))
+                        assignment.get(0).set(1, node);
+                    else if (nodeConsistentIds[1].equals(node.consistentId())) {
+                        assignment.get(1).set(0, node);
+                        assignment.get(0).set(0, node);
+                    }
+                    else if (nodeConsistentIds[2].equals(node.consistentId()))
+                        assignment.get(1).set(1, node);
+                    else if (!nodeConsistentIds[3].equals(node.consistentId()))
+                        throw new AssertionError("Unexpected node consistent id is " + node.consistentId());
+                }
+            }
+
+            return assignment;
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 6cebab9..a8325d2 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -101,6 +101,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheCo
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedBackupNodeFailureRecoveryTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCrossCacheTxNearEnabledSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteTxConsistencyColocatedRestartSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.dht.RebalanceIsProcessingWhenAssignmentIsEmptyTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.IgniteCacheContainsKeyColocatedAtomicSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.IgniteCacheContainsKeyNearAtomicSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicNearEvictionEventSelfTest;
@@ -269,6 +270,7 @@ public class IgniteCacheTestSuite2 {
         GridTestUtils.addTestIfNeeded(suite, GridCachePartitionedPreloadLifecycleSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadDelayedSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridCacheDhtPreloadWaitForBackupsTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, RebalanceIsProcessingWhenAssignmentIsEmptyTest.class, ignoredTests);
 
         GridTestUtils.addTestIfNeeded(suite, CacheDhtLocalPartitionAfterRemoveSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, CacheLoadingConcurrentGridStartSelfTest.class, ignoredTests);