You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sk...@apache.org on 2020/12/10 10:21:25 UTC

[ignite] branch master updated: IGNITE-13353 Got rid of unnecessary rebalance on starting new cache.

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

sk0x50 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 afc73ec  IGNITE-13353 Got rid of unnecessary rebalance on starting new cache.
afc73ec is described below

commit afc73ece389851aa0f5ce1bd422035dc31d0a2a4
Author: sergeyuttsel <ut...@gmail.com>
AuthorDate: Thu Dec 10 13:20:49 2020 +0300

    IGNITE-13353 Got rid of unnecessary rebalance on starting new cache.
    
    Signed-off-by: Slava Koptilin <sl...@gmail.com>
---
 .../preloader/GridDhtPartitionsExchangeFuture.java |  34 +++--
 .../java/org/apache/ignite/TestStorageUtils.java   | 104 +++++++++++++
 .../ignite/cache/NoUnnecessaryRebalancesTest.java  | 163 +++++++++++++++++++++
 .../ignite/testsuites/IgnitePdsTestSuite4.java     |   2 +
 4 files changed, 293 insertions(+), 10 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 4a9435c..65e4b4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -134,6 +134,8 @@ import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteRunnable;
 import org.jetbrains.annotations.Nullable;
 
+import static java.util.Collections.emptySet;
+import static java.util.stream.Stream.concat;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_LONG_OPERATIONS_DUMP_TIMEOUT_LIMIT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PARTITION_RELEASE_FUTURE_DUMP_THRESHOLD;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_THREAD_DUMP_ON_EXCHANGE_TIMEOUT;
@@ -1237,7 +1239,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         top.update(null,
                             clientTop.partitionMap(true),
                             clientTop.fullUpdateCounters(),
-                            Collections.emptySet(),
+                            emptySet(),
                             null,
                             null,
                             null,
@@ -3933,7 +3935,7 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                 assert firstDiscoEvt instanceof DiscoveryCustomEvent;
 
                 if (activateCluster() || changedBaseline())
-                    assignPartitionsStates(true);
+                    assignPartitionsStates(null);
 
                 DiscoveryCustomMessage discoveryCustomMessage = ((DiscoveryCustomEvent) firstDiscoEvt).customMessage();
 
@@ -3944,20 +3946,26 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         if (!F.isEmpty(caches))
                             resetLostPartitions(caches);
 
-                        assignPartitionsStates(true);
+                        Set<Integer> cacheGroupsToResetOwners = concat(exchActions.cacheGroupsToStart().stream()
+                                .map(grp -> grp.descriptor().groupId()),
+                            exchActions.cachesToResetLostPartitions().stream()
+                                .map(CU::cacheId))
+                            .collect(Collectors.toSet());
+
+                        assignPartitionsStates(cacheGroupsToResetOwners);
                     }
                 }
                 else if (discoveryCustomMessage instanceof SnapshotDiscoveryMessage
                     && ((SnapshotDiscoveryMessage)discoveryCustomMessage).needAssignPartitions()) {
                     markAffinityReassign();
 
-                    assignPartitionsStates(true);
+                    assignPartitionsStates(null);
                 }
             }
             else if (exchCtx.events().hasServerJoin())
-                assignPartitionsStates(true);
+                assignPartitionsStates(null);
             else if (exchCtx.events().hasServerLeft())
-                assignPartitionsStates(false);
+                assignPartitionsStates(emptySet());
 
             // Validation should happen after resetting owners to avoid false desync reporting.
             validatePartitionsState();
@@ -4248,9 +4256,10 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
     }
 
     /**
-     * @param resetOwners True if reset partitions state needed, false otherwise.
+     * @param cacheGroupsToResetOwners Set of cache groups which need to reset partitions state,
+     *                                 null if reset partitions state for all cache groups needed
      */
-    private void assignPartitionsStates(boolean resetOwners) {
+    private void assignPartitionsStates(Set<Integer> cacheGroupsToResetOwners) {
         Map<String, List<SupplyPartitionInfo>> supplyInfoMap = log.isInfoEnabled() ?
             new ConcurrentHashMap<>() : null;
 
@@ -4266,12 +4275,17 @@ public class GridDhtPartitionsExchangeFuture extends GridDhtTopologyFutureAdapte
                         : cctx.exchange().clientTopology(grpDesc.groupId(), events().discoveryCache());
 
                     if (CU.isPersistentCache(grpDesc.config(), cctx.gridConfig().getDataStorageConfiguration())) {
-                        List<SupplyPartitionInfo> list = assignPartitionStates(top, resetOwners);
+                        List<SupplyPartitionInfo> list;
+
+                        if (cacheGroupsToResetOwners == null || cacheGroupsToResetOwners.contains(grpDesc.groupId()))
+                            list = assignPartitionStates(top, true);
+                        else
+                            list = assignPartitionStates(top, false);
 
                         if (supplyInfoMap != null && !F.isEmpty(list))
                             supplyInfoMap.put(grpDesc.cacheOrGroupName(), list);
                     }
-                    else if (resetOwners)
+                    else if (cacheGroupsToResetOwners == null)
                         assignPartitionSizes(top);
 
                     return null;
diff --git a/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java b/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java
new file mode 100644
index 0000000..17ff241
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/TestStorageUtils.java
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+import org.apache.ignite.binary.BinaryObject;
+import org.apache.ignite.cache.CacheEntry;
+import org.apache.ignite.internal.pagemem.wal.record.DataEntry;
+import org.apache.ignite.internal.processors.cache.CacheObjectImpl;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.GridCacheOperation;
+import org.apache.ignite.internal.processors.cache.KeyCacheObject;
+import org.apache.ignite.internal.processors.cache.KeyCacheObjectImpl;
+import org.apache.ignite.internal.processors.cache.persistence.GridCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.persistence.IgniteCacheDatabaseSharedManager;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.typedef.internal.U;
+
+/**
+ * Test methods for storage manipulation.
+ */
+public class TestStorageUtils {
+    /**
+     * Corrupts data entry.
+     *
+     * @param ctx Context.
+     * @param key Key.
+     * @param breakCntr Break counter.
+     * @param breakData Break data.
+     * @param ver GridCacheVersion to use.
+     * @param brokenValPostfix Postfix to add to value if breakData flag is set to true.
+     */
+    public static void corruptDataEntry(
+        GridCacheContext<?, ?> ctx,
+        Object key,
+        boolean breakCntr,
+        boolean breakData,
+        GridCacheVersion ver,
+        String brokenValPostfix
+    ) {
+        int partId = ctx.affinity().partition(key);
+
+        try {
+            long updateCntr = ctx.topology().localPartition(partId).updateCounter();
+
+            CacheEntry<Object, Object> e = ctx.cache().keepBinary().getEntry(key);
+
+            Object valToPut = e.getValue();
+
+            KeyCacheObject keyCacheObj = e.getKey() instanceof BinaryObject ?
+                (KeyCacheObject)e.getKey() :
+                new KeyCacheObjectImpl(e.getKey(), null, partId);
+
+            if (breakCntr)
+                updateCntr++;
+
+            if (breakData)
+                valToPut = e.getValue().toString() + brokenValPostfix;
+
+            // Create data entry
+
+            DataEntry dataEntry = new DataEntry(
+                ctx.cacheId(),
+                keyCacheObj,
+                new CacheObjectImpl(valToPut, null),
+                GridCacheOperation.UPDATE,
+                new GridCacheVersion(),
+                ver,
+                0L,
+                partId,
+                updateCntr
+            );
+
+            IgniteCacheDatabaseSharedManager db = ctx.shared().database();
+
+            db.checkpointReadLock();
+
+            try {
+                U.invoke(GridCacheDatabaseSharedManager.class, db, "applyUpdate", ctx, dataEntry,
+                    false);
+            }
+            finally {
+                db.checkpointReadUnlock();
+            }
+        }
+        catch (IgniteCheckedException e) {
+            e.printStackTrace();
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/NoUnnecessaryRebalancesTest.java b/modules/core/src/test/java/org/apache/ignite/cache/NoUnnecessaryRebalancesTest.java
new file mode 100644
index 0000000..a972c6d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/cache/NoUnnecessaryRebalancesTest.java
@@ -0,0 +1,163 @@
+/*
+ * 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.cache;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.cluster.ClusterState;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.internal.processors.cache.GridCacheContext;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessage;
+import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
+import org.apache.ignite.internal.util.typedef.internal.CU;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+import org.junit.Test;
+
+import static org.apache.ignite.TestStorageUtils.corruptDataEntry;
+
+/**
+ * Tests check that unnecessary rebalances doesn't happen
+ */
+public class NoUnnecessaryRebalancesTest extends GridCommonAbstractTest {
+    /** */
+    private static final String CACHE_NAME = "testCache";
+
+    /** */
+    private static final int nodeCount = 3;
+
+    /**
+     * @return Grid test configuration.
+     * @throws Exception If failed.
+     */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCommunicationSpi(new SpecialSpi());
+
+        cfg.setDataStorageConfiguration(new DataStorageConfiguration().setDefaultDataRegionConfiguration(
+            new DataRegionConfiguration().setPersistenceEnabled(true).setMaxSize(200 * 1024 * 1024)
+        ));
+
+        return cfg;
+    }
+
+    /**
+     * Test check that cache creation doesn't invoke rebalance on cache in other cache group
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNoRebalancesOnCacheCreation() throws Exception {
+        startGrids(nodeCount);
+
+        Ignite g0 = grid(0);
+
+        g0.cluster().state(ClusterState.ACTIVE);
+
+        g0.createCache(getCacheConfiguration(0));
+
+        awaitPartitionMapExchange();
+
+        IgniteCache<Object, Object> cache0 = g0.cache(CACHE_NAME + 0);
+
+        for (int i = 0; i < 100; i++)
+            cache0.put(i, i);
+
+        awaitPartitionMapExchange();
+
+        GridCacheContext<Object, Object> cacheCtx0 = grid(0).cachex(CACHE_NAME + 0).context();
+
+        corruptDataEntry(cacheCtx0, 1, true, false, new GridCacheVersion(0, 0, 0), "broken");
+
+        g0.createCache(getCacheConfiguration(1));
+
+        awaitPartitionMapExchange(true, true, null);
+
+        Assert.assertFalse(SpecialSpi.rebGrpIds.contains(CU.cacheId(CACHE_NAME + 0)));
+    }
+
+    /** */
+    private CacheConfiguration<Object, Object> getCacheConfiguration(int idx) {
+        return new CacheConfiguration<>(CACHE_NAME + idx)
+            .setBackups(2)
+            .setAffinity(new RendezvousAffinityFunction().setPartitions(8));
+    }
+
+    /**
+     * Wrapper of communication spi to detect on which cache groups rebalances were happened.
+     */
+    public static class SpecialSpi extends TestRecordingCommunicationSpi {
+        /** Cache groups on which rebalances were happened */
+        public static final Set<Integer> rebGrpIds = new HashSet<>();
+
+        /** Lock object. */
+        private static final Object mux = new Object();
+
+        /** */
+        public static Set<Integer> allRebalances() {
+            synchronized (mux) {
+                return Collections.unmodifiableSet(rebGrpIds);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg, IgniteInClosure<IgniteException> ackC) throws IgniteSpiException {
+            if (((GridIoMessage)msg).message() instanceof GridDhtPartitionSupplyMessage) {
+                GridDhtPartitionSupplyMessage supplyMsg = (GridDhtPartitionSupplyMessage) ((GridIoMessage)msg).message();
+
+                synchronized (mux) {
+                    rebGrpIds.add(supplyMsg.groupId());
+                }
+            }
+
+            super.sendMessage(node, msg, ackC);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
index d634395..392c2b9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite4.java
@@ -22,6 +22,7 @@ import java.util.Collection;
 import java.util.List;
 import org.apache.ignite.cache.BreakRebalanceChainTest;
 import org.apache.ignite.cache.CircledRebalanceTest;
+import org.apache.ignite.cache.NoUnnecessaryRebalancesTest;
 import org.apache.ignite.cache.NotOptimizedRebalanceTest;
 import org.apache.ignite.cache.RebalanceAfterResettingLostPartitionTest;
 import org.apache.ignite.cache.RebalanceCancellationTest;
@@ -115,6 +116,7 @@ public class IgnitePdsTestSuite4 {
         GridTestUtils.addTestIfNeeded(suite, OffHeapLockStackTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, HistoricalReservationTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, CircledRebalanceTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, NoUnnecessaryRebalancesTest.class, ignoredTests);
 
         GridTestUtils.addTestIfNeeded(suite, IgnitePdsCacheEntriesExpirationTest.class, ignoredTests);