Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 3545119104 for ; Mon, 11 Apr 2016 14:26:36 +0000 (UTC) Received: (qmail 48458 invoked by uid 500); 11 Apr 2016 14:26:36 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 48040 invoked by uid 500); 11 Apr 2016 14:26:34 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 45230 invoked by uid 99); 11 Apr 2016 14:25:33 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 11 Apr 2016 14:25:33 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 7C8E7EAB5D; Mon, 11 Apr 2016 14:25:29 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: vozerov@apache.org To: commits@ignite.apache.org Date: Mon, 11 Apr 2016 14:25:59 -0000 Message-Id: <6eb76e4c664e4e3aac7ce84daec783cd@git.apache.org> In-Reply-To: <15dbdc0d71374accaf29cbb3b7b59af4@git.apache.org> References: <15dbdc0d71374accaf29cbb3b7b59af4@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [32/50] [abbrv] ignite git commit: gridgain-7.5.11 Affinity history cleanup ignoring client discovery event. (cherry picked from commit e7ab8eef504cdcf8987941e8b7a552ada451aec8) gridgain-7.5.11 Affinity history cleanup ignoring client discovery event. (cherry picked from commit e7ab8eef504cdcf8987941e8b7a552ada451aec8) Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/4d435537 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/4d435537 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/4d435537 Branch: refs/heads/ignite-1786 Commit: 4d4355375497134e881188a7fcad831eb8a9013e Parents: da47901 Author: sboikov Authored: Thu Apr 7 10:03:09 2016 +0300 Committer: sboikov Committed: Thu Apr 7 10:03:09 2016 +0300 ---------------------------------------------------------------------- .../affinity/GridAffinityAssignment.java | 15 + .../affinity/GridAffinityAssignmentCache.java | 72 +++- .../cache/CacheAffinitySharedManager.java | 10 - .../cache/GridCacheAffinityManager.java | 11 - .../GridCachePartitionExchangeManager.java | 16 - .../affinity/AffinityHistoryCleanupTest.java | 414 +++++++++++++++++++ .../ignite/testsuites/IgniteCacheTestSuite.java | 2 + 7 files changed, 486 insertions(+), 54 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/4d435537/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java index 92908cc..568e4e8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignment.java @@ -57,6 +57,9 @@ public class GridAffinityAssignment implements Serializable { /** */ private transient List> idealAssignment; + /** */ + private final boolean clientEvtChange; + /** * Constructs cached affinity calculations item. * @@ -66,6 +69,7 @@ public class GridAffinityAssignment implements Serializable { this.topVer = topVer; primary = new HashMap<>(); backup = new HashMap<>(); + clientEvtChange = false; } /** @@ -86,6 +90,7 @@ public class GridAffinityAssignment implements Serializable { primary = new HashMap<>(); backup = new HashMap<>(); + clientEvtChange = false; initPrimaryBackupMaps(); } @@ -101,6 +106,16 @@ public class GridAffinityAssignment implements Serializable { idealAssignment = aff.idealAssignment; primary = aff.primary; backup = aff.backup; + + clientEvtChange = true; + } + + /** + * @return {@code True} if related discovery event did not not cause affinity assignment change and + * this assignment is just reference to the previous one. + */ + public boolean clientEventChange() { + return clientEvtChange; } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/4d435537/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java index 0cacf68..a81b34d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java @@ -27,7 +27,7 @@ import java.util.UUID; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentNavigableMap; import java.util.concurrent.ConcurrentSkipListMap; -import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicReference; import org.apache.ignite.IgniteCheckedException; import org.apache.ignite.IgniteException; @@ -48,12 +48,17 @@ import org.apache.ignite.lang.IgnitePredicate; import org.jetbrains.annotations.Nullable; import org.jsr166.ConcurrentHashMap8; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE; +import static org.apache.ignite.IgniteSystemProperties.getInteger; import static org.apache.ignite.internal.events.DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT; /** * Affinity cached function. */ public class GridAffinityAssignmentCache { + /** Cleanup history size. */ + private final int MAX_HIST_SIZE = getInteger(IGNITE_AFFINITY_HISTORY_SIZE, 500); + /** Cache name. */ private final String cacheName; @@ -96,6 +101,12 @@ public class GridAffinityAssignmentCache { /** Node stop flag. */ private volatile IgniteCheckedException stopErr; + /** History size ignoring client events changes. */ + private final AtomicInteger histSize = new AtomicInteger(); + + /** Full history size. */ + private final AtomicInteger fullHistSize = new AtomicInteger(); + /** * Constructs affinity cached calculations. * @@ -171,6 +182,8 @@ public class GridAffinityAssignmentCache { entry.getValue().onDone(topVer); } } + + onHistoryAdded(assignment); } /** @@ -299,6 +312,8 @@ public class GridAffinityAssignmentCache { entry.getValue().onDone(topVer); } } + + onHistoryAdded(assignmentCpy); } /** @@ -309,22 +324,6 @@ public class GridAffinityAssignmentCache { } /** - * Clean up outdated cache items. - * - * @param topVer Actual topology version, older versions will be removed. - */ - public void cleanUpCache(AffinityTopologyVersion topVer) { - if (log.isDebugEnabled()) - log.debug("Cleaning up cache for version [locNodeId=" + ctx.localNodeId() + - ", topVer=" + topVer + ']'); - - for (Iterator it = affCache.keySet().iterator(); it.hasNext(); ) { - if (it.next().compareTo(topVer) < 0) - it.remove(); - } - } - - /** * @param topVer Topology version. * @return Affinity assignment. */ @@ -530,6 +529,45 @@ public class GridAffinityAssignmentCache { } /** + * @param aff Added affinity assignment. + */ + private void onHistoryAdded(GridAffinityAssignment aff) { + int fullSize = fullHistSize.incrementAndGet(); + + int size; + + if (aff.clientEventChange()) + size = histSize.get(); + else + size = histSize.incrementAndGet(); + + int rmvCnt = size - MAX_HIST_SIZE; + + if (rmvCnt <= 0) { + if (fullSize > MAX_HIST_SIZE * 2) + rmvCnt = MAX_HIST_SIZE; + } + + if (rmvCnt > 0) { + Iterator it = affCache.values().iterator(); + + while (it.hasNext() && rmvCnt > 0) { + GridAffinityAssignment aff0 = it.next(); + + it.remove(); + + rmvCnt--; + + if (!aff0.clientEventChange()) + histSize.decrementAndGet(); + + fullHistSize.decrementAndGet(); + } + } + } + + + /** * Affinity ready future. Will remove itself from ready futures map. */ private class AffinityReadyFuture extends GridFutureAdapter { http://git-wip-us.apache.org/repos/asf/ignite/blob/4d435537/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java index eb85288..1aedf4e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheAffinitySharedManager.java @@ -925,16 +925,6 @@ public class CacheAffinitySharedManager extends GridCacheSharedManagerAdap } /** - * @param topVer Actual topology version. - */ - public void cleanUpCache(AffinityTopologyVersion topVer) { - for (CacheHolder cache : caches.values()) { - if (cache.client()) - cache.affinity().cleanUpCache(topVer); - } - } - - /** * @param fut Exchange future. * @throws IgniteCheckedException If failed. */ http://git-wip-us.apache.org/repos/asf/ignite/blob/4d435537/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java index e264043..f1767e0 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java @@ -155,17 +155,6 @@ public class GridCacheAffinityManager extends GridCacheManagerAdapter { } /** - * Clean up outdated cache items. - * - * @param topVer Actual topology version, older versions will be removed. - */ - public void cleanUpCache(AffinityTopologyVersion topVer) { - assert !cctx.isLocal(); - - aff.cleanUpCache(topVer); - } - - /** * @param topVer Topology version. * @return Affinity assignments. */ http://git-wip-us.apache.org/repos/asf/ignite/blob/4d435537/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java index 2b7b621..78a8d6c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java @@ -89,9 +89,7 @@ import org.jsr166.ConcurrentHashMap8; import org.jsr166.ConcurrentLinkedDeque8; import static java.util.concurrent.TimeUnit.MILLISECONDS; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE; import static org.apache.ignite.IgniteSystemProperties.IGNITE_PRELOAD_RESEND_TIMEOUT; -import static org.apache.ignite.IgniteSystemProperties.getInteger; import static org.apache.ignite.IgniteSystemProperties.getLong; import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STARTED; import static org.apache.ignite.events.EventType.EVT_NODE_FAILED; @@ -109,9 +107,6 @@ public class GridCachePartitionExchangeManager extends GridCacheSharedMana /** Exchange history size. */ private static final int EXCHANGE_HISTORY_SIZE = 1000; - /** Cleanup history size. */ - public static final int EXCH_FUT_CLEANUP_HISTORY_SIZE = getInteger(IGNITE_AFFINITY_HISTORY_SIZE, 500); - /** Atomic reference for pending timeout object. */ private AtomicReference pendingResend = new AtomicReference<>(); @@ -962,17 +957,6 @@ public class GridCachePartitionExchangeManager extends GridCacheSharedMana skipped++; - if (skipped == EXCH_FUT_CLEANUP_HISTORY_SIZE) { - if (err == null) { - for (GridCacheContext cacheCtx : cctx.cacheContexts()) { - if (!cacheCtx.isLocal()) - cacheCtx.affinity().cleanUpCache(fut.topologyVersion()); - } - - cctx.affinity().cleanUpCache(fut.topologyVersion()); - } - } - if (skipped > 10) fut.cleanUp(); } http://git-wip-us.apache.org/repos/asf/ignite/blob/4d435537/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java new file mode 100644 index 0000000..87d1e5b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AffinityHistoryCleanupTest.java @@ -0,0 +1,414 @@ +/* + * 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.affinity; + +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicInteger; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteSystemProperties; +import org.apache.ignite.cache.affinity.fair.FairAffinityFunction; +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.IgniteKernal; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.internal.processors.affinity.GridAffinityAssignmentCache; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheProcessor; +import org.apache.ignite.internal.util.typedef.F; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class AffinityHistoryCleanupTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private boolean client; + + /** */ + private boolean lateAffAssignment; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder); + + CacheConfiguration[] ccfgs = new CacheConfiguration[4]; + + for (int i = 0; i < ccfgs.length; i++) { + CacheConfiguration ccfg = new CacheConfiguration(); + + ccfg.setName("static-cache-" + i); + ccfg.setAffinity(i % 2 == 0 ? new RendezvousAffinityFunction() : new FairAffinityFunction()); + + ccfgs[i] = ccfg; + } + + cfg.setCacheConfiguration(ccfgs); + + cfg.setClientMode(client); + + cfg.setLateAffinityAssignment(lateAffAssignment); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** + * @throws Exception If failed. + */ + public void testAffinityHistoryCleanup() throws Exception { + String histProp = System.getProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE); + + try { + System.setProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, "5"); + + Ignite ignite = startGrid(0); + + checkHistory(ignite, F.asList(topVer(1, 0)), 1); + + for (int i = 0; i < 3; i++) { + startGrid(1); + + stopGrid(1); + } + + checkHistory(ignite, F.asList( + topVer(3, 0), + topVer(4, 0), + topVer(5, 0), + topVer(6, 0), + topVer(7, 0)), + 5); + + client = true; + + startGrid(1); + + stopGrid(1); + + checkHistory(ignite, F.asList( + topVer(3, 0), + topVer(4, 0), + topVer(5, 0), + topVer(6, 0), + topVer(7, 0), + topVer(8, 0), + topVer(9, 0)), + 5); + + startGrid(1); + + stopGrid(1); + + checkHistory(ignite, F.asList( + topVer(3, 0), + topVer(4, 0), + topVer(5, 0), + topVer(6, 0), + topVer(7, 0), + topVer(8, 0), + topVer(9, 0), + topVer(10, 0), + topVer(11, 0)), + 5); + + startGrid(1); + + checkHistory(ignite, F.asList( + topVer(3, 0), + topVer(4, 0), + topVer(5, 0), + topVer(6, 0), + topVer(7, 0), + topVer(8, 0), + topVer(9, 0), + topVer(10, 0), + topVer(11, 0), + topVer(12, 0)), + 5); + + stopGrid(1); + + checkHistory(ignite, F.asList( + topVer(8, 0), + topVer(9, 0), + topVer(10, 0), + topVer(11, 0), + topVer(12, 0), + topVer(13, 0)), + 0); + + client = false; + + startGrid(1); + + stopGrid(1); + + checkHistory(ignite, F.asList( + topVer(8, 0), + topVer(9, 0), + topVer(10, 0), + topVer(11, 0), + topVer(12, 0), + topVer(13, 0), + topVer(14, 0), + topVer(15, 0)), + 2); + + startGrid(1); + + stopGrid(1); + + checkHistory(ignite, F.asList( + topVer(8, 0), + topVer(9, 0), + topVer(10, 0), + topVer(11, 0), + topVer(12, 0), + topVer(13, 0), + topVer(14, 0), + topVer(15, 0), + topVer(16, 0), + topVer(17, 0)), + 4); + + startGrid(1); + + checkHistory(ignite, F.asList( + topVer(13, 0), + topVer(14, 0), + topVer(15, 0), + topVer(16, 0), + topVer(17, 0), + topVer(18, 0)), + 5); + + stopGrid(1); + + checkHistory(ignite, F.asList( + topVer(14, 0), + topVer(15, 0), + topVer(16, 0), + topVer(17, 0), + topVer(18, 0), + topVer(19, 0)), + 6); + + startGrid(1); + + checkHistory(ignite, F.asList( + topVer(16, 0), + topVer(17, 0), + topVer(18, 0), + topVer(19, 0), + topVer(20, 0)), + 5); + + client = true; + + startGrid(2); + + stopGrid(2); + + checkHistory(ignite, F.asList( + topVer(16, 0), + topVer(17, 0), + topVer(18, 0), + topVer(19, 0), + topVer(20, 0), + topVer(21, 0), + topVer(22, 0)), + 5); + } + finally { + if (histProp != null) + System.setProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, histProp); + else + System.clearProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE); + } + } + + /** + * @throws Exception If failed. + */ + public void testAffinityHistoryCleanupLateAffinityAssignment() throws Exception { + lateAffAssignment = true; + + String histProp = System.getProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE); + + try { + System.setProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, "5"); + + Ignite ignite = startGrid(0); + + checkHistory(ignite, F.asList(topVer(1, 0)), 1); + + startGrid(1); + + checkHistory(ignite, F.asList( + topVer(1, 0), + topVer(2, 0), + topVer(2, 1)), + 3); + + startGrid(2); + + checkHistory(ignite, F.asList( + topVer(1, 0), + topVer(2, 0), + topVer(2, 1), + topVer(3, 0), + topVer(3, 1)), + 5); + + startGrid(3); + + checkHistory(ignite, F.asList( + topVer(2, 1), + topVer(3, 0), + topVer(3, 1), + topVer(4, 0), + topVer(4, 1)), + 5); + + client = true; + + startGrid(4); + + stopGrid(4); + + checkHistory(ignite, F.asList( + topVer(2, 1), + topVer(3, 0), + topVer(3, 1), + topVer(4, 0), + topVer(4, 1), + topVer(5, 0), + topVer(6, 0)), + 5); + + startGrid(4); + + stopGrid(4); + + checkHistory(ignite, F.asList( + topVer(2, 1), + topVer(3, 0), + topVer(3, 1), + topVer(4, 0), + topVer(4, 1), + topVer(5, 0), + topVer(6, 0), + topVer(7, 0), + topVer(8, 0)), + 5); + + startGrid(4); + + stopGrid(4); + + checkHistory(ignite, F.asList( + topVer(5, 0), + topVer(6, 0), + topVer(7, 0), + topVer(8, 0), + topVer(9, 0), + topVer(10, 0)), + 0); + + client = false; + + startGrid(4); + + checkHistory(ignite, F.asList( + topVer(5, 0), + topVer(6, 0), + topVer(7, 0), + topVer(8, 0), + topVer(9, 0), + topVer(10, 0), + topVer(11, 0), + topVer(11, 1)), + 2); + } + finally { + if (histProp != null) + System.setProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE, histProp); + else + System.clearProperty(IgniteSystemProperties.IGNITE_AFFINITY_HISTORY_SIZE); + } + } + + /** + * @param ignite Node. + * @param expHist Expected history. + * @param expSize Expected 'non client events' history size. + * @throws Exception If failed. + */ + private void checkHistory(Ignite ignite, List expHist, int expSize) throws Exception { + awaitPartitionMapExchange(); + + GridCacheProcessor proc = ((IgniteKernal)ignite).context().cache(); + + int cnt = 0; + + for (GridCacheContext cctx : proc.context().cacheContexts()) { + GridAffinityAssignmentCache aff = GridTestUtils.getFieldValue(cctx.affinity(), "aff"); + + AtomicInteger histSize = GridTestUtils.getFieldValue(aff, "histSize"); + + assertEquals(expSize, histSize.get()); + + Map cache = GridTestUtils.getFieldValue(aff, "affCache"); + + assertEquals("Unexpected history: " + cache.keySet(), expHist.size(), cache.size()); + + for (AffinityTopologyVersion topVer : expHist) + assertTrue("No history [ver=" + topVer + ", hist=" + cache.keySet() + ']', cache.containsKey(topVer)); + + cnt++; + } + + assert cnt > 4; + } + + /** + * @param major Major version. + * @param minor Minor version. + * @return Version. + */ + private static AffinityTopologyVersion topVer(int major, int minor) { + return new AffinityTopologyVersion(major, minor); + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/4d435537/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index 75674fa..0fdf817 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -23,6 +23,7 @@ import org.apache.ignite.GridCacheAffinityBackupsSelfTest; import org.apache.ignite.IgniteCacheAffinitySelfTest; import org.apache.ignite.cache.IgniteWarmupClosureSelfTest; import org.apache.ignite.cache.affinity.AffinityClientNodeSelfTest; +import org.apache.ignite.cache.affinity.AffinityHistoryCleanupTest; import org.apache.ignite.cache.affinity.fair.FairAffinityDynamicCacheSelfTest; import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionNodesSelfTest; import org.apache.ignite.cache.affinity.fair.FairAffinityFunctionSelfTest; @@ -210,6 +211,7 @@ public class IgniteCacheTestSuite extends TestSuite { suite.addTestSuite(GridCacheAffinityBackupsSelfTest.class); suite.addTestSuite(IgniteCacheAffinitySelfTest.class); suite.addTestSuite(AffinityClientNodeSelfTest.class); + suite.addTestSuite(AffinityHistoryCleanupTest.class); // Swap tests. suite.addTestSuite(GridCacheSwapPreloadSelfTest.class);