From cf5d3fbe6e574d9340c859b1098ce7a3b8ec1e78 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Wed, 14 Nov 2018 15:56:54 +0300 Subject: [PATCH 01/25] IGNITE-10058 Fix draft. --- .../cache/CacheAffinitySharedManager.java | 2 +- .../ignite/cache/ResetLostPartitionTest.java | 50 ++++++++++++++++++- 2 files changed, 50 insertions(+), 2 deletions(-) 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 07fbef1284c62..d10f764f815d6 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 @@ -286,7 +286,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { GridDhtPartitionState state = top.partitionState(waitNode, part); - if (state != GridDhtPartitionState.OWNING) { + if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.LOST) { rebalanced = false; break; diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index 405e32fd874bd..d13137c3b851f 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -18,6 +18,7 @@ package org.apache.ignite.cache; import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.stream.Collectors; import org.apache.ignite.Ignite; @@ -58,6 +59,9 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { /** Cache size */ public static final int CACHE_SIZE = 100000 / CACHE_NAMES.length; + /** */ + private boolean persistenceEnabled = true; + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { if (MvccFeatureChecker.forcedMvcc()) @@ -88,7 +92,7 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { DataStorageConfiguration storageCfg = new DataStorageConfiguration(); storageCfg.getDefaultDataRegionConfiguration() - .setPersistenceEnabled(true) + .setPersistenceEnabled(persistenceEnabled) .setMaxSize(500L * 1024 * 1024); cfg.setDataStorageConfiguration(storageCfg); @@ -238,6 +242,50 @@ private void doRebalanceAfterPartitionsWereLost(boolean reactivateGridBeforeRese assertEquals(CACHE_NAMES.length * CACHE_SIZE, averageSizeAroundAllNodes()); } + /** + * todo + * + * @throws Exception if fail. + */ + public void testDuplicateOwners() throws Exception { + persistenceEnabled = false; + + int gridCnt = 4; + + Ignite node = startGridsMultiThreaded(gridCnt); + + IgniteCache cache = node.createCache( + new CacheConfiguration(DEFAULT_CACHE_NAME) + .setBackups(0) + .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE)); + + for (int i = 0; i < CACHE_SIZE; i++) + cache.put(i, i); + + stopGrid(gridCnt - 1); + + startGrid(gridCnt - 1); + + // todo + U.sleep(5_000); + + grid(0).resetLostPartitions(Collections.singleton(DEFAULT_CACHE_NAME)); + + // todo + U.sleep(5_000); + + int parts = grid(0).cachex(DEFAULT_CACHE_NAME).configuration().getAffinity().partitions(); + + int owners = 0; + + for (int i = 0; i < gridCnt; i++) { + owners += getPartitionsStates(i, DEFAULT_CACHE_NAME).stream(). + filter(v -> v == GridDhtPartitionState.OWNING).collect(Collectors.toList()).size(); + } + + assertEquals(parts, owners); + } + /** * @param gridNumber Grid number. * @param cacheName Cache name. From 9af319db71232a698bf642d47a8173cda380a860 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 16 Nov 2018 12:50:05 +0300 Subject: [PATCH 02/25] IGNITE-10058 Evict lost parts on non-affinity nodes after first exchange. --- .../dht/topology/GridDhtPartitionTopologyImpl.java | 4 +++- .../java/org/apache/ignite/cache/ResetLostPartitionTest.java | 3 ++- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index c90d40f9ce15e..2393a01cb0354 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2337,7 +2337,9 @@ private boolean checkEvictions(long updateSeq, AffinityAssignment aff) { if (affNodes.contains(ctx.localNode())) continue; - List nodes = nodes(p, aff.topologyVersion(), OWNING); + GridDhtPartitionState[] lost = part.state() == LOST ? new GridDhtPartitionState[] {LOST} : null; + + List nodes = nodes(p, aff.topologyVersion(), OWNING, lost); Collection nodeIds = F.nodeIds(nodes); // If all affinity nodes are owners, then evict partition from local node. diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index d13137c3b851f..91b415b54d3c2 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -20,6 +20,7 @@ import java.util.Arrays; import java.util.Collections; import java.util.List; +import java.util.function.Predicate; import java.util.stream.Collectors; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; @@ -280,7 +281,7 @@ public void testDuplicateOwners() throws Exception { for (int i = 0; i < gridCnt; i++) { owners += getPartitionsStates(i, DEFAULT_CACHE_NAME).stream(). - filter(v -> v == GridDhtPartitionState.OWNING).collect(Collectors.toList()).size(); + filter(Predicate.isEqual(GridDhtPartitionState.OWNING)).collect(Collectors.toList()).size(); } assertEquals(parts, owners); From a2e8236196d0a5bc270cea344c1e7051091b07df Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 16 Nov 2018 20:28:38 +0300 Subject: [PATCH 03/25] IGNITE-10058 Rework test timeouts (wip). --- .../ignite/cache/ResetLostPartitionTest.java | 122 +++++++++++++----- 1 file changed, 91 insertions(+), 31 deletions(-) diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index 91b415b54d3c2..267679bd7a8b0 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -18,21 +18,27 @@ package org.apache.ignite.cache; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.function.Predicate; -import java.util.stream.Collectors; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.IntStream; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgnitionEx; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.GridCacheContext; +import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl; +import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -44,9 +50,14 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; +import static java.util.function.Predicate.isEqual; +import static java.util.stream.Collectors.toList; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PRELOAD_RESEND_TIMEOUT; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.LOST; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; +import static org.apache.ignite.testframework.GridTestUtils.getFieldValue; +import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** * @@ -63,6 +74,13 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { /** */ private boolean persistenceEnabled = true; + /** */ + CacheConfiguration[] ccfg = new CacheConfiguration[] { + cacheConfiguration(CACHE_NAMES[0], CacheAtomicityMode.ATOMIC), + cacheConfiguration(CACHE_NAMES[1], CacheAtomicityMode.ATOMIC), + cacheConfiguration(CACHE_NAMES[2], CacheAtomicityMode.TRANSACTIONAL) + }; + /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { if (MvccFeatureChecker.forcedMvcc()) @@ -98,12 +116,6 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { cfg.setDataStorageConfiguration(storageCfg); - CacheConfiguration[] ccfg = new CacheConfiguration[] { - cacheConfiguration(CACHE_NAMES[0], CacheAtomicityMode.ATOMIC), - cacheConfiguration(CACHE_NAMES[1], CacheAtomicityMode.ATOMIC), - cacheConfiguration(CACHE_NAMES[2], CacheAtomicityMode.TRANSACTIONAL) - }; - cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); cfg.setCacheConfiguration(ccfg); @@ -251,40 +263,82 @@ private void doRebalanceAfterPartitionsWereLost(boolean reactivateGridBeforeRese public void testDuplicateOwners() throws Exception { persistenceEnabled = false; - int gridCnt = 4; + ccfg = null; - Ignite node = startGridsMultiThreaded(gridCnt); + System.setProperty(IGNITE_PRELOAD_RESEND_TIMEOUT, "0"); - IgniteCache cache = node.createCache( - new CacheConfiguration(DEFAULT_CACHE_NAME) - .setBackups(0) - .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE)); + try { + int gridCnt = 4; - for (int i = 0; i < CACHE_SIZE; i++) - cache.put(i, i); + long timeout = 3_000; - stopGrid(gridCnt - 1); + Ignite node = startGridsMultiThreaded(gridCnt); - startGrid(gridCnt - 1); + IgniteCache cache = node.createCache( + new CacheConfiguration(DEFAULT_CACHE_NAME) + .setBackups(0) + .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE)); - // todo - U.sleep(5_000); + for (int i = 0; i < CACHE_SIZE; i++) + cache.put(i, i); - grid(0).resetLostPartitions(Collections.singleton(DEFAULT_CACHE_NAME)); + int failedNodeIdx = gridCnt - 1; - // todo - U.sleep(5_000); + int lostPartsCnt = count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx); - int parts = grid(0).cachex(DEFAULT_CACHE_NAME).configuration().getAffinity().partitions(); + grid(0).affinity(DEFAULT_CACHE_NAME).partitions(); - int owners = 0; + stopGrid(failedNodeIdx); - for (int i = 0; i < gridCnt; i++) { - owners += getPartitionsStates(i, DEFAULT_CACHE_NAME).stream(). - filter(Predicate.isEqual(GridDhtPartitionState.OWNING)).collect(Collectors.toList()).size(); - } + boolean allLostPartsDetected = waitForCondition(() -> { + for (Ignite grid : G.allGrids()) { + if (grid.cache(DEFAULT_CACHE_NAME).lostPartitions().size() != lostPartsCnt) + return false; + } + + return true; + }, timeout); + + assertTrue(allLostPartsDetected); + + startGrid(failedNodeIdx); + + boolean victimLostPartsDetected = + waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx), timeout); + + assertTrue(victimLostPartsDetected); + + // Wait for rebalance. + for (Ignite grid : G.allGrids()) { + GridCacheSharedContext cctx = ((IgniteEx)grid).context().cache().context(); + + cctx.exchange().affinityReadyFuture(new AffinityTopologyVersion(6, 0)).get(timeout); + + for (GridCacheContext ctx : (Collection)cctx.cacheContexts()) + ctx.preloader().rebalanceFuture().get(timeout); + + AtomicReference ref = getFieldValue(cctx.exchange(), "pendingResend"); - assertEquals(parts, owners); + assert waitForCondition(() -> ref == null || ref.get() == null, 5_000); + } + + grid(0).resetLostPartitions(Collections.singleton(DEFAULT_CACHE_NAME)); + + boolean victimPartsDetected = + waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx), timeout); + + assertTrue(victimPartsDetected); + + int parts = grid(0).cachex(DEFAULT_CACHE_NAME).configuration().getAffinity().partitions(); + + int[] idxs = IntStream.range(0, gridCnt).toArray(); + + waitForCondition(() -> count(DEFAULT_CACHE_NAME, OWNING, idxs) == parts, timeout); + + assertEquals(parts, count(DEFAULT_CACHE_NAME, OWNING, idxs)); + } finally { + System.clearProperty(IGNITE_PRELOAD_RESEND_TIMEOUT); + } } /** @@ -299,7 +353,13 @@ private List getPartitionsStates(int gridNumber, String c return top.localPartitions().stream() .map(GridDhtLocalPartition::state) - .collect(Collectors.toList()); + .collect(toList()); + } + + /** */ + private int count(String cacheName, GridDhtPartitionState state, int ... gridIdx) { + return Arrays.stream(gridIdx).map(idx -> + getPartitionsStates(idx, cacheName).stream().filter(isEqual(state)).collect(toList()).size()).sum(); } /** From 5e2349c0342ce6e4d0e4d3fc8a52f28da88054db Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Mon, 19 Nov 2018 22:05:55 +0300 Subject: [PATCH 04/25] IGNITE-10058 Improved test and check on rebalance. --- .../cache/CacheAffinitySharedManager.java | 32 +++++- .../ignite/cache/ResetLostPartitionTest.java | 108 +++++++----------- 2 files changed, 70 insertions(+), 70 deletions(-) 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 d10f764f815d6..56e61c05943ed 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 @@ -286,7 +286,8 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { GridDhtPartitionState state = top.partitionState(waitNode, part); - if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.LOST) { + if (state != GridDhtPartitionState.OWNING && + !lostPartitionMoving(top, grpHolder, waitNode, part, state)) { rebalanced = false; break; @@ -317,6 +318,35 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { } } + /** + * @param top Topology version. + * @param grpHolder Cache group holder. + * @param waitNode Node rebalancing data. + * @param p Parition id. + * @param state Partition state. + * @return {@code True} If this partition is moving in LOST state. + */ + private boolean lostPartitionMoving( + GridDhtPartitionTopology top, + CacheGroupHolder grpHolder, + UUID waitNode, + Integer p, + GridDhtPartitionState state + ) { + if (state != GridDhtPartitionState.LOST) + return false; + + List assignment = grpHolder.affinity().assignments(top.readyTopologyVersion()).get(p); + + assert !assignment.isEmpty(); + + UUID curPrimary = assignment.get(0).id(); + + assert !waitNode.equals(curPrimary) : curPrimary; + + return top.partitionState(curPrimary, p) == GridDhtPartitionState.LOST; + } + /** * @return Group IDs. */ diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index 267679bd7a8b0..f04b96dc28c1c 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -21,8 +21,6 @@ import java.util.Collection; import java.util.Collections; import java.util.List; -import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.IntStream; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; @@ -31,7 +29,6 @@ import org.apache.ignite.configuration.IgniteConfiguration; import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgnitionEx; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; @@ -52,11 +49,9 @@ import static java.util.function.Predicate.isEqual; import static java.util.stream.Collectors.toList; -import static org.apache.ignite.IgniteSystemProperties.IGNITE_PRELOAD_RESEND_TIMEOUT; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.LOST; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; -import static org.apache.ignite.testframework.GridTestUtils.getFieldValue; import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** @@ -74,13 +69,6 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { /** */ private boolean persistenceEnabled = true; - /** */ - CacheConfiguration[] ccfg = new CacheConfiguration[] { - cacheConfiguration(CACHE_NAMES[0], CacheAtomicityMode.ATOMIC), - cacheConfiguration(CACHE_NAMES[1], CacheAtomicityMode.ATOMIC), - cacheConfiguration(CACHE_NAMES[2], CacheAtomicityMode.TRANSACTIONAL) - }; - /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { if (MvccFeatureChecker.forcedMvcc()) @@ -118,6 +106,12 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + CacheConfiguration[] ccfg = new CacheConfiguration[] { + cacheConfiguration(CACHE_NAMES[0], CacheAtomicityMode.ATOMIC), + cacheConfiguration(CACHE_NAMES[1], CacheAtomicityMode.ATOMIC), + cacheConfiguration(CACHE_NAMES[2], CacheAtomicityMode.TRANSACTIONAL) + }; + cfg.setCacheConfiguration(ccfg); return cfg; @@ -256,89 +250,65 @@ private void doRebalanceAfterPartitionsWereLost(boolean reactivateGridBeforeRese } /** - * todo + * Check that there is no duplicate partition owners after reset lost partitions. * * @throws Exception if fail. */ public void testDuplicateOwners() throws Exception { persistenceEnabled = false; - ccfg = null; - - System.setProperty(IGNITE_PRELOAD_RESEND_TIMEOUT, "0"); - - try { - int gridCnt = 4; - - long timeout = 3_000; - - Ignite node = startGridsMultiThreaded(gridCnt); + int gridCnt = 4; - IgniteCache cache = node.createCache( - new CacheConfiguration(DEFAULT_CACHE_NAME) - .setBackups(0) - .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE)); + long timeout = 5_000; - for (int i = 0; i < CACHE_SIZE; i++) - cache.put(i, i); + Ignite node = startGridsMultiThreaded(gridCnt); - int failedNodeIdx = gridCnt - 1; + IgniteCache cache = node.createCache( + new CacheConfiguration(DEFAULT_CACHE_NAME) + .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE)); - int lostPartsCnt = count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx); + for (int i = 0; i < CACHE_SIZE; i++) + cache.put(i, i); - grid(0).affinity(DEFAULT_CACHE_NAME).partitions(); + int failedNodeIdx = gridCnt - 1; - stopGrid(failedNodeIdx); + int lostPartsCnt = count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx); - boolean allLostPartsDetected = waitForCondition(() -> { - for (Ignite grid : G.allGrids()) { - if (grid.cache(DEFAULT_CACHE_NAME).lostPartitions().size() != lostPartsCnt) - return false; - } + stopGrid(failedNodeIdx); - return true; - }, timeout); + int[] liveIdxs = new int[] {0, 1, 2}; - assertTrue(allLostPartsDetected); + waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, liveIdxs), timeout); + assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, LOST, liveIdxs)); - startGrid(failedNodeIdx); + startGrid(failedNodeIdx); - boolean victimLostPartsDetected = - waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx), timeout); + waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx), timeout); + assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx)); - assertTrue(victimLostPartsDetected); + waitForCondition(() -> 0 == count(DEFAULT_CACHE_NAME, LOST, liveIdxs), timeout); + assertEquals(0, count(DEFAULT_CACHE_NAME, LOST, liveIdxs)); - // Wait for rebalance. - for (Ignite grid : G.allGrids()) { - GridCacheSharedContext cctx = ((IgniteEx)grid).context().cache().context(); + for (Ignite grid : G.allGrids()) { + GridCacheSharedContext cctx = ((IgniteEx)grid).context().cache().context(); - cctx.exchange().affinityReadyFuture(new AffinityTopologyVersion(6, 0)).get(timeout); + cctx.exchange().affinityReadyFuture(cctx.discovery().topologyVersionEx()).get(timeout); - for (GridCacheContext ctx : (Collection)cctx.cacheContexts()) - ctx.preloader().rebalanceFuture().get(timeout); - - AtomicReference ref = getFieldValue(cctx.exchange(), "pendingResend"); - - assert waitForCondition(() -> ref == null || ref.get() == null, 5_000); - } - - grid(0).resetLostPartitions(Collections.singleton(DEFAULT_CACHE_NAME)); - - boolean victimPartsDetected = - waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx), timeout); + for (GridCacheContext ctx : (Collection)cctx.cacheContexts()) + ctx.preloader().rebalanceFuture().get(timeout); + } - assertTrue(victimPartsDetected); + node.resetLostPartitions(Collections.singleton(DEFAULT_CACHE_NAME)); - int parts = grid(0).cachex(DEFAULT_CACHE_NAME).configuration().getAffinity().partitions(); + waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx), timeout); + assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx)); - int[] idxs = IntStream.range(0, gridCnt).toArray(); + int parts = grid(0).affinity(DEFAULT_CACHE_NAME).partitions(); - waitForCondition(() -> count(DEFAULT_CACHE_NAME, OWNING, idxs) == parts, timeout); + int[] allIdxs = new int[] {0, 1, 2, 3}; - assertEquals(parts, count(DEFAULT_CACHE_NAME, OWNING, idxs)); - } finally { - System.clearProperty(IGNITE_PRELOAD_RESEND_TIMEOUT); - } + waitForCondition(() -> parts == count(DEFAULT_CACHE_NAME, OWNING, allIdxs), timeout); + assertEquals(parts, count(DEFAULT_CACHE_NAME, OWNING, allIdxs)); } /** From b3a264cc148510c64854e37ca87ceb014849c91f Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Mon, 19 Nov 2018 22:47:31 +0300 Subject: [PATCH 05/25] IGNITE-10058 Minor. --- .../processors/cache/CacheAffinitySharedManager.java | 8 ++++---- .../dht/topology/GridDhtPartitionTopologyImpl.java | 4 +++- 2 files changed, 7 insertions(+), 5 deletions(-) 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 56e61c05943ed..c0afe47854a7d 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 @@ -324,7 +324,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { * @param waitNode Node rebalancing data. * @param p Parition id. * @param state Partition state. - * @return {@code True} If this partition is moving in LOST state. + * @return {@code True} if this partition is moving in LOST state. */ private boolean lostPartitionMoving( GridDhtPartitionTopology top, @@ -340,11 +340,11 @@ private boolean lostPartitionMoving( assert !assignment.isEmpty(); - UUID curPrimary = assignment.get(0).id(); + UUID oldPrimary = assignment.get(0).id(); - assert !waitNode.equals(curPrimary) : curPrimary; + assert !waitNode.equals(oldPrimary) : oldPrimary; - return top.partitionState(curPrimary, p) == GridDhtPartitionState.LOST; + return top.partitionState(oldPrimary, p) == GridDhtPartitionState.LOST; } /** diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 2393a01cb0354..d84d16baf8a8c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2092,7 +2092,9 @@ else if (plc != PartitionLossPolicy.IGNORE) { if (e.getKey().equals(ctx.localNodeId())) continue; - if (e.getValue().get(part) != EVICTED) + GridDhtPartitionState state = e.getValue().get(part); + + if (state != null && state != EVICTED) e.getValue().put(part, LOST); } } From 9e1f3e04e4f45a06e1e73dbcdd87a6a11cbfdf55 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 20 Nov 2018 00:18:25 +0300 Subject: [PATCH 06/25] IGNITE-10058 Minor codestyle. --- .../dht/topology/GridDhtPartitionTopologyImpl.java | 5 +++-- .../apache/ignite/cache/ResetLostPartitionTest.java | 11 +++++++++-- 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index d84d16baf8a8c..dd160d3effed5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2339,9 +2339,10 @@ private boolean checkEvictions(long updateSeq, AffinityAssignment aff) { if (affNodes.contains(ctx.localNode())) continue; - GridDhtPartitionState[] lost = part.state() == LOST ? new GridDhtPartitionState[] {LOST} : null; + AffinityTopologyVersion topVer = aff.topologyVersion(); + + List nodes = part.state() == LOST ? nodes(p, topVer, OWNING, LOST) : nodes(p, topVer, OWNING); - List nodes = nodes(p, aff.topologyVersion(), OWNING, lost); Collection nodeIds = F.nodeIds(nodes); // If all affinity nodes are owners, then evict partition from local node. diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index f04b96dc28c1c..43d9c861a6751 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -66,7 +66,7 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { /** Cache size */ public static final int CACHE_SIZE = 100000 / CACHE_NAMES.length; - /** */ + /** Persistence enabled flag. */ private boolean persistenceEnabled = true; /** {@inheritDoc} */ @@ -326,7 +326,14 @@ private List getPartitionsStates(int gridNumber, String c .collect(toList()); } - /** */ + /** + * Counts partitions in the specified state on the specified nodes. + * + * @param cacheName Cache name. + * @param state Partition state. + * @param gridIdx Grid index. + * @return Number of local partitions in the specified state. + */ private int count(String cacheName, GridDhtPartitionState state, int ... gridIdx) { return Arrays.stream(gridIdx).map(idx -> getPartitionsStates(idx, cacheName).stream().filter(isEqual(state)).collect(toList()).size()).sum(); From f4484fe429070333684db002620195ce9b3da02c Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 20 Nov 2018 11:44:33 +0300 Subject: [PATCH 07/25] IGNITE-10058 Skip forceAffReassignment. --- .../processors/cache/CacheAffinitySharedManager.java | 8 ++++---- .../dht/topology/GridDhtPartitionTopologyImpl.java | 4 +--- .../org/apache/ignite/cache/ResetLostPartitionTest.java | 4 ++-- 3 files changed, 7 insertions(+), 9 deletions(-) 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 c0afe47854a7d..91a5af945c913 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 @@ -287,7 +287,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { GridDhtPartitionState state = top.partitionState(waitNode, part); if (state != GridDhtPartitionState.OWNING && - !lostPartitionMoving(top, grpHolder, waitNode, part, state)) { + !lostPartitionMoving(top, grpHolder.affinity(), waitNode, part, state)) { rebalanced = false; break; @@ -320,7 +320,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { /** * @param top Topology version. - * @param grpHolder Cache group holder. + * @param aff Affinity assignment cache. * @param waitNode Node rebalancing data. * @param p Parition id. * @param state Partition state. @@ -328,7 +328,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { */ private boolean lostPartitionMoving( GridDhtPartitionTopology top, - CacheGroupHolder grpHolder, + GridAffinityAssignmentCache aff, UUID waitNode, Integer p, GridDhtPartitionState state @@ -336,7 +336,7 @@ private boolean lostPartitionMoving( if (state != GridDhtPartitionState.LOST) return false; - List assignment = grpHolder.affinity().assignments(top.readyTopologyVersion()).get(p); + List assignment = aff.assignments(aff.lastVersion()).get(p); assert !assignment.isEmpty(); diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index dd160d3effed5..18fda3c0b00c7 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2092,9 +2092,7 @@ else if (plc != PartitionLossPolicy.IGNORE) { if (e.getKey().equals(ctx.localNodeId())) continue; - GridDhtPartitionState state = e.getValue().get(part); - - if (state != null && state != EVICTED) + if (e.getValue().get(part) != EVICTED) e.getValue().put(part, LOST); } } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index 43d9c861a6751..3cc6c2caa3a91 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -104,14 +104,14 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { cfg.setDataStorageConfiguration(storageCfg); - cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); - CacheConfiguration[] ccfg = new CacheConfiguration[] { cacheConfiguration(CACHE_NAMES[0], CacheAtomicityMode.ATOMIC), cacheConfiguration(CACHE_NAMES[1], CacheAtomicityMode.ATOMIC), cacheConfiguration(CACHE_NAMES[2], CacheAtomicityMode.TRANSACTIONAL) }; + cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); + cfg.setCacheConfiguration(ccfg); return cfg; From 872361ae9facc14d5412d5a4ff8ee444b09959a2 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 29 Nov 2018 14:54:18 +0300 Subject: [PATCH 08/25] IGNITE-10058 Review notes: removed redundant checking. --- .../cache/CacheAffinitySharedManager.java | 32 +------------------ 1 file changed, 1 insertion(+), 31 deletions(-) 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 91a5af945c913..d10f764f815d6 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 @@ -286,8 +286,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { GridDhtPartitionState state = top.partitionState(waitNode, part); - if (state != GridDhtPartitionState.OWNING && - !lostPartitionMoving(top, grpHolder.affinity(), waitNode, part, state)) { + if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.LOST) { rebalanced = false; break; @@ -318,35 +317,6 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { } } - /** - * @param top Topology version. - * @param aff Affinity assignment cache. - * @param waitNode Node rebalancing data. - * @param p Parition id. - * @param state Partition state. - * @return {@code True} if this partition is moving in LOST state. - */ - private boolean lostPartitionMoving( - GridDhtPartitionTopology top, - GridAffinityAssignmentCache aff, - UUID waitNode, - Integer p, - GridDhtPartitionState state - ) { - if (state != GridDhtPartitionState.LOST) - return false; - - List assignment = aff.assignments(aff.lastVersion()).get(p); - - assert !assignment.isEmpty(); - - UUID oldPrimary = assignment.get(0).id(); - - assert !waitNode.equals(oldPrimary) : oldPrimary; - - return top.partitionState(oldPrimary, p) == GridDhtPartitionState.LOST; - } - /** * @return Group IDs. */ From 945c3d82f8c6b43a7afcc054ff4996cd7e8cd76c Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 4 Dec 2018 13:11:19 +0300 Subject: [PATCH 09/25] IGNITE-10058 TC check. --- .../cache/CacheAffinitySharedManager.java | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 deletions(-) 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 d10f764f815d6..326e8275d680c 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 @@ -286,7 +286,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { GridDhtPartitionState state = top.partitionState(waitNode, part); - if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.LOST) { + if (state != GridDhtPartitionState.OWNING) { rebalanced = false; break; @@ -2136,7 +2136,8 @@ private CacheGroupHolder groupHolder(AffinityTopologyVersion topVer, final Cache grp.affinity(), null, grp.rebalanceEnabled(), - affCache); + affCache, + grp.topology()); cctx.exchange().exchangerUpdateHeartbeat(); } @@ -2160,7 +2161,8 @@ private CacheGroupHolder groupHolder(AffinityTopologyVersion topVer, final Cache cache.affinity(), waitRebalanceInfo, latePrimary, - affCache); + affCache, + cache.topology(evts.discoveryCache())); if (grpAdded) { AffinityAssignment aff = cache.aff.cachedAffinity(cache.aff.lastVersion()); @@ -2227,7 +2229,8 @@ private void initAffinityOnNodeJoin( GridAffinityAssignmentCache aff, WaitRebalanceInfo rebalanceInfo, boolean latePrimary, - Map>> affCache + Map>> affCache, + GridDhtPartitionTopology top ) { if (addedOnExchnage) { if (!aff.lastVersion().equals(evts.topologyVersion())) @@ -2259,7 +2262,9 @@ private void initAffinityOnNodeJoin( if (curPrimary != null && newPrimary != null && !curPrimary.equals(newPrimary)) { assert cctx.discovery().node(evts.topologyVersion(), curPrimary.id()) != null : curPrimary; - List nodes0 = latePrimaryAssignment(aff, + boolean lost = GridDhtPartitionState.LOST == top.partitionState(curPrimary.id(), p); + + List nodes0 = lost ? newNodes : latePrimaryAssignment(aff, p, curPrimary, newNodes, From 5ba78294128b17df00dcf88cd372bb408a5ff2a3 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Tue, 4 Dec 2018 19:06:09 +0300 Subject: [PATCH 10/25] Revert "IGNITE-10058 TC check." This reverts commit 6ddd8c9 --- .../cache/CacheAffinitySharedManager.java | 15 +++++---------- 1 file changed, 5 insertions(+), 10 deletions(-) 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 326e8275d680c..d10f764f815d6 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 @@ -286,7 +286,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { GridDhtPartitionState state = top.partitionState(waitNode, part); - if (state != GridDhtPartitionState.OWNING) { + if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.LOST) { rebalanced = false; break; @@ -2136,8 +2136,7 @@ private CacheGroupHolder groupHolder(AffinityTopologyVersion topVer, final Cache grp.affinity(), null, grp.rebalanceEnabled(), - affCache, - grp.topology()); + affCache); cctx.exchange().exchangerUpdateHeartbeat(); } @@ -2161,8 +2160,7 @@ private CacheGroupHolder groupHolder(AffinityTopologyVersion topVer, final Cache cache.affinity(), waitRebalanceInfo, latePrimary, - affCache, - cache.topology(evts.discoveryCache())); + affCache); if (grpAdded) { AffinityAssignment aff = cache.aff.cachedAffinity(cache.aff.lastVersion()); @@ -2229,8 +2227,7 @@ private void initAffinityOnNodeJoin( GridAffinityAssignmentCache aff, WaitRebalanceInfo rebalanceInfo, boolean latePrimary, - Map>> affCache, - GridDhtPartitionTopology top + Map>> affCache ) { if (addedOnExchnage) { if (!aff.lastVersion().equals(evts.topologyVersion())) @@ -2262,9 +2259,7 @@ private void initAffinityOnNodeJoin( if (curPrimary != null && newPrimary != null && !curPrimary.equals(newPrimary)) { assert cctx.discovery().node(evts.topologyVersion(), curPrimary.id()) != null : curPrimary; - boolean lost = GridDhtPartitionState.LOST == top.partitionState(curPrimary.id(), p); - - List nodes0 = lost ? newNodes : latePrimaryAssignment(aff, + List nodes0 = latePrimaryAssignment(aff, p, curPrimary, newNodes, From e098747ab8ff144cbefcc2f788437c8d9c4050f5 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 6 Dec 2018 20:07:47 +0300 Subject: [PATCH 11/25] IGNITE-10058 Revert rebalance changes. --- .../cache/CacheAffinitySharedManager.java | 2 +- .../GridDhtPartitionsExchangeFuture.java | 2 + .../GridDhtPartitionTopologyImpl.java | 4 +- .../ignite/cache/ResetLostPartitionTest.java | 47 +++++++++++++------ 4 files changed, 37 insertions(+), 18 deletions(-) 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 d10f764f815d6..07fbef1284c62 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 @@ -286,7 +286,7 @@ void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { GridDhtPartitionState state = top.partitionState(waitNode, part); - if (state != GridDhtPartitionState.OWNING && state != GridDhtPartitionState.LOST) { + if (state != GridDhtPartitionState.OWNING) { rebalanced = false; break; 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 89e03a2c67ac4..2359e901b0197 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 @@ -3011,6 +3011,8 @@ private void detectLostPartitions(AffinityTopologyVersion resTopVer) { private void resetLostPartitions(Collection cacheNames) { assert !exchCtx.mergeExchanges(); + U.dumpStack("reset lost partitions"); + synchronized (cctx.exchange().interruptLock()) { if (Thread.currentThread().isInterrupted()) return; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 18fda3c0b00c7..9f323175d164b 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2337,9 +2337,7 @@ private boolean checkEvictions(long updateSeq, AffinityAssignment aff) { if (affNodes.contains(ctx.localNode())) continue; - AffinityTopologyVersion topVer = aff.topologyVersion(); - - List nodes = part.state() == LOST ? nodes(p, topVer, OWNING, LOST) : nodes(p, topVer, OWNING); + List nodes = nodes(p, aff.topologyVersion(), OWNING); Collection nodeIds = F.nodeIds(nodes); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index 3cc6c2caa3a91..f0b3cc46067c2 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -32,6 +32,7 @@ import org.apache.ignite.internal.processors.cache.CacheGroupContext; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; +import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl; @@ -265,6 +266,7 @@ public void testDuplicateOwners() throws Exception { IgniteCache cache = node.createCache( new CacheConfiguration(DEFAULT_CACHE_NAME) + .setAffinity(new RendezvousAffinityFunction(false, 32)) .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE)); for (int i = 0; i < CACHE_SIZE; i++) @@ -274,31 +276,44 @@ public void testDuplicateOwners() throws Exception { int lostPartsCnt = count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx); + log.info(">xxx> stop " + failedNodeIdx); + stopGrid(failedNodeIdx); int[] liveIdxs = new int[] {0, 1, 2}; - waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, liveIdxs), timeout); - assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, LOST, liveIdxs)); +// waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, liveIdxs), timeout); +// assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, LOST, liveIdxs)); - startGrid(failedNodeIdx); + U.sleep(5_000); - waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx), timeout); - assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx)); + log.info(">xxx> start " + failedNodeIdx); - waitForCondition(() -> 0 == count(DEFAULT_CACHE_NAME, LOST, liveIdxs), timeout); - assertEquals(0, count(DEFAULT_CACHE_NAME, LOST, liveIdxs)); + startGrid(failedNodeIdx); - for (Ignite grid : G.allGrids()) { - GridCacheSharedContext cctx = ((IgniteEx)grid).context().cache().context(); + U.sleep(5_000); +// waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx), timeout); +// assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx)); +// +// waitForCondition(() -> 0 == count(DEFAULT_CACHE_NAME, LOST, liveIdxs), timeout); +// assertEquals(0, count(DEFAULT_CACHE_NAME, LOST, liveIdxs)); +// +// for (Ignite grid : G.allGrids()) { +// GridCacheSharedContext cctx = ((IgniteEx)grid).context().cache().context(); +// +// cctx.exchange().affinityReadyFuture(cctx.discovery().topologyVersionEx()).get(timeout); +// +// for (GridCacheContext ctx : (Collection)cctx.cacheContexts()) +// ctx.preloader().rebalanceFuture().get(timeout); +// } + + log.info(">xxx> reset lost partitions"); - cctx.exchange().affinityReadyFuture(cctx.discovery().topologyVersionEx()).get(timeout); + node.resetLostPartitions(Collections.singleton(DEFAULT_CACHE_NAME)); - for (GridCacheContext ctx : (Collection)cctx.cacheContexts()) - ctx.preloader().rebalanceFuture().get(timeout); - } + U.sleep(5_000); - node.resetLostPartitions(Collections.singleton(DEFAULT_CACHE_NAME)); + log.info(">xxx> check duplicate owners"); waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx), timeout); assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx)); @@ -309,6 +324,10 @@ public void testDuplicateOwners() throws Exception { waitForCondition(() -> parts == count(DEFAULT_CACHE_NAME, OWNING, allIdxs), timeout); assertEquals(parts, count(DEFAULT_CACHE_NAME, OWNING, allIdxs)); + + for (int idx : allIdxs) + assertEquals("" + idx, 0, count(DEFAULT_CACHE_NAME, LOST, idx)); + //assert grid(idx).cache(DEFAULT_CACHE_NAME).lostPartitions().isEmpty(); } /** From 62a7a8d05a5391c658d00e6e4f96ee954e04f600 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Wed, 12 Dec 2018 10:06:41 +0300 Subject: [PATCH 12/25] IGNITE-10058 Fix reset lost parts (in-mem, empty counters). --- .../GridDhtPartitionsExchangeFuture.java | 5 ++++- .../GridDhtPartitionTopologyImpl.java | 20 ++++++++++++++++--- .../ignite/cache/ResetLostPartitionTest.java | 2 +- 3 files changed, 22 insertions(+), 5 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 2359e901b0197..37d2362ebe366 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 @@ -3249,8 +3249,11 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe Set caches = exchActions.cachesToResetLostPartitions(); - if (!F.isEmpty(caches)) + if (!F.isEmpty(caches)) { resetLostPartitions(caches); + + cctx.exchange().refreshPartitions(); + } } } else if (discoveryCustomMessage instanceof SnapshotDiscoveryMessage diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 9f323175d164b..b165f2d663b9e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2130,13 +2130,27 @@ else if (plc != PartitionLossPolicy.IGNORE) { long updSeq = updateSeq.incrementAndGet(); for (Map.Entry e : node2part.entrySet()) { - for (Map.Entry e0 : e.getValue().entrySet()) { - if (e0.getValue() != LOST) + GridDhtPartitionMap partMap = e.getValue(); + + for (Map.Entry e0 : partMap.entrySet()) { + GridDhtPartitionState state = e0.getValue(); + + int part = e0.getKey(); + + if (state != LOST) + continue; + + AffinityAssignment assignment = grp.affinity().cachedAffinity(resTopVer); + + if (!assignment.idealAssignment().get(part).contains(ctx.discovery().node(e.getKey()))) continue; e0.setValue(OWNING); - GridDhtLocalPartition locPart = localPartition(e0.getKey(), resTopVer, false); + if (!ctx.localNodeId().equals(e.getKey())) + continue;; + + GridDhtLocalPartition locPart = localPartition(part, resTopVer, false); if (locPart != null && locPart.state() == LOST) { boolean marked = locPart.own(); diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index f0b3cc46067c2..feb4e3daeb5d3 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -113,7 +113,7 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); - cfg.setCacheConfiguration(ccfg); +// cfg.setCacheConfiguration(ccfg); return cfg; } From 8695bd53d7b9229f8c6f58fc49fbcaaafa7c4a05 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Wed, 12 Dec 2018 11:41:08 +0300 Subject: [PATCH 13/25] IGNITE-10058 Fix reset update counters (wip). --- .../GridDhtPartitionsExchangeFuture.java | 2 -- .../GridDhtPartitionTopologyImpl.java | 31 +++++++++++++------ 2 files changed, 22 insertions(+), 11 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 37d2362ebe366..f79f8d51006b0 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 @@ -3011,8 +3011,6 @@ private void detectLostPartitions(AffinityTopologyVersion resTopVer) { private void resetLostPartitions(Collection cacheNames) { assert !exchCtx.mergeExchanges(); - U.dumpStack("reset lost partitions"); - synchronized (cctx.exchange().interruptLock()) { if (Thread.currentThread().isInterrupted()) return; diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index b165f2d663b9e..f846046a77527 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2129,6 +2129,8 @@ else if (plc != PartitionLossPolicy.IGNORE) { try { long updSeq = updateSeq.incrementAndGet(); + Set hasOwners = new HashSet<>(); + for (Map.Entry e : node2part.entrySet()) { GridDhtPartitionMap partMap = e.getValue(); @@ -2137,6 +2139,9 @@ else if (plc != PartitionLossPolicy.IGNORE) { int part = e0.getKey(); + if (state == OWNING) + hasOwners.add(part); + if (state != LOST) continue; @@ -2146,23 +2151,32 @@ else if (plc != PartitionLossPolicy.IGNORE) { continue; e0.setValue(OWNING); + } + } - if (!ctx.localNodeId().equals(e.getKey())) - continue;; + // Reset local partitions. + GridDhtPartitionMap locPartMap = node2part.get(ctx.localNodeId()); - GridDhtLocalPartition locPart = localPartition(part, resTopVer, false); + if (locPartMap != null) { + for (Map.Entry e : locPartMap.entrySet()){ + GridDhtLocalPartition locPart = localPartition(e.getKey(), resTopVer, false); if (locPart != null && locPart.state() == LOST) { + long updateCntr = locPart.updateCounter(); + + if (updateCntr == 0 && e.getValue() != OWNING) + continue; + boolean marked = locPart.own(); if (marked) { updateLocal(locPart.id(), locPart.state(), updSeq, resTopVer); - long updateCntr = locPart.updateCounter(); - - //Set update counters to 0, for full rebalance. - locPart.updateCounter(updateCntr, -updateCntr); - locPart.initialUpdateCounter(0); + if (!hasOwners.contains(e.getKey())) { + //Set update counters to 0, for full rebalance. + locPart.updateCounter(updateCntr, -updateCntr); + locPart.initialUpdateCounter(0); + } } } } @@ -2352,7 +2366,6 @@ private boolean checkEvictions(long updateSeq, AffinityAssignment aff) { continue; List nodes = nodes(p, aff.topologyVersion(), OWNING); - Collection nodeIds = F.nodeIds(nodes); // If all affinity nodes are owners, then evict partition from local node. From 4e5b226c24475ccadeb0478e0e8fbb5a99ff4337 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Wed, 12 Dec 2018 18:05:45 +0300 Subject: [PATCH 14/25] IGNITE-10058 rework (wip). --- .../GridDhtPartitionsExchangeFuture.java | 12 ++--- .../GridDhtPartitionTopologyImpl.java | 44 ++++--------------- .../ignite/cache/ResetLostPartitionTest.java | 2 +- 3 files changed, 16 insertions(+), 42 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 f79f8d51006b0..1cda9f9d1dbfd 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 @@ -3243,15 +3243,17 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe if (discoveryCustomMessage instanceof DynamicCacheChangeBatch) { if (exchActions != null) { - assignPartitionsStates(); - Set caches = exchActions.cachesToResetLostPartitions(); - if (!F.isEmpty(caches)) { + boolean resetPartitions = !F.isEmpty(caches); + + if (resetPartitions) resetLostPartitions(caches); - cctx.exchange().refreshPartitions(); - } + assignPartitionsStates(); + + if (resetPartitions) + cctx.exchange().scheduleResendPartitions(); } } else if (discoveryCustomMessage instanceof SnapshotDiscoveryMessage diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index f846046a77527..c90d40f9ce15e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2129,54 +2129,26 @@ else if (plc != PartitionLossPolicy.IGNORE) { try { long updSeq = updateSeq.incrementAndGet(); - Set hasOwners = new HashSet<>(); - for (Map.Entry e : node2part.entrySet()) { - GridDhtPartitionMap partMap = e.getValue(); - - for (Map.Entry e0 : partMap.entrySet()) { - GridDhtPartitionState state = e0.getValue(); - - int part = e0.getKey(); - - if (state == OWNING) - hasOwners.add(part); - - if (state != LOST) - continue; - - AffinityAssignment assignment = grp.affinity().cachedAffinity(resTopVer); - - if (!assignment.idealAssignment().get(part).contains(ctx.discovery().node(e.getKey()))) + for (Map.Entry e0 : e.getValue().entrySet()) { + if (e0.getValue() != LOST) continue; e0.setValue(OWNING); - } - } - // Reset local partitions. - GridDhtPartitionMap locPartMap = node2part.get(ctx.localNodeId()); - - if (locPartMap != null) { - for (Map.Entry e : locPartMap.entrySet()){ - GridDhtLocalPartition locPart = localPartition(e.getKey(), resTopVer, false); + GridDhtLocalPartition locPart = localPartition(e0.getKey(), resTopVer, false); if (locPart != null && locPart.state() == LOST) { - long updateCntr = locPart.updateCounter(); - - if (updateCntr == 0 && e.getValue() != OWNING) - continue; - boolean marked = locPart.own(); if (marked) { updateLocal(locPart.id(), locPart.state(), updSeq, resTopVer); - if (!hasOwners.contains(e.getKey())) { - //Set update counters to 0, for full rebalance. - locPart.updateCounter(updateCntr, -updateCntr); - locPart.initialUpdateCounter(0); - } + long updateCntr = locPart.updateCounter(); + + //Set update counters to 0, for full rebalance. + locPart.updateCounter(updateCntr, -updateCntr); + locPart.initialUpdateCounter(0); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index feb4e3daeb5d3..f0b3cc46067c2 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -113,7 +113,7 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER)); -// cfg.setCacheConfiguration(ccfg); + cfg.setCacheConfiguration(ccfg); return cfg; } From 4b4cecf83027e31d8725b0a42efac7043b644fb0 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 13 Dec 2018 11:40:10 +0300 Subject: [PATCH 15/25] IGNITE-10058 Reset counters only if owners exists. --- .../GridDhtPartitionsExchangeFuture.java | 17 ++++++------ .../GridDhtPartitionTopologyImpl.java | 27 ++++++++++++++++--- .../ignite/cache/ResetLostPartitionTest.java | 1 + 3 files changed, 34 insertions(+), 11 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 1cda9f9d1dbfd..719fb90c1421e 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 @@ -1449,6 +1449,14 @@ private void distributedExchange() throws IgniteCheckedException { try { if (crd.isLocal()) { + if (exchActions != null) { + Collection caches = exchActions.cachesToResetLostPartitions(); + + // Reset lost partitions on coordinator before update cache topology from sinlge messages. + if (!F.isEmpty(caches)) + resetLostPartitions(caches); + } + if (remaining.isEmpty()) onAllReceived(null); } @@ -3243,16 +3251,9 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe if (discoveryCustomMessage instanceof DynamicCacheChangeBatch) { if (exchActions != null) { - Set caches = exchActions.cachesToResetLostPartitions(); - - boolean resetPartitions = !F.isEmpty(caches); - - if (resetPartitions) - resetLostPartitions(caches); - assignPartitionsStates(); - if (resetPartitions) + if (!F.isEmpty(exchActions.cachesToResetLostPartitions())) cctx.exchange().scheduleResendPartitions(); } } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index c90d40f9ce15e..e2f6665597186 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2127,6 +2127,25 @@ else if (plc != PartitionLossPolicy.IGNORE) { lock.writeLock().lock(); try { + // LOST partitions that has at least one owner. + Set hasOwner = new HashSet<>(); + + for (GridDhtLocalPartition part : localPartitions()) { + if (part.state() != LOST) + continue; + + for (Map.Entry e : node2part.entrySet()) { + if (e.getValue().get(part.id()) != OWNING) + continue; + + assert !ctx.localNodeId().equals(e.getKey()); + + hasOwner.add(part.id()); + + break; + } + } + long updSeq = updateSeq.incrementAndGet(); for (Map.Entry e : node2part.entrySet()) { @@ -2146,9 +2165,11 @@ else if (plc != PartitionLossPolicy.IGNORE) { long updateCntr = locPart.updateCounter(); - //Set update counters to 0, for full rebalance. - locPart.updateCounter(updateCntr, -updateCntr); - locPart.initialUpdateCounter(0); + if (hasOwner.contains(locPart.id())) { + // Set update counters to 0, for full rebalance. + locPart.updateCounter(updateCntr, -updateCntr); + locPart.initialUpdateCounter(0); + } } } } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index f0b3cc46067c2..bb839d6b86c7b 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -255,6 +255,7 @@ private void doRebalanceAfterPartitionsWereLost(boolean reactivateGridBeforeRese * * @throws Exception if fail. */ + @Test public void testDuplicateOwners() throws Exception { persistenceEnabled = false; From d83ec8f4052ffbf2b0c76caf6b6e9d3cce4e3d17 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Thu, 13 Dec 2018 12:17:19 +0300 Subject: [PATCH 16/25] IGNITE-10058 Rework test. --- .../ignite/cache/PartitionLossPolicy.java | 4 + .../GridDhtPartitionsExchangeFuture.java | 2 +- .../GridDhtPartitionTopologyImpl.java | 9 +- .../ignite/cache/ResetLostPartitionTest.java | 114 +----------------- ...gniteCachePartitionLossPolicySelfTest.java | 81 ++++++++++++- 5 files changed, 87 insertions(+), 123 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/cache/PartitionLossPolicy.java b/modules/core/src/main/java/org/apache/ignite/cache/PartitionLossPolicy.java index 7c9b2a7ffa692..1ad4cc62f55c8 100644 --- a/modules/core/src/main/java/org/apache/ignite/cache/PartitionLossPolicy.java +++ b/modules/core/src/main/java/org/apache/ignite/cache/PartitionLossPolicy.java @@ -59,7 +59,11 @@ public enum PartitionLossPolicy { /** * All reads and writes will proceed as if all partitions were in a consistent state. The result of reading * from a lost partition is undefined and may be different on different nodes in the cluster. + * + * @deprecated Since 2.8. This policy can lead to weird scenarios that are impossible to solve with current + * architecture. */ + @Deprecated READ_WRITE_ALL, /** 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 719fb90c1421e..93d497db4c3f9 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 @@ -1452,7 +1452,7 @@ private void distributedExchange() throws IgniteCheckedException { if (exchActions != null) { Collection caches = exchActions.cachesToResetLostPartitions(); - // Reset lost partitions on coordinator before update cache topology from sinlge messages. + // Reset lost partitions on coordinator before update cache topology from single messages. if (!F.isEmpty(caches)) resetLostPartitions(caches); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index e2f6665597186..1f9c52d13f2b8 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2130,17 +2130,14 @@ else if (plc != PartitionLossPolicy.IGNORE) { // LOST partitions that has at least one owner. Set hasOwner = new HashSet<>(); - for (GridDhtLocalPartition part : localPartitions()) { - if (part.state() != LOST) - continue; - + for (int p : lostPartitions()) { for (Map.Entry e : node2part.entrySet()) { - if (e.getValue().get(part.id()) != OWNING) + if (e.getValue().get(p) != OWNING) continue; assert !ctx.localNodeId().equals(e.getKey()); - hasOwner.add(part.id()); + hasOwner.add(p); break; } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index bb839d6b86c7b..584dd7352f0b3 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -18,25 +18,19 @@ package org.apache.ignite.cache; import java.util.Arrays; -import java.util.Collection; -import java.util.Collections; import java.util.List; +import java.util.stream.Collectors; import org.apache.ignite.Ignite; import org.apache.ignite.IgniteCache; import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; import org.apache.ignite.configuration.CacheConfiguration; import org.apache.ignite.configuration.DataStorageConfiguration; import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.IgniteEx; import org.apache.ignite.internal.IgnitionEx; import org.apache.ignite.internal.processors.cache.CacheGroupContext; -import org.apache.ignite.internal.processors.cache.GridCacheContext; -import org.apache.ignite.internal.processors.cache.GridCacheSharedContext; -import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState; import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionTopologyImpl; -import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.internal.CU; import org.apache.ignite.internal.util.typedef.internal.U; import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; @@ -48,12 +42,9 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import static java.util.function.Predicate.isEqual; -import static java.util.stream.Collectors.toList; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.LOST; import static org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState.OWNING; import static org.apache.ignite.internal.processors.cache.persistence.file.FilePageStoreManager.DFLT_STORE_DIR; -import static org.apache.ignite.testframework.GridTestUtils.waitForCondition; /** * @@ -67,9 +58,6 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { /** Cache size */ public static final int CACHE_SIZE = 100000 / CACHE_NAMES.length; - /** Persistence enabled flag. */ - private boolean persistenceEnabled = true; - /** {@inheritDoc} */ @Override protected void beforeTest() throws Exception { if (MvccFeatureChecker.forcedMvcc()) @@ -100,8 +88,8 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { DataStorageConfiguration storageCfg = new DataStorageConfiguration(); storageCfg.getDefaultDataRegionConfiguration() - .setPersistenceEnabled(persistenceEnabled) - .setMaxSize(500L * 1024 * 1024); + .setPersistenceEnabled(true) + .setMaxSize(300L * 1024 * 1024); cfg.setDataStorageConfiguration(storageCfg); @@ -250,87 +238,6 @@ private void doRebalanceAfterPartitionsWereLost(boolean reactivateGridBeforeRese assertEquals(CACHE_NAMES.length * CACHE_SIZE, averageSizeAroundAllNodes()); } - /** - * Check that there is no duplicate partition owners after reset lost partitions. - * - * @throws Exception if fail. - */ - @Test - public void testDuplicateOwners() throws Exception { - persistenceEnabled = false; - - int gridCnt = 4; - - long timeout = 5_000; - - Ignite node = startGridsMultiThreaded(gridCnt); - - IgniteCache cache = node.createCache( - new CacheConfiguration(DEFAULT_CACHE_NAME) - .setAffinity(new RendezvousAffinityFunction(false, 32)) - .setPartitionLossPolicy(PartitionLossPolicy.READ_WRITE_SAFE)); - - for (int i = 0; i < CACHE_SIZE; i++) - cache.put(i, i); - - int failedNodeIdx = gridCnt - 1; - - int lostPartsCnt = count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx); - - log.info(">xxx> stop " + failedNodeIdx); - - stopGrid(failedNodeIdx); - - int[] liveIdxs = new int[] {0, 1, 2}; - -// waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, liveIdxs), timeout); -// assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, LOST, liveIdxs)); - - U.sleep(5_000); - - log.info(">xxx> start " + failedNodeIdx); - - startGrid(failedNodeIdx); - - U.sleep(5_000); -// waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx), timeout); -// assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, LOST, failedNodeIdx)); -// -// waitForCondition(() -> 0 == count(DEFAULT_CACHE_NAME, LOST, liveIdxs), timeout); -// assertEquals(0, count(DEFAULT_CACHE_NAME, LOST, liveIdxs)); -// -// for (Ignite grid : G.allGrids()) { -// GridCacheSharedContext cctx = ((IgniteEx)grid).context().cache().context(); -// -// cctx.exchange().affinityReadyFuture(cctx.discovery().topologyVersionEx()).get(timeout); -// -// for (GridCacheContext ctx : (Collection)cctx.cacheContexts()) -// ctx.preloader().rebalanceFuture().get(timeout); -// } - - log.info(">xxx> reset lost partitions"); - - node.resetLostPartitions(Collections.singleton(DEFAULT_CACHE_NAME)); - - U.sleep(5_000); - - log.info(">xxx> check duplicate owners"); - - waitForCondition(() -> lostPartsCnt == count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx), timeout); - assertEquals(lostPartsCnt, count(DEFAULT_CACHE_NAME, OWNING, failedNodeIdx)); - - int parts = grid(0).affinity(DEFAULT_CACHE_NAME).partitions(); - - int[] allIdxs = new int[] {0, 1, 2, 3}; - - waitForCondition(() -> parts == count(DEFAULT_CACHE_NAME, OWNING, allIdxs), timeout); - assertEquals(parts, count(DEFAULT_CACHE_NAME, OWNING, allIdxs)); - - for (int idx : allIdxs) - assertEquals("" + idx, 0, count(DEFAULT_CACHE_NAME, LOST, idx)); - //assert grid(idx).cache(DEFAULT_CACHE_NAME).lostPartitions().isEmpty(); - } - /** * @param gridNumber Grid number. * @param cacheName Cache name. @@ -343,20 +250,7 @@ private List getPartitionsStates(int gridNumber, String c return top.localPartitions().stream() .map(GridDhtLocalPartition::state) - .collect(toList()); - } - - /** - * Counts partitions in the specified state on the specified nodes. - * - * @param cacheName Cache name. - * @param state Partition state. - * @param gridIdx Grid index. - * @return Number of local partitions in the specified state. - */ - private int count(String cacheName, GridDhtPartitionState state, int ... gridIdx) { - return Arrays.stream(gridIdx).map(idx -> - getPartitionsStates(idx, cacheName).stream().filter(isEqual(state)).collect(toList()).size()).sum(); + .collect(Collectors.toList()); } /** diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java index 3855c39cb58af..72df86d900c94 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java @@ -48,6 +48,7 @@ import org.apache.ignite.internal.managers.communication.GridIoMessage; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.CacheGroupContext; +import org.apache.ignite.internal.processors.cache.distributed.CacheBlockOnReadAbstractTest.RunnableX; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsAbstractMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; import org.apache.ignite.internal.util.typedef.F; @@ -66,6 +67,7 @@ import static java.util.Arrays.asList; import static java.util.Collections.singletonList; +import static org.apache.ignite.IgniteSystemProperties.IGNITE_PRELOAD_RESEND_TIMEOUT; import static org.apache.ignite.cache.CacheMode.PARTITIONED; import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; @@ -169,9 +171,15 @@ protected CacheConfiguration cacheConfiguration() { */ @Test public void testReadOnlySafe() throws Exception { - partLossPlc = PartitionLossPolicy.READ_ONLY_SAFE; + checkReadOnlySafe(); + } - checkLostPartition(false, true, killSingleNode); + /** + * @throws Exception if failed. + */ + @Test + public void testReadOnlySafeNoResendDelay() throws Exception { + withoutResendDelay(this::checkReadOnlySafe); } /** @@ -179,10 +187,27 @@ public void testReadOnlySafe() throws Exception { */ @Test public void testReadOnlySafeWithPersistence() throws Exception { - partLossPlc = PartitionLossPolicy.READ_ONLY_SAFE; + isPersistenceEnabled = true; + + checkReadOnlySafe(); + } + /** + * @throws Exception if failed. + */ + @Test + public void testReadOnlySafeWithPersistenceNoResendDelay() throws Exception { isPersistenceEnabled = true; + withoutResendDelay(this::checkReadOnlySafe); + } + + /** + * @throws Exception if failed. + */ + private void checkReadOnlySafe() throws Exception { + partLossPlc = PartitionLossPolicy.READ_ONLY_SAFE; + checkLostPartition(false, true, killSingleNode); } @@ -215,9 +240,15 @@ public void testReadOnlyAllWithPersistence() throws Exception { */ @Test public void testReadWriteSafe() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; + checkReadWriteSafe(); + } - checkLostPartition(true, true, killSingleNode); + /** + * @throws Exception if failed. + */ + @Test + public void testReadWriteSafeNoResendDelay() throws Exception { + withoutResendDelay(this::checkReadWriteSafe); } /** @@ -227,7 +258,24 @@ public void testReadWriteSafe() throws Exception { public void testReadWriteSafeWithPersistence() throws Exception { partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - isPersistenceEnabled = true; + checkReadWriteSafe(); + } + + /** + * @throws Exception if failed. + */ + @Test + public void testReadWriteSafeWithPersistenceNoResendDelay() throws Exception { + partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; + + withoutResendDelay(this::checkReadWriteSafe); + } + + /** + * @throws Exception if failed. + */ + private void checkReadWriteSafe() throws Exception { + partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; checkLostPartition(true, true, killSingleNode); } @@ -842,6 +890,27 @@ protected void checkQueryFails(Ignite node, boolean loc, int... parts) { // No-op. } + /** + * Executes test with IGNITE_PRELOAD_RESEND_TIMEOUT set to zero. + * + * @param r The test method to be called. + * @throws Exception if failed. + */ + private void withoutResendDelay(RunnableX r) throws Exception { + String propName = IGNITE_PRELOAD_RESEND_TIMEOUT; + + String prevVal = System.setProperty(propName, "0"); + + try { + r.runx(); + } finally { + if (prevVal == null) + System.clearProperty(propName); + else + System.setProperty(propName, prevVal); + } + } + /** */ private class TopologyChanger { /** Flag to delay partition exchange */ From 6bdd03ee703af6a8220da46e023dbabd6035bc6b Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 14 Dec 2018 15:26:07 +0300 Subject: [PATCH 17/25] IGNITE-10058 Fix testReactivateGridBeforeResetLostPartitions (lostparts set is empty after restart). --- .../dht/topology/GridDhtPartitionTopologyImpl.java | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 1f9c52d13f2b8..e2f6665597186 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2130,14 +2130,17 @@ else if (plc != PartitionLossPolicy.IGNORE) { // LOST partitions that has at least one owner. Set hasOwner = new HashSet<>(); - for (int p : lostPartitions()) { + for (GridDhtLocalPartition part : localPartitions()) { + if (part.state() != LOST) + continue; + for (Map.Entry e : node2part.entrySet()) { - if (e.getValue().get(p) != OWNING) + if (e.getValue().get(part.id()) != OWNING) continue; assert !ctx.localNodeId().equals(e.getKey()); - hasOwner.add(p); + hasOwner.add(part.id()); break; } From f296972d25695a20dea7de9551c5ad26ec738371 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Fri, 14 Dec 2018 18:15:37 +0300 Subject: [PATCH 18/25] IGNITE-10058 flaky failure (wip). --- .../GridDhtPartitionTopologyImpl.java | 19 +++++++++++++++++++ .../junits/common/GridCommonAbstractTest.java | 2 ++ 2 files changed, 21 insertions(+) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index e2f6665597186..10b6267a6bbd1 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2153,6 +2153,12 @@ else if (plc != PartitionLossPolicy.IGNORE) { if (e0.getValue() != LOST) continue; + + AffinityAssignment assignment = grp.affinity().cachedAffinity(resTopVer); + + if (!assignment.idealAssignment().get(e0.getKey()).contains(ctx.discovery().node(e.getKey()))) + continue; + e0.setValue(OWNING); GridDhtLocalPartition locPart = localPartition(e0.getKey(), resTopVer, false); @@ -2281,6 +2287,19 @@ else if (plc != PartitionLossPolicy.IGNORE) { } } + if (lostParts != null) { + for (Map.Entry e : node2part.entrySet()) { + if (e.getKey().equals(ctx.localNodeId())) + continue; + + for (Integer part : lostParts) { + GridDhtPartitionState state = e.getValue().get(part); + if (state != null && state.active()) + e.getValue().put(part, LOST); + } + } + } + node2part = new GridDhtPartitionFullMap(node2part, updateSeq.incrementAndGet()); } finally { lock.writeLock().unlock(); diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java index e32617d4a2cf1..48a3b1802db95 100755 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java @@ -780,6 +780,8 @@ protected void awaitPartitionMapExchange( for (Map.Entry entry : pMap.entrySet()) { if (System.currentTimeMillis() - start > timeout) { + printPartitionState(dht.name(), 0); + U.dumpThreads(log); throw new IgniteException("Timeout of waiting for partition state update [" + From b2767637fb6a807ba86fd96f5b03b8294e348beb Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Sat, 15 Dec 2018 12:12:12 +0300 Subject: [PATCH 19/25] flaky failure (wip) --- .../processors/cache/CacheAffinitySharedManager.java | 2 +- .../cache/GridCachePartitionExchangeManager.java | 3 +++ .../dht/preloader/GridDhtPartitionsExchangeFuture.java | 9 +++++++-- .../dht/topology/GridDhtPartitionTopologyImpl.java | 2 +- 4 files changed, 12 insertions(+), 4 deletions(-) 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 07fbef1284c62..b7ccecebb335d 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 @@ -263,7 +263,7 @@ private void onCacheGroupStopped(AffinityTopologyVersion topVer) { * @param top Topology. * @param checkGrpId Group ID. */ - void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { + public void checkRebalanceState(GridDhtPartitionTopology top, Integer checkGrpId) { CacheAffinityChangeMessage msg = null; synchronized (mux) { 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 96158d70554ad..0e67794afe539 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 @@ -3157,6 +3157,9 @@ private class ResendTimeoutObject implements GridTimeoutObject { try { if (started.compareAndSet(false, true)) refreshPartitions(); + else { + log.warning("resend already started"); + } } finally { busyLock.readLock().unlock(); 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 93d497db4c3f9..e53ac33a6246d 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 @@ -3253,8 +3253,13 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe if (exchActions != null) { assignPartitionsStates(); - if (!F.isEmpty(exchActions.cachesToResetLostPartitions())) - cctx.exchange().scheduleResendPartitions(); + if (!F.isEmpty(exchActions.cachesToResetLostPartitions())) { + for (String name : exchActions.cachesToResetLostPartitions()) { + GridCacheContext ctx = cctx.cacheContext(CU.cacheId(name)); + + cctx.affinity().checkRebalanceState(ctx.topology(), ctx.groupId()); + } + } } } else if (discoveryCustomMessage instanceof SnapshotDiscoveryMessage diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 10b6267a6bbd1..f87ba903089ab 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2153,7 +2153,6 @@ else if (plc != PartitionLossPolicy.IGNORE) { if (e0.getValue() != LOST) continue; - AffinityAssignment assignment = grp.affinity().cachedAffinity(resTopVer); if (!assignment.idealAssignment().get(e0.getKey()).contains(ctx.discovery().node(e.getKey()))) @@ -2455,6 +2454,7 @@ private boolean checkEvictions(long updateSeq, AffinityAssignment aff) { log.debug("Partitions have been scheduled to resend [reason=" + "Evictions are done [grp" + grp.cacheOrGroupName() + "]"); + log.info("schedule resend"); ctx.exchange().scheduleResendPartitions(); } finally { From 1157382a863b3ad2180d8d253773eda2ecab8c9f Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Sat, 15 Dec 2018 12:23:27 +0300 Subject: [PATCH 20/25] IGNITE-10058 Flaky test mass run. --- .../GridCachePartitionExchangeManager.java | 3 - .../GridDhtPartitionsExchangeFuture.java | 8 +- .../ignite/cache/ResetLostPartitionTest.java | 2 +- ...gniteCachePartitionLossPolicySelfTest.java | 670 +++++++-------- .../IgniteBinaryCacheQueryTestSuite.java | 788 +++++++----------- 5 files changed, 627 insertions(+), 844 deletions(-) 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 0e67794afe539..96158d70554ad 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 @@ -3157,9 +3157,6 @@ private class ResendTimeoutObject implements GridTimeoutObject { try { if (started.compareAndSet(false, true)) refreshPartitions(); - else { - log.warning("resend already started"); - } } finally { busyLock.readLock().unlock(); 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 e53ac33a6246d..ce62993e42f66 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 @@ -3253,12 +3253,10 @@ private void finishExchangeOnCoordinator(@Nullable Collection sndRe if (exchActions != null) { assignPartitionsStates(); - if (!F.isEmpty(exchActions.cachesToResetLostPartitions())) { - for (String name : exchActions.cachesToResetLostPartitions()) { - GridCacheContext ctx = cctx.cacheContext(CU.cacheId(name)); + for (String cache : exchActions.cachesToResetLostPartitions()) { + GridCacheContext ctx = cctx.cacheContext(CU.cacheId(cache)); - cctx.affinity().checkRebalanceState(ctx.topology(), ctx.groupId()); - } + cctx.affinity().checkRebalanceState(ctx.topology(), ctx.groupId()); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java index 584dd7352f0b3..405e32fd874bd 100644 --- a/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java +++ b/modules/core/src/test/java/org/apache/ignite/cache/ResetLostPartitionTest.java @@ -89,7 +89,7 @@ public class ResetLostPartitionTest extends GridCommonAbstractTest { storageCfg.getDefaultDataRegionConfiguration() .setPersistenceEnabled(true) - .setMaxSize(300L * 1024 * 1024); + .setMaxSize(500L * 1024 * 1024); cfg.setDataStorageConfiguration(storageCfg); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java index 72df86d900c94..56068f1a33ed4 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java @@ -157,91 +157,91 @@ protected CacheConfiguration cacheConfiguration() { isPersistenceEnabled = false; } - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - stopAllGrids(); - - cleanPersistenceDir(); - - super.afterTest(); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadOnlySafe() throws Exception { - checkReadOnlySafe(); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadOnlySafeNoResendDelay() throws Exception { - withoutResendDelay(this::checkReadOnlySafe); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadOnlySafeWithPersistence() throws Exception { - isPersistenceEnabled = true; - - checkReadOnlySafe(); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadOnlySafeWithPersistenceNoResendDelay() throws Exception { - isPersistenceEnabled = true; - - withoutResendDelay(this::checkReadOnlySafe); - } - - /** - * @throws Exception if failed. - */ - private void checkReadOnlySafe() throws Exception { - partLossPlc = PartitionLossPolicy.READ_ONLY_SAFE; - - checkLostPartition(false, true, killSingleNode); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadOnlyAll() throws Exception { - partLossPlc = PartitionLossPolicy.READ_ONLY_ALL; - - checkLostPartition(false, false, killSingleNode); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadOnlyAllWithPersistence() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-10041"); - - partLossPlc = PartitionLossPolicy.READ_ONLY_ALL; - - isPersistenceEnabled = true; - - checkLostPartition(false, false, killSingleNode); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafe() throws Exception { - checkReadWriteSafe(); - } +// /** {@inheritDoc} */ +// @Override protected void afterTest() throws Exception { +// stopAllGrids(); +// +// cleanPersistenceDir(); +// +// super.afterTest(); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadOnlySafe() throws Exception { +// checkReadOnlySafe(); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadOnlySafeNoResendDelay() throws Exception { +// withoutResendDelay(this::checkReadOnlySafe); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadOnlySafeWithPersistence() throws Exception { +// isPersistenceEnabled = true; +// +// checkReadOnlySafe(); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadOnlySafeWithPersistenceNoResendDelay() throws Exception { +// isPersistenceEnabled = true; +// +// withoutResendDelay(this::checkReadOnlySafe); +// } +// +// /** +// * @throws Exception if failed. +// */ +// private void checkReadOnlySafe() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_ONLY_SAFE; +// +// checkLostPartition(false, true, killSingleNode); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadOnlyAll() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_ONLY_ALL; +// +// checkLostPartition(false, false, killSingleNode); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadOnlyAllWithPersistence() throws Exception { +// fail("https://issues.apache.org/jira/browse/IGNITE-10041"); +// +// partLossPlc = PartitionLossPolicy.READ_ONLY_ALL; +// +// isPersistenceEnabled = true; +// +// checkLostPartition(false, false, killSingleNode); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafe() throws Exception { +// checkReadWriteSafe(); +// } /** * @throws Exception if failed. @@ -251,26 +251,26 @@ public void testReadWriteSafeNoResendDelay() throws Exception { withoutResendDelay(this::checkReadWriteSafe); } - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeWithPersistence() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - checkReadWriteSafe(); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeWithPersistenceNoResendDelay() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - withoutResendDelay(this::checkReadWriteSafe); - } - +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeWithPersistence() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// checkReadWriteSafe(); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeWithPersistenceNoResendDelay() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// withoutResendDelay(this::checkReadWriteSafe); +// } +// /** * @throws Exception if failed. */ @@ -279,236 +279,236 @@ private void checkReadWriteSafe() throws Exception { checkLostPartition(true, true, killSingleNode); } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteAll() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_ALL; - - checkLostPartition(true, false, killSingleNode); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteAllWithPersistence() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-10041"); - - partLossPlc = PartitionLossPolicy.READ_WRITE_ALL; - - isPersistenceEnabled = true; - - checkLostPartition(true, false, killSingleNode); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeAfterKillTwoNodes() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeAfterKillTwoNodesWithPersistence() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - isPersistenceEnabled = true; - - checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeAfterKillTwoNodesWithDelay() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 20)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeAfterKillTwoNodesWithDelayWithPersistence() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - isPersistenceEnabled = true; - - checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 20)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeWithBackupsAfterKillThreeNodes() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - backups = 1; - - checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2, 1), asList(0, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeWithBackupsAfterKillThreeNodesWithPersistence() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - backups = 1; - - isPersistenceEnabled = true; - - checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2, 1), asList(0, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeAfterKillCrd() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeAfterKillCrdWithPersistence() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - isPersistenceEnabled = true; - - checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeWithBackups() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - backups = 1; - - checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2), asList(0, 1, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeWithBackupsWithPersistence() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - backups = 1; - - isPersistenceEnabled = true; - - checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2), asList(0, 1, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeWithBackupsAfterKillCrd() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - backups = 1; - - checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafeWithBackupsAfterKillCrdWithPersistence() throws Exception { - partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; - - backups = 1; - - isPersistenceEnabled = true; - - checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0)); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testIgnore() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-5078"); - - partLossPlc = PartitionLossPolicy.IGNORE; - - checkIgnore(killSingleNode); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testIgnoreWithPersistence() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-5078"); - - fail("https://issues.apache.org/jira/browse/IGNITE-10041"); - - partLossPlc = PartitionLossPolicy.IGNORE; - - isPersistenceEnabled = true; - - checkIgnore(killSingleNode); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testIgnoreKillThreeNodes() throws Exception { - partLossPlc = PartitionLossPolicy.IGNORE; - - // TODO aliveNodes should include node 4, but it fails due to https://issues.apache.org/jira/browse/IGNITE-5078. - // TODO need to add 4 to the aliveNodes after IGNITE-5078 is fixed. - // TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, Arrays.asList(1, 2, 3), Arrays.asList(0, 4), 0); - TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, asList(1, 2, 3), singletonList(0), 0); - - checkIgnore(onlyCrdIsAlive); - } - - /** - * @throws Exception if failed. - */ - @Test - public void testIgnoreKillThreeNodesWithPersistence() throws Exception { - fail("https://issues.apache.org/jira/browse/IGNITE-10041"); - - partLossPlc = PartitionLossPolicy.IGNORE; - - isPersistenceEnabled = true; - - // TODO aliveNodes should include node 4, but it fails due to https://issues.apache.org/jira/browse/IGNITE-5078. - // TODO need to add 4 to the aliveNodes after IGNITE-5078 is fixed. - // TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, Arrays.asList(1, 2, 3), Arrays.asList(0, 4), 0); - TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, asList(1, 2, 3), singletonList(0), 0); - - checkIgnore(onlyCrdIsAlive); - } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteAll() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_ALL; +// +// checkLostPartition(true, false, killSingleNode); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteAllWithPersistence() throws Exception { +// fail("https://issues.apache.org/jira/browse/IGNITE-10041"); +// +// partLossPlc = PartitionLossPolicy.READ_WRITE_ALL; +// +// isPersistenceEnabled = true; +// +// checkLostPartition(true, false, killSingleNode); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeAfterKillTwoNodes() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeAfterKillTwoNodesWithPersistence() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// isPersistenceEnabled = true; +// +// checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeAfterKillTwoNodesWithDelay() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 20)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeAfterKillTwoNodesWithDelayWithPersistence() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// isPersistenceEnabled = true; +// +// checkLostPartition(true, true, new TopologyChanger(false, asList(3, 2), asList(0, 1, 4), 20)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeWithBackupsAfterKillThreeNodes() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// backups = 1; +// +// checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2, 1), asList(0, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeWithBackupsAfterKillThreeNodesWithPersistence() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// backups = 1; +// +// isPersistenceEnabled = true; +// +// checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2, 1), asList(0, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeAfterKillCrd() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeAfterKillCrdWithPersistence() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// isPersistenceEnabled = true; +// +// checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeWithBackups() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// backups = 1; +// +// checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2), asList(0, 1, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeWithBackupsWithPersistence() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// backups = 1; +// +// isPersistenceEnabled = true; +// +// checkLostPartition(true, true, new TopologyChanger(true, asList(3, 2), asList(0, 1, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeWithBackupsAfterKillCrd() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// backups = 1; +// +// checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafeWithBackupsAfterKillCrdWithPersistence() throws Exception { +// partLossPlc = PartitionLossPolicy.READ_WRITE_SAFE; +// +// backups = 1; +// +// isPersistenceEnabled = true; +// +// checkLostPartition(true, true, new TopologyChanger(true, asList(3, 0), asList(1, 2, 4), 0)); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testIgnore() throws Exception { +// fail("https://issues.apache.org/jira/browse/IGNITE-5078"); +// +// partLossPlc = PartitionLossPolicy.IGNORE; +// +// checkIgnore(killSingleNode); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testIgnoreWithPersistence() throws Exception { +// fail("https://issues.apache.org/jira/browse/IGNITE-5078"); +// +// fail("https://issues.apache.org/jira/browse/IGNITE-10041"); +// +// partLossPlc = PartitionLossPolicy.IGNORE; +// +// isPersistenceEnabled = true; +// +// checkIgnore(killSingleNode); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testIgnoreKillThreeNodes() throws Exception { +// partLossPlc = PartitionLossPolicy.IGNORE; +// +// // TODO aliveNodes should include node 4, but it fails due to https://issues.apache.org/jira/browse/IGNITE-5078. +// // TODO need to add 4 to the aliveNodes after IGNITE-5078 is fixed. +// // TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, Arrays.asList(1, 2, 3), Arrays.asList(0, 4), 0); +// TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, asList(1, 2, 3), singletonList(0), 0); +// +// checkIgnore(onlyCrdIsAlive); +// } +// +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testIgnoreKillThreeNodesWithPersistence() throws Exception { +// fail("https://issues.apache.org/jira/browse/IGNITE-10041"); +// +// partLossPlc = PartitionLossPolicy.IGNORE; +// +// isPersistenceEnabled = true; +// +// // TODO aliveNodes should include node 4, but it fails due to https://issues.apache.org/jira/browse/IGNITE-5078. +// // TODO need to add 4 to the aliveNodes after IGNITE-5078 is fixed. +// // TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, Arrays.asList(1, 2, 3), Arrays.asList(0, 4), 0); +// TopologyChanger onlyCrdIsAlive = new TopologyChanger(false, asList(1, 2, 3), singletonList(0), 0); +// +// checkIgnore(onlyCrdIsAlive); +// } /** * @param topChanger topology changer. diff --git a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java index d510b1115d5f6..fc959f0c4898c 100644 --- a/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java +++ b/modules/indexing/src/test/java/org/apache/ignite/testsuites/IgniteBinaryCacheQueryTestSuite.java @@ -18,220 +18,7 @@ package org.apache.ignite.testsuites; import junit.framework.TestSuite; -import org.apache.ignite.internal.processors.cache.AffinityKeyNameAndValueFieldNameConflictTest; -import org.apache.ignite.internal.processors.cache.BigEntryQueryTest; -import org.apache.ignite.internal.processors.cache.BinaryMetadataConcurrentUpdateWithIndexesTest; -import org.apache.ignite.internal.processors.cache.BinarySerializationQuerySelfTest; -import org.apache.ignite.internal.processors.cache.BinarySerializationQueryWithReflectiveSerializerSelfTest; -import org.apache.ignite.internal.processors.cache.CacheIteratorScanQueryTest; -import org.apache.ignite.internal.processors.cache.CacheLocalQueryDetailMetricsSelfTest; -import org.apache.ignite.internal.processors.cache.CacheLocalQueryMetricsSelfTest; -import org.apache.ignite.internal.processors.cache.CacheOffheapBatchIndexingMultiTypeTest; -import org.apache.ignite.internal.processors.cache.CacheOffheapBatchIndexingSingleTypeTest; -import org.apache.ignite.internal.processors.cache.CachePartitionedQueryDetailMetricsDistributedSelfTest; -import org.apache.ignite.internal.processors.cache.CachePartitionedQueryDetailMetricsLocalSelfTest; -import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsDistributedSelfTest; -import org.apache.ignite.internal.processors.cache.CachePartitionedQueryMetricsLocalSelfTest; -import org.apache.ignite.internal.processors.cache.CacheQueryBuildValueTest; -import org.apache.ignite.internal.processors.cache.CacheQueryEvictDataLostTest; -import org.apache.ignite.internal.processors.cache.CacheQueryNewClientSelfTest; -import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryDetailMetricsDistributedSelfTest; -import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryDetailMetricsLocalSelfTest; -import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsDistributedSelfTest; -import org.apache.ignite.internal.processors.cache.CacheReplicatedQueryMetricsLocalSelfTest; -import org.apache.ignite.internal.processors.cache.CacheSqlQueryValueCopySelfTest; -import org.apache.ignite.internal.processors.cache.DdlTransactionSelfTest; -import org.apache.ignite.internal.processors.cache.GridCacheCrossCacheQuerySelfTest; -import org.apache.ignite.internal.processors.cache.GridCacheDynamicLoadOnClientPersistentTest; -import org.apache.ignite.internal.processors.cache.GridCacheDynamicLoadOnClientTest; -import org.apache.ignite.internal.processors.cache.GridCacheFullTextQuerySelfTest; -import org.apache.ignite.internal.processors.cache.GridCacheLazyQueryPartitionsReleaseTest; -import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexDisabledSelfTest; -import org.apache.ignite.internal.processors.cache.GridCacheQueryIndexingDisabledSelfTest; -import org.apache.ignite.internal.processors.cache.GridCacheQueryInternalKeysSelfTest; -import org.apache.ignite.internal.processors.cache.GridCacheQuerySerializationSelfTest; -import org.apache.ignite.internal.processors.cache.GridCacheQuerySqlFieldInlineSizeSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectLocalQueryArgumentsTest; -import org.apache.ignite.internal.processors.cache.IgniteBinaryObjectQueryArgumentsTest; -import org.apache.ignite.internal.processors.cache.IgniteBinaryWrappedObjectFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheBinaryObjectsScanSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheBinaryObjectsScanWithEventsSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheCollocatedQuerySelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheDeleteSqlQuerySelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCollocatedAndNotTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinCustomAffinityMapper; -import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinNoIndexTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinPartitionedAndReplicatedTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinQueryConditionsTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheDistributedJoinTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheDuplicateEntityConfigurationSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheFieldsQueryNoDataSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheFullTextQueryNodeJoiningSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheInsertSqlQuerySelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheJoinPartitionedAndReplicatedCollocationTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheJoinPartitionedAndReplicatedTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheJoinQueryWithAffinityKeyTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheLargeResultSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheMergeSqlQuerySelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheMultipleIndexedTypesTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheNoClassQuerySelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapEvictQueryTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheOffheapIndexScanTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheP2pUnmarshallingQueryErrorTest; -import org.apache.ignite.internal.processors.cache.IgniteCachePrimitiveFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheQueryH2IndexingLeakTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheQueryIndexSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheQueryLoadSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheSqlQueryErrorSelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheUnionDuplicatesTest; -import org.apache.ignite.internal.processors.cache.IgniteCacheUpdateSqlQuerySelfTest; -import org.apache.ignite.internal.processors.cache.IgniteCheckClusterStateBeforeExecuteQueryTest; -import org.apache.ignite.internal.processors.cache.IgniteClientReconnectCacheQueriesFailoverTest; -import org.apache.ignite.internal.processors.cache.IgniteCrossCachesJoinsQueryTest; -import org.apache.ignite.internal.processors.cache.IgniteDynamicSqlRestoreTest; -import org.apache.ignite.internal.processors.cache.IgniteErrorOnRebalanceTest; -import org.apache.ignite.internal.processors.cache.IncorrectQueryEntityTest; import org.apache.ignite.internal.processors.cache.IndexingCachePartitionLossPolicySelfTest; -import org.apache.ignite.internal.processors.cache.QueryEntityCaseMismatchTest; -import org.apache.ignite.internal.processors.cache.SqlFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.authentication.SqlUserCommandSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicNearEnabledQuerySelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheAtomicQuerySelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedPartitionQueryConfigurationSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedPartitionQueryNodeRestartsSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedPartitionQuerySelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheDistributedQueryCancelSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQueryEvtsDisabledSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQueryP2PDisabledSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedQuerySelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCachePartitionedSnapshotEnabledQuerySelfTest; -import org.apache.ignite.internal.processors.cache.distributed.near.IgniteCacheQueryNoRebalanceSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryJoinNoPrimaryPartitionsSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQueryROSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryEvtsDisabledSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQueryP2PDisabledSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheReplicatedQuerySelfTest; -import org.apache.ignite.internal.processors.cache.encryption.EncryptedSqlTableTest; -import org.apache.ignite.internal.processors.cache.index.BasicIndexTest; -import org.apache.ignite.internal.processors.cache.index.ComplexPrimaryKeyUnwrapSelfTest; -import org.apache.ignite.internal.processors.cache.index.DuplicateKeyValueClassesSelfTest; -import org.apache.ignite.internal.processors.cache.index.DynamicIndexClientBasicSelfTest; -import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerBasicSelfTest; -import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerCoordinatorBasicSelfTest; -import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFIlterBasicSelfTest; -import org.apache.ignite.internal.processors.cache.index.DynamicIndexServerNodeFilterCoordinatorBasicSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2ConnectionLeaksSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsClientBasicSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsServerBasicSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicColumnsServerCoordinatorBasicSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexAtomicPartitionedNearSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexAtomicPartitionedSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexAtomicReplicatedSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexTransactionalPartitionedNearSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexTransactionalPartitionedSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexTransactionalReplicatedSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexClientAtomicPartitionedNoBackupsTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexClientAtomicPartitionedTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexClientAtomicReplicatedTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexClientTransactionalPartitionedNoBackupsTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexClientTransactionalPartitionedTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexClientTransactionalReplicatedTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexServerAtomicPartitionedNoBackupsTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexServerAtomicPartitionedTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexServerAtomicReplicatedTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexServerTransactionalPartitionedTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicIndexingComplexServerTransactionalReplicatedTest; -import org.apache.ignite.internal.processors.cache.index.H2DynamicTableSelfTest; -import org.apache.ignite.internal.processors.cache.index.H2RowCachePageEvictionTest; -import org.apache.ignite.internal.processors.cache.index.H2RowCacheSelfTest; -import org.apache.ignite.internal.processors.cache.index.IgniteDecimalSelfTest; -import org.apache.ignite.internal.processors.cache.index.LongIndexNameTest; -import org.apache.ignite.internal.processors.cache.index.OptimizedMarshallerIndexNameTest; -import org.apache.ignite.internal.processors.cache.index.QueryEntityValidationSelfTest; -import org.apache.ignite.internal.processors.cache.index.SchemaExchangeSelfTest; -import org.apache.ignite.internal.processors.cache.index.SqlTransactionCommandsWithMvccDisabledSelfTest; -import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalAtomicQuerySelfTest; -import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQueryCancelOrTimeoutSelfTest; -import org.apache.ignite.internal.processors.cache.local.IgniteCacheLocalQuerySelfTest; -import org.apache.ignite.internal.processors.cache.query.CacheScanQueryFailoverTest; -import org.apache.ignite.internal.processors.cache.query.GridCacheQueryTransformerSelfTest; -import org.apache.ignite.internal.processors.cache.query.GridCircularQueueTest; -import org.apache.ignite.internal.processors.cache.query.IgniteCacheQueryCacheDestroySelfTest; -import org.apache.ignite.internal.processors.cache.query.IndexingSpiQuerySelfTest; -import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryTxSelfTest; -import org.apache.ignite.internal.processors.cache.query.IndexingSpiQueryWithH2IndexingSelfTest; -import org.apache.ignite.internal.processors.cache.transaction.DmlInsideTransactionTest; -import org.apache.ignite.internal.processors.client.ClientConnectorConfigurationValidationSelfTest; -import org.apache.ignite.internal.processors.database.baseline.IgniteStableBaselineBinObjFieldsQuerySelfTest; -import org.apache.ignite.internal.processors.query.IgniteCachelessQueriesSelfTest; -import org.apache.ignite.internal.processors.query.IgniteQueryDedicatedPoolTest; -import org.apache.ignite.internal.processors.query.IgniteSqlDefaultValueTest; -import org.apache.ignite.internal.processors.query.IgniteSqlDistributedJoinSelfTest; -import org.apache.ignite.internal.processors.query.IgniteSqlEntryCacheModeAgnosticTest; -import org.apache.ignite.internal.processors.query.IgniteSqlGroupConcatCollocatedTest; -import org.apache.ignite.internal.processors.query.IgniteSqlGroupConcatNotCollocatedTest; -import org.apache.ignite.internal.processors.query.IgniteSqlKeyValueFieldsTest; -import org.apache.ignite.internal.processors.query.IgniteSqlNotNullConstraintTest; -import org.apache.ignite.internal.processors.query.IgniteSqlParameterizedQueryTest; -import org.apache.ignite.internal.processors.query.IgniteSqlQueryParallelismTest; -import org.apache.ignite.internal.processors.query.IgniteSqlRoutingTest; -import org.apache.ignite.internal.processors.query.IgniteSqlSchemaIndexingTest; -import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexMultiNodeSelfTest; -import org.apache.ignite.internal.processors.query.IgniteSqlSegmentedIndexSelfTest; -import org.apache.ignite.internal.processors.query.IgniteSqlSkipReducerOnUpdateDmlFlagSelfTest; -import org.apache.ignite.internal.processors.query.IgniteSqlSkipReducerOnUpdateDmlSelfTest; -import org.apache.ignite.internal.processors.query.IgniteSqlSplitterSelfTest; -import org.apache.ignite.internal.processors.query.LazyQuerySelfTest; -import org.apache.ignite.internal.processors.query.MultipleStatementsSqlQuerySelfTest; -import org.apache.ignite.internal.processors.query.RunningQueriesTest; -import org.apache.ignite.internal.processors.query.SqlIllegalSchemaSelfTest; -import org.apache.ignite.internal.processors.query.SqlNestedQuerySelfTest; -import org.apache.ignite.internal.processors.query.SqlPushDownFunctionTest; -import org.apache.ignite.internal.processors.query.SqlSchemaSelfTest; -import org.apache.ignite.internal.processors.query.SqlSystemViewsSelfTest; -import org.apache.ignite.internal.processors.query.h2.GridH2IndexingInMemSelfTest; -import org.apache.ignite.internal.processors.query.h2.GridH2IndexingOffheapSelfTest; -import org.apache.ignite.internal.processors.query.h2.GridIndexRebuildSelfTest; -import org.apache.ignite.internal.processors.query.h2.H2ResultSetIteratorNullifyOnEndSelfTest; -import org.apache.ignite.internal.processors.query.h2.H2StatementCacheSelfTest; -import org.apache.ignite.internal.processors.query.h2.IgniteSqlBigIntegerKeyTest; -import org.apache.ignite.internal.processors.query.h2.IgniteSqlQueryMinMaxTest; -import org.apache.ignite.internal.processors.query.h2.PreparedStatementExSelfTest; -import org.apache.ignite.internal.processors.query.h2.ThreadLocalObjectPoolSelfTest; -import org.apache.ignite.internal.processors.query.h2.sql.BaseH2CompareQueryTest; -import org.apache.ignite.internal.processors.query.h2.sql.ExplainSelfTest; -import org.apache.ignite.internal.processors.query.h2.sql.GridQueryParsingTest; -import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryDistributedJoinsTest; -import org.apache.ignite.internal.processors.query.h2.sql.H2CompareBigQueryTest; -import org.apache.ignite.internal.processors.query.h2.twostep.AndOperationExtractPartitionSelfTest; -import org.apache.ignite.internal.processors.query.h2.twostep.InOperationExtractPartitionSelfTest; -import org.apache.ignite.internal.processors.sql.IgniteCachePartitionedAtomicColumnConstraintsTest; -import org.apache.ignite.internal.processors.sql.IgniteCachePartitionedTransactionalColumnConstraintsTest; -import org.apache.ignite.internal.processors.sql.IgniteCachePartitionedTransactionalSnapshotColumnConstraintTest; -import org.apache.ignite.internal.processors.sql.IgniteCacheReplicatedAtomicColumnConstraintsTest; -import org.apache.ignite.internal.processors.sql.IgniteCacheReplicatedTransactionalColumnConstraintsTest; -import org.apache.ignite.internal.processors.sql.IgniteCacheReplicatedTransactionalSnapshotColumnConstraintTest; -import org.apache.ignite.internal.processors.sql.IgniteSQLColumnConstraintsTest; -import org.apache.ignite.internal.processors.sql.IgniteTransactionSQLColumnConstraintTest; -import org.apache.ignite.internal.processors.sql.SqlConnectorConfigurationValidationSelfTest; -import org.apache.ignite.internal.sql.SqlParserBulkLoadSelfTest; -import org.apache.ignite.internal.sql.SqlParserCreateIndexSelfTest; -import org.apache.ignite.internal.sql.SqlParserDropIndexSelfTest; -import org.apache.ignite.internal.sql.SqlParserSetStreamingSelfTest; -import org.apache.ignite.internal.sql.SqlParserTransactionalKeywordsSelfTest; -import org.apache.ignite.internal.sql.SqlParserUserSelfTest; -import org.apache.ignite.spi.communication.tcp.GridOrderedMessageCancelSelfTest; -import org.apache.ignite.sqltests.PartitionedSqlTest; -import org.apache.ignite.sqltests.ReplicatedSqlTest; import org.apache.ignite.testframework.IgniteTestSuite; /** @@ -244,295 +31,296 @@ public class IgniteBinaryCacheQueryTestSuite extends TestSuite { public static TestSuite suite() { IgniteTestSuite suite = new IgniteTestSuite("Ignite Cache Queries Test Suite"); - suite.addTestSuite(AffinityKeyNameAndValueFieldNameConflictTest.class); - suite.addTestSuite(DmlInsideTransactionTest.class); - suite.addTestSuite(ComplexPrimaryKeyUnwrapSelfTest.class); - suite.addTestSuite(SqlNestedQuerySelfTest.class); - suite.addTestSuite(ExplainSelfTest.class); - suite.addTestSuite(RunningQueriesTest.class); - - suite.addTestSuite(ComplexPrimaryKeyUnwrapSelfTest.class); - - suite.addTestSuite(PartitionedSqlTest.class); - suite.addTestSuite(ReplicatedSqlTest.class); - - suite.addTestSuite(SqlParserCreateIndexSelfTest.class); - suite.addTestSuite(SqlParserDropIndexSelfTest.class); - suite.addTestSuite(SqlParserTransactionalKeywordsSelfTest.class); - suite.addTestSuite(SqlParserBulkLoadSelfTest.class); - suite.addTestSuite(SqlParserSetStreamingSelfTest.class); - - suite.addTestSuite(SqlConnectorConfigurationValidationSelfTest.class); - suite.addTestSuite(ClientConnectorConfigurationValidationSelfTest.class); - - suite.addTestSuite(SqlSchemaSelfTest.class); - suite.addTestSuite(SqlIllegalSchemaSelfTest.class); - suite.addTestSuite(MultipleStatementsSqlQuerySelfTest.class); - - suite.addTestSuite(BasicIndexTest.class); - - // Misc tests. - suite.addTestSuite(QueryEntityValidationSelfTest.class); - suite.addTestSuite(DuplicateKeyValueClassesSelfTest.class); - suite.addTestSuite(GridCacheLazyQueryPartitionsReleaseTest.class); - - // Dynamic index create/drop tests. - suite.addTestSuite(SchemaExchangeSelfTest.class); - - suite.addTestSuite(DynamicIndexServerCoordinatorBasicSelfTest.class); - suite.addTestSuite(DynamicIndexServerBasicSelfTest.class); - suite.addTestSuite(DynamicIndexServerNodeFilterCoordinatorBasicSelfTest.class); - suite.addTestSuite(DynamicIndexServerNodeFIlterBasicSelfTest.class); - suite.addTestSuite(DynamicIndexClientBasicSelfTest.class); - - // H2 tests. - - suite.addTestSuite(GridH2IndexingInMemSelfTest.class); - suite.addTestSuite(GridH2IndexingOffheapSelfTest.class); - - // Parsing - suite.addTestSuite(GridQueryParsingTest.class); - suite.addTestSuite(IgniteCacheSqlQueryErrorSelfTest.class); - - // Config. - suite.addTestSuite(IgniteCacheDuplicateEntityConfigurationSelfTest.class); - suite.addTestSuite(IncorrectQueryEntityTest.class); - suite.addTestSuite(IgniteDynamicSqlRestoreTest.class); - - // Queries tests. - suite.addTestSuite(LazyQuerySelfTest.class); - suite.addTestSuite(IgniteSqlSplitterSelfTest.class); - suite.addTestSuite(SqlPushDownFunctionTest.class); - suite.addTestSuite(IgniteSqlSegmentedIndexSelfTest.class); - suite.addTestSuite(IgniteCachelessQueriesSelfTest.class); - suite.addTestSuite(IgniteSqlSegmentedIndexMultiNodeSelfTest.class); - suite.addTestSuite(IgniteSqlSchemaIndexingTest.class); - suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class); - suite.addTestSuite(IgniteCacheQueryLoadSelfTest.class); - suite.addTestSuite(IgniteCacheLocalQuerySelfTest.class); - suite.addTestSuite(IgniteCacheLocalAtomicQuerySelfTest.class); - suite.addTestSuite(IgniteCacheReplicatedQuerySelfTest.class); - suite.addTestSuite(IgniteCacheReplicatedQueryP2PDisabledSelfTest.class); - suite.addTestSuite(IgniteCacheReplicatedQueryEvtsDisabledSelfTest.class); - suite.addTestSuite(IgniteCachePartitionedQuerySelfTest.class); - suite.addTestSuite(IgniteCachePartitionedSnapshotEnabledQuerySelfTest.class); - suite.addTestSuite(IgniteCacheAtomicQuerySelfTest.class); - suite.addTestSuite(IgniteCacheAtomicNearEnabledQuerySelfTest.class); - suite.addTestSuite(IgniteCachePartitionedQueryP2PDisabledSelfTest.class); - suite.addTestSuite(IgniteCachePartitionedQueryEvtsDisabledSelfTest.class); - - suite.addTestSuite(IgniteCacheUnionDuplicatesTest.class); - suite.addTestSuite(IgniteCacheJoinPartitionedAndReplicatedCollocationTest.class); - suite.addTestSuite(IgniteClientReconnectCacheQueriesFailoverTest.class); - suite.addTestSuite(IgniteErrorOnRebalanceTest.class); - suite.addTestSuite(CacheQueryBuildValueTest.class); - suite.addTestSuite(CacheOffheapBatchIndexingMultiTypeTest.class); - - suite.addTestSuite(IgniteCacheQueryIndexSelfTest.class); - suite.addTestSuite(IgniteCacheCollocatedQuerySelfTest.class); - suite.addTestSuite(IgniteCacheLargeResultSelfTest.class); - suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class); - suite.addTestSuite(H2ResultSetIteratorNullifyOnEndSelfTest.class); - suite.addTestSuite(IgniteSqlBigIntegerKeyTest.class); - suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class); - suite.addTestSuite(IgniteCacheOffheapIndexScanTest.class); - - suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class); - suite.addTestSuite(GridCacheQuerySerializationSelfTest.class); - suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class); - suite.addTestSuite(IgniteStableBaselineBinObjFieldsQuerySelfTest.class); - suite.addTestSuite(IgniteBinaryWrappedObjectFieldsQuerySelfTest.class); - suite.addTestSuite(IgniteCacheQueryH2IndexingLeakTest.class); - suite.addTestSuite(IgniteCacheQueryNoRebalanceSelfTest.class); - suite.addTestSuite(GridCacheQueryTransformerSelfTest.class); - suite.addTestSuite(CacheScanQueryFailoverTest.class); - suite.addTestSuite(IgniteCachePrimitiveFieldsQuerySelfTest.class); - - suite.addTestSuite(IgniteCacheJoinQueryWithAffinityKeyTest.class); - suite.addTestSuite(IgniteCacheJoinPartitionedAndReplicatedTest.class); - suite.addTestSuite(IgniteCrossCachesJoinsQueryTest.class); - - suite.addTestSuite(IgniteCacheMultipleIndexedTypesTest.class); - - // DML. - suite.addTestSuite(IgniteCacheMergeSqlQuerySelfTest.class); - suite.addTestSuite(IgniteCacheInsertSqlQuerySelfTest.class); - suite.addTestSuite(IgniteCacheUpdateSqlQuerySelfTest.class); - suite.addTestSuite(IgniteCacheDeleteSqlQuerySelfTest.class); - suite.addTestSuite(IgniteSqlSkipReducerOnUpdateDmlSelfTest.class); - suite.addTestSuite(IgniteSqlSkipReducerOnUpdateDmlFlagSelfTest.class); - - suite.addTestSuite(IgniteBinaryObjectQueryArgumentsTest.class); - suite.addTestSuite(IgniteBinaryObjectLocalQueryArgumentsTest.class); - - suite.addTestSuite(IndexingSpiQuerySelfTest.class); - suite.addTestSuite(IndexingSpiQueryTxSelfTest.class); - - suite.addTestSuite(IgniteCacheMultipleIndexedTypesTest.class); - suite.addTestSuite(IgniteSqlQueryMinMaxTest.class); - - suite.addTestSuite(GridCircularQueueTest.class); - suite.addTestSuite(IndexingSpiQueryWithH2IndexingSelfTest.class); - - // DDL. - suite.addTestSuite(H2DynamicIndexTransactionalReplicatedSelfTest.class); - suite.addTestSuite(H2DynamicIndexTransactionalPartitionedSelfTest.class); - suite.addTestSuite(H2DynamicIndexTransactionalPartitionedNearSelfTest.class); - suite.addTestSuite(H2DynamicIndexAtomicReplicatedSelfTest.class); - suite.addTestSuite(H2DynamicIndexAtomicPartitionedSelfTest.class); - suite.addTestSuite(H2DynamicIndexAtomicPartitionedNearSelfTest.class); - suite.addTestSuite(H2DynamicTableSelfTest.class); - suite.addTestSuite(H2DynamicColumnsClientBasicSelfTest.class); - suite.addTestSuite(H2DynamicColumnsServerBasicSelfTest.class); - suite.addTestSuite(H2DynamicColumnsServerCoordinatorBasicSelfTest.class); - - // DML+DDL. - suite.addTestSuite(H2DynamicIndexingComplexClientAtomicPartitionedTest.class); - suite.addTestSuite(H2DynamicIndexingComplexClientAtomicPartitionedNoBackupsTest.class); - suite.addTestSuite(H2DynamicIndexingComplexClientAtomicReplicatedTest.class); - suite.addTestSuite(H2DynamicIndexingComplexClientTransactionalPartitionedTest.class); - suite.addTestSuite(H2DynamicIndexingComplexClientTransactionalPartitionedNoBackupsTest.class); - suite.addTestSuite(H2DynamicIndexingComplexClientTransactionalReplicatedTest.class); - suite.addTestSuite(H2DynamicIndexingComplexServerAtomicPartitionedTest.class); - suite.addTestSuite(H2DynamicIndexingComplexServerAtomicPartitionedNoBackupsTest.class); - suite.addTestSuite(H2DynamicIndexingComplexServerAtomicReplicatedTest.class); - suite.addTestSuite(H2DynamicIndexingComplexServerTransactionalPartitionedTest.class); - suite.addTestSuite(H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest.class); - suite.addTestSuite(H2DynamicIndexingComplexServerTransactionalReplicatedTest.class); - - suite.addTestSuite(DdlTransactionSelfTest.class); - - // Fields queries. - suite.addTestSuite(SqlFieldsQuerySelfTest.class); - suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class); - suite.addTestSuite(IgniteCacheReplicatedFieldsQuerySelfTest.class); - suite.addTestSuite(IgniteCacheReplicatedFieldsQueryROSelfTest.class); - suite.addTestSuite(IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest.class); - suite.addTestSuite(IgniteCacheReplicatedFieldsQueryJoinNoPrimaryPartitionsSelfTest.class); - suite.addTestSuite(IgniteCachePartitionedFieldsQuerySelfTest.class); - suite.addTestSuite(IgniteCacheAtomicFieldsQuerySelfTest.class); - suite.addTestSuite(IgniteCacheAtomicNearEnabledFieldsQuerySelfTest.class); - suite.addTestSuite(IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest.class); - suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class); - suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class); - suite.addTestSuite(GridOrderedMessageCancelSelfTest.class); - suite.addTestSuite(CacheQueryEvictDataLostTest.class); - - // Full text queries. - suite.addTestSuite(GridCacheFullTextQuerySelfTest.class); - suite.addTestSuite(IgniteCacheFullTextQueryNodeJoiningSelfTest.class); - - // Ignite cache and H2 comparison. - suite.addTestSuite(BaseH2CompareQueryTest.class); - suite.addTestSuite(H2CompareBigQueryTest.class); - suite.addTestSuite(H2CompareBigQueryDistributedJoinsTest.class); - - // Cache query metrics. - suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class); - suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class); - suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class); - suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class); - suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class); - - // Cache query metrics. - suite.addTestSuite(CacheLocalQueryDetailMetricsSelfTest.class); - suite.addTestSuite(CachePartitionedQueryDetailMetricsDistributedSelfTest.class); - suite.addTestSuite(CachePartitionedQueryDetailMetricsLocalSelfTest.class); - suite.addTestSuite(CacheReplicatedQueryDetailMetricsDistributedSelfTest.class); - suite.addTestSuite(CacheReplicatedQueryDetailMetricsLocalSelfTest.class); - - // Unmarshalling query test. - suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class); - suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class); - - // Cancellation. - suite.addTestSuite(IgniteCacheDistributedQueryCancelSelfTest.class); - suite.addTestSuite(IgniteCacheLocalQueryCancelOrTimeoutSelfTest.class); - - // Distributed joins. - suite.addTestSuite(H2CompareBigQueryDistributedJoinsTest.class); - suite.addTestSuite(IgniteCacheDistributedJoinCollocatedAndNotTest.class); - suite.addTestSuite(IgniteCacheDistributedJoinCustomAffinityMapper.class); - suite.addTestSuite(IgniteCacheDistributedJoinNoIndexTest.class); - suite.addTestSuite(IgniteCacheDistributedJoinPartitionedAndReplicatedTest.class); - suite.addTestSuite(IgniteCacheDistributedJoinQueryConditionsTest.class); - suite.addTestSuite(IgniteCacheDistributedJoinTest.class); - suite.addTestSuite(IgniteSqlDistributedJoinSelfTest.class); - suite.addTestSuite(IgniteSqlQueryParallelismTest.class); - - // Other. - suite.addTestSuite(CacheIteratorScanQueryTest.class); - suite.addTestSuite(CacheQueryNewClientSelfTest.class); - suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class); - suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class); - suite.addTestSuite(IgniteCacheQueryCacheDestroySelfTest.class); - suite.addTestSuite(IgniteQueryDedicatedPoolTest.class); - suite.addTestSuite(IgniteSqlEntryCacheModeAgnosticTest.class); - suite.addTestSuite(QueryEntityCaseMismatchTest.class); - suite.addTestSuite(IgniteCacheDistributedPartitionQuerySelfTest.class); - suite.addTestSuite(IgniteCacheDistributedPartitionQueryNodeRestartsSelfTest.class); - suite.addTestSuite(IgniteCacheDistributedPartitionQueryConfigurationSelfTest.class); - suite.addTestSuite(IgniteSqlKeyValueFieldsTest.class); - suite.addTestSuite(IgniteSqlRoutingTest.class); - suite.addTestSuite(IgniteSqlNotNullConstraintTest.class); - suite.addTestSuite(LongIndexNameTest.class); - suite.addTestSuite(GridCacheQuerySqlFieldInlineSizeSelfTest.class); - suite.addTestSuite(IgniteSqlParameterizedQueryTest.class); - suite.addTestSuite(H2ConnectionLeaksSelfTest.class); - suite.addTestSuite(IgniteCheckClusterStateBeforeExecuteQueryTest.class); - suite.addTestSuite(OptimizedMarshallerIndexNameTest.class); - suite.addTestSuite(SqlSystemViewsSelfTest.class); - - suite.addTestSuite(GridIndexRebuildSelfTest.class); - - suite.addTestSuite(SqlTransactionCommandsWithMvccDisabledSelfTest.class); - - suite.addTestSuite(IgniteSqlDefaultValueTest.class); - suite.addTestSuite(IgniteDecimalSelfTest.class); - suite.addTestSuite(IgniteSQLColumnConstraintsTest.class); - suite.addTestSuite(IgniteTransactionSQLColumnConstraintTest.class); - - suite.addTestSuite(IgniteCachePartitionedAtomicColumnConstraintsTest.class); - suite.addTestSuite(IgniteCachePartitionedTransactionalColumnConstraintsTest.class); - suite.addTestSuite(IgniteCachePartitionedTransactionalSnapshotColumnConstraintTest.class); - suite.addTestSuite(IgniteCacheReplicatedAtomicColumnConstraintsTest.class); - suite.addTestSuite(IgniteCacheReplicatedTransactionalColumnConstraintsTest.class); - suite.addTestSuite(IgniteCacheReplicatedTransactionalSnapshotColumnConstraintTest.class); - - // H2 Rows on-heap cache - suite.addTestSuite(H2RowCacheSelfTest.class); - suite.addTestSuite(H2RowCachePageEvictionTest.class); - - // User operation SQL - suite.addTestSuite(SqlParserUserSelfTest.class); - suite.addTestSuite(SqlUserCommandSelfTest.class); - suite.addTestSuite(EncryptedSqlTableTest.class); - - suite.addTestSuite(ThreadLocalObjectPoolSelfTest.class); - suite.addTestSuite(H2StatementCacheSelfTest.class); - suite.addTestSuite(PreparedStatementExSelfTest.class); +// suite.addTestSuite(AffinityKeyNameAndValueFieldNameConflictTest.class); +// suite.addTestSuite(DmlInsideTransactionTest.class); +// suite.addTestSuite(ComplexPrimaryKeyUnwrapSelfTest.class); +// suite.addTestSuite(SqlNestedQuerySelfTest.class); +// suite.addTestSuite(ExplainSelfTest.class); +// suite.addTestSuite(RunningQueriesTest.class); +// +// suite.addTestSuite(ComplexPrimaryKeyUnwrapSelfTest.class); +// +// suite.addTestSuite(PartitionedSqlTest.class); +// suite.addTestSuite(ReplicatedSqlTest.class); +// +// suite.addTestSuite(SqlParserCreateIndexSelfTest.class); +// suite.addTestSuite(SqlParserDropIndexSelfTest.class); +// suite.addTestSuite(SqlParserTransactionalKeywordsSelfTest.class); +// suite.addTestSuite(SqlParserBulkLoadSelfTest.class); +// suite.addTestSuite(SqlParserSetStreamingSelfTest.class); +// +// suite.addTestSuite(SqlConnectorConfigurationValidationSelfTest.class); +// suite.addTestSuite(ClientConnectorConfigurationValidationSelfTest.class); +// +// suite.addTestSuite(SqlSchemaSelfTest.class); +// suite.addTestSuite(SqlIllegalSchemaSelfTest.class); +// suite.addTestSuite(MultipleStatementsSqlQuerySelfTest.class); +// +// suite.addTestSuite(BasicIndexTest.class); +// +// // Misc tests. +// suite.addTestSuite(QueryEntityValidationSelfTest.class); +// suite.addTestSuite(DuplicateKeyValueClassesSelfTest.class); +// suite.addTestSuite(GridCacheLazyQueryPartitionsReleaseTest.class); +// +// // Dynamic index create/drop tests. +// suite.addTestSuite(SchemaExchangeSelfTest.class); +// +// suite.addTestSuite(DynamicIndexServerCoordinatorBasicSelfTest.class); +// suite.addTestSuite(DynamicIndexServerBasicSelfTest.class); +// suite.addTestSuite(DynamicIndexServerNodeFilterCoordinatorBasicSelfTest.class); +// suite.addTestSuite(DynamicIndexServerNodeFIlterBasicSelfTest.class); +// suite.addTestSuite(DynamicIndexClientBasicSelfTest.class); +// +// // H2 tests. +// +// suite.addTestSuite(GridH2IndexingInMemSelfTest.class); +// suite.addTestSuite(GridH2IndexingOffheapSelfTest.class); +// +// // Parsing +// suite.addTestSuite(GridQueryParsingTest.class); +// suite.addTestSuite(IgniteCacheSqlQueryErrorSelfTest.class); +// +// // Config. +// suite.addTestSuite(IgniteCacheDuplicateEntityConfigurationSelfTest.class); +// suite.addTestSuite(IncorrectQueryEntityTest.class); +// suite.addTestSuite(IgniteDynamicSqlRestoreTest.class); +// +// // Queries tests. +// suite.addTestSuite(LazyQuerySelfTest.class); +// suite.addTestSuite(IgniteSqlSplitterSelfTest.class); +// suite.addTestSuite(SqlPushDownFunctionTest.class); +// suite.addTestSuite(IgniteSqlSegmentedIndexSelfTest.class); +// suite.addTestSuite(IgniteCachelessQueriesSelfTest.class); +// suite.addTestSuite(IgniteSqlSegmentedIndexMultiNodeSelfTest.class); +// suite.addTestSuite(IgniteSqlSchemaIndexingTest.class); +// suite.addTestSuite(GridCacheQueryIndexDisabledSelfTest.class); +// suite.addTestSuite(IgniteCacheQueryLoadSelfTest.class); +// suite.addTestSuite(IgniteCacheLocalQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheLocalAtomicQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheReplicatedQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheReplicatedQueryP2PDisabledSelfTest.class); +// suite.addTestSuite(IgniteCacheReplicatedQueryEvtsDisabledSelfTest.class); +// suite.addTestSuite(IgniteCachePartitionedQuerySelfTest.class); +// suite.addTestSuite(IgniteCachePartitionedSnapshotEnabledQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheAtomicQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheAtomicNearEnabledQuerySelfTest.class); +// suite.addTestSuite(IgniteCachePartitionedQueryP2PDisabledSelfTest.class); +// suite.addTestSuite(IgniteCachePartitionedQueryEvtsDisabledSelfTest.class); +// +// suite.addTestSuite(IgniteCacheUnionDuplicatesTest.class); +// suite.addTestSuite(IgniteCacheJoinPartitionedAndReplicatedCollocationTest.class); +// suite.addTestSuite(IgniteClientReconnectCacheQueriesFailoverTest.class); +// suite.addTestSuite(IgniteErrorOnRebalanceTest.class); +// suite.addTestSuite(CacheQueryBuildValueTest.class); +// suite.addTestSuite(CacheOffheapBatchIndexingMultiTypeTest.class); +// +// suite.addTestSuite(IgniteCacheQueryIndexSelfTest.class); +// suite.addTestSuite(IgniteCacheCollocatedQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheLargeResultSelfTest.class); +// suite.addTestSuite(GridCacheQueryInternalKeysSelfTest.class); +// suite.addTestSuite(H2ResultSetIteratorNullifyOnEndSelfTest.class); +// suite.addTestSuite(IgniteSqlBigIntegerKeyTest.class); +// suite.addTestSuite(IgniteCacheOffheapEvictQueryTest.class); +// suite.addTestSuite(IgniteCacheOffheapIndexScanTest.class); +// +// suite.addTestSuite(GridCacheCrossCacheQuerySelfTest.class); +// suite.addTestSuite(GridCacheQuerySerializationSelfTest.class); +// suite.addTestSuite(IgniteBinaryObjectFieldsQuerySelfTest.class); +// suite.addTestSuite(IgniteStableBaselineBinObjFieldsQuerySelfTest.class); +// suite.addTestSuite(IgniteBinaryWrappedObjectFieldsQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheQueryH2IndexingLeakTest.class); +// suite.addTestSuite(IgniteCacheQueryNoRebalanceSelfTest.class); +// suite.addTestSuite(GridCacheQueryTransformerSelfTest.class); +// suite.addTestSuite(CacheScanQueryFailoverTest.class); +// suite.addTestSuite(IgniteCachePrimitiveFieldsQuerySelfTest.class); +// +// suite.addTestSuite(IgniteCacheJoinQueryWithAffinityKeyTest.class); +// suite.addTestSuite(IgniteCacheJoinPartitionedAndReplicatedTest.class); +// suite.addTestSuite(IgniteCrossCachesJoinsQueryTest.class); +// +// suite.addTestSuite(IgniteCacheMultipleIndexedTypesTest.class); +// +// // DML. +// suite.addTestSuite(IgniteCacheMergeSqlQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheInsertSqlQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheUpdateSqlQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheDeleteSqlQuerySelfTest.class); +// suite.addTestSuite(IgniteSqlSkipReducerOnUpdateDmlSelfTest.class); +// suite.addTestSuite(IgniteSqlSkipReducerOnUpdateDmlFlagSelfTest.class); +// +// suite.addTestSuite(IgniteBinaryObjectQueryArgumentsTest.class); +// suite.addTestSuite(IgniteBinaryObjectLocalQueryArgumentsTest.class); +// +// suite.addTestSuite(IndexingSpiQuerySelfTest.class); +// suite.addTestSuite(IndexingSpiQueryTxSelfTest.class); +// +// suite.addTestSuite(IgniteCacheMultipleIndexedTypesTest.class); +// suite.addTestSuite(IgniteSqlQueryMinMaxTest.class); +// +// suite.addTestSuite(GridCircularQueueTest.class); +// suite.addTestSuite(IndexingSpiQueryWithH2IndexingSelfTest.class); +// +// // DDL. +// suite.addTestSuite(H2DynamicIndexTransactionalReplicatedSelfTest.class); +// suite.addTestSuite(H2DynamicIndexTransactionalPartitionedSelfTest.class); +// suite.addTestSuite(H2DynamicIndexTransactionalPartitionedNearSelfTest.class); +// suite.addTestSuite(H2DynamicIndexAtomicReplicatedSelfTest.class); +// suite.addTestSuite(H2DynamicIndexAtomicPartitionedSelfTest.class); +// suite.addTestSuite(H2DynamicIndexAtomicPartitionedNearSelfTest.class); +// suite.addTestSuite(H2DynamicTableSelfTest.class); +// suite.addTestSuite(H2DynamicColumnsClientBasicSelfTest.class); +// suite.addTestSuite(H2DynamicColumnsServerBasicSelfTest.class); +// suite.addTestSuite(H2DynamicColumnsServerCoordinatorBasicSelfTest.class); +// +// // DML+DDL. +// suite.addTestSuite(H2DynamicIndexingComplexClientAtomicPartitionedTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexClientAtomicPartitionedNoBackupsTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexClientAtomicReplicatedTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexClientTransactionalPartitionedTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexClientTransactionalPartitionedNoBackupsTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexClientTransactionalReplicatedTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexServerAtomicPartitionedTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexServerAtomicPartitionedNoBackupsTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexServerAtomicReplicatedTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexServerTransactionalPartitionedTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexServerTransactionalPartitionedNoBackupsTest.class); +// suite.addTestSuite(H2DynamicIndexingComplexServerTransactionalReplicatedTest.class); +// +// suite.addTestSuite(DdlTransactionSelfTest.class); +// +// // Fields queries. +// suite.addTestSuite(SqlFieldsQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheLocalFieldsQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheReplicatedFieldsQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheReplicatedFieldsQueryROSelfTest.class); +// suite.addTestSuite(IgniteCacheReplicatedFieldsQueryP2PEnabledSelfTest.class); +// suite.addTestSuite(IgniteCacheReplicatedFieldsQueryJoinNoPrimaryPartitionsSelfTest.class); +// suite.addTestSuite(IgniteCachePartitionedFieldsQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheAtomicFieldsQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheAtomicNearEnabledFieldsQuerySelfTest.class); +// suite.addTestSuite(IgniteCachePartitionedFieldsQueryP2PEnabledSelfTest.class); +// suite.addTestSuite(IgniteCacheFieldsQueryNoDataSelfTest.class); +// suite.addTestSuite(GridCacheQueryIndexingDisabledSelfTest.class); +// suite.addTestSuite(GridOrderedMessageCancelSelfTest.class); +// suite.addTestSuite(CacheQueryEvictDataLostTest.class); +// +// // Full text queries. +// suite.addTestSuite(GridCacheFullTextQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheFullTextQueryNodeJoiningSelfTest.class); +// +// // Ignite cache and H2 comparison. +// suite.addTestSuite(BaseH2CompareQueryTest.class); +// suite.addTestSuite(H2CompareBigQueryTest.class); +// suite.addTestSuite(H2CompareBigQueryDistributedJoinsTest.class); +// +// // Cache query metrics. +// suite.addTestSuite(CacheLocalQueryMetricsSelfTest.class); +// suite.addTestSuite(CachePartitionedQueryMetricsDistributedSelfTest.class); +// suite.addTestSuite(CachePartitionedQueryMetricsLocalSelfTest.class); +// suite.addTestSuite(CacheReplicatedQueryMetricsDistributedSelfTest.class); +// suite.addTestSuite(CacheReplicatedQueryMetricsLocalSelfTest.class); +// +// // Cache query metrics. +// suite.addTestSuite(CacheLocalQueryDetailMetricsSelfTest.class); +// suite.addTestSuite(CachePartitionedQueryDetailMetricsDistributedSelfTest.class); +// suite.addTestSuite(CachePartitionedQueryDetailMetricsLocalSelfTest.class); +// suite.addTestSuite(CacheReplicatedQueryDetailMetricsDistributedSelfTest.class); +// suite.addTestSuite(CacheReplicatedQueryDetailMetricsLocalSelfTest.class); +// +// // Unmarshalling query test. +// suite.addTestSuite(IgniteCacheP2pUnmarshallingQueryErrorTest.class); +// suite.addTestSuite(IgniteCacheNoClassQuerySelfTest.class); +// +// // Cancellation. +// suite.addTestSuite(IgniteCacheDistributedQueryCancelSelfTest.class); +// suite.addTestSuite(IgniteCacheLocalQueryCancelOrTimeoutSelfTest.class); +// +// // Distributed joins. +// suite.addTestSuite(H2CompareBigQueryDistributedJoinsTest.class); +// suite.addTestSuite(IgniteCacheDistributedJoinCollocatedAndNotTest.class); +// suite.addTestSuite(IgniteCacheDistributedJoinCustomAffinityMapper.class); +// suite.addTestSuite(IgniteCacheDistributedJoinNoIndexTest.class); +// suite.addTestSuite(IgniteCacheDistributedJoinPartitionedAndReplicatedTest.class); +// suite.addTestSuite(IgniteCacheDistributedJoinQueryConditionsTest.class); +// suite.addTestSuite(IgniteCacheDistributedJoinTest.class); +// suite.addTestSuite(IgniteSqlDistributedJoinSelfTest.class); +// suite.addTestSuite(IgniteSqlQueryParallelismTest.class); +// +// // Other. +// suite.addTestSuite(CacheIteratorScanQueryTest.class); +// suite.addTestSuite(CacheQueryNewClientSelfTest.class); +// suite.addTestSuite(CacheOffheapBatchIndexingSingleTypeTest.class); +// suite.addTestSuite(CacheSqlQueryValueCopySelfTest.class); +// suite.addTestSuite(IgniteCacheQueryCacheDestroySelfTest.class); +// suite.addTestSuite(IgniteQueryDedicatedPoolTest.class); +// suite.addTestSuite(IgniteSqlEntryCacheModeAgnosticTest.class); +// suite.addTestSuite(QueryEntityCaseMismatchTest.class); +// suite.addTestSuite(IgniteCacheDistributedPartitionQuerySelfTest.class); +// suite.addTestSuite(IgniteCacheDistributedPartitionQueryNodeRestartsSelfTest.class); +// suite.addTestSuite(IgniteCacheDistributedPartitionQueryConfigurationSelfTest.class); +// suite.addTestSuite(IgniteSqlKeyValueFieldsTest.class); +// suite.addTestSuite(IgniteSqlRoutingTest.class); +// suite.addTestSuite(IgniteSqlNotNullConstraintTest.class); +// suite.addTestSuite(LongIndexNameTest.class); +// suite.addTestSuite(GridCacheQuerySqlFieldInlineSizeSelfTest.class); +// suite.addTestSuite(IgniteSqlParameterizedQueryTest.class); +// suite.addTestSuite(H2ConnectionLeaksSelfTest.class); +// suite.addTestSuite(IgniteCheckClusterStateBeforeExecuteQueryTest.class); +// suite.addTestSuite(OptimizedMarshallerIndexNameTest.class); +// suite.addTestSuite(SqlSystemViewsSelfTest.class); +// +// suite.addTestSuite(GridIndexRebuildSelfTest.class); +// +// suite.addTestSuite(SqlTransactionCommandsWithMvccDisabledSelfTest.class); +// +// suite.addTestSuite(IgniteSqlDefaultValueTest.class); +// suite.addTestSuite(IgniteDecimalSelfTest.class); +// suite.addTestSuite(IgniteSQLColumnConstraintsTest.class); +// suite.addTestSuite(IgniteTransactionSQLColumnConstraintTest.class); +// +// suite.addTestSuite(IgniteCachePartitionedAtomicColumnConstraintsTest.class); +// suite.addTestSuite(IgniteCachePartitionedTransactionalColumnConstraintsTest.class); +// suite.addTestSuite(IgniteCachePartitionedTransactionalSnapshotColumnConstraintTest.class); +// suite.addTestSuite(IgniteCacheReplicatedAtomicColumnConstraintsTest.class); +// suite.addTestSuite(IgniteCacheReplicatedTransactionalColumnConstraintsTest.class); +// suite.addTestSuite(IgniteCacheReplicatedTransactionalSnapshotColumnConstraintTest.class); +// +// // H2 Rows on-heap cache +// suite.addTestSuite(H2RowCacheSelfTest.class); +// suite.addTestSuite(H2RowCachePageEvictionTest.class); +// +// // User operation SQL +// suite.addTestSuite(SqlParserUserSelfTest.class); +// suite.addTestSuite(SqlUserCommandSelfTest.class); +// suite.addTestSuite(EncryptedSqlTableTest.class); +// +// suite.addTestSuite(ThreadLocalObjectPoolSelfTest.class); +// suite.addTestSuite(H2StatementCacheSelfTest.class); +// suite.addTestSuite(PreparedStatementExSelfTest.class); // Partition loss. - suite.addTestSuite(IndexingCachePartitionLossPolicySelfTest.class); - - // GROUP_CONCAT - suite.addTestSuite(IgniteSqlGroupConcatCollocatedTest.class); - suite.addTestSuite(IgniteSqlGroupConcatNotCollocatedTest.class); - - // Binary - suite.addTestSuite(BinarySerializationQuerySelfTest.class); - suite.addTestSuite(BinarySerializationQueryWithReflectiveSerializerSelfTest.class); - suite.addTestSuite(IgniteCacheBinaryObjectsScanSelfTest.class); - suite.addTestSuite(IgniteCacheBinaryObjectsScanWithEventsSelfTest.class); - suite.addTestSuite(BigEntryQueryTest.class); - suite.addTestSuite(BinaryMetadataConcurrentUpdateWithIndexesTest.class); - - // Partition pruning. - suite.addTestSuite(InOperationExtractPartitionSelfTest.class); - suite.addTestSuite(AndOperationExtractPartitionSelfTest.class); - - suite.addTestSuite(GridCacheDynamicLoadOnClientTest.class); - suite.addTestSuite(GridCacheDynamicLoadOnClientPersistentTest.class); + for (int i = 0; i < 250; i++) + suite.addTestSuite(IndexingCachePartitionLossPolicySelfTest.class); + +// // GROUP_CONCAT +// suite.addTestSuite(IgniteSqlGroupConcatCollocatedTest.class); +// suite.addTestSuite(IgniteSqlGroupConcatNotCollocatedTest.class); +// +// // Binary +// suite.addTestSuite(BinarySerializationQuerySelfTest.class); +// suite.addTestSuite(BinarySerializationQueryWithReflectiveSerializerSelfTest.class); +// suite.addTestSuite(IgniteCacheBinaryObjectsScanSelfTest.class); +// suite.addTestSuite(IgniteCacheBinaryObjectsScanWithEventsSelfTest.class); +// suite.addTestSuite(BigEntryQueryTest.class); +// suite.addTestSuite(BinaryMetadataConcurrentUpdateWithIndexesTest.class); +// +// // Partition pruning. +// suite.addTestSuite(InOperationExtractPartitionSelfTest.class); +// suite.addTestSuite(AndOperationExtractPartitionSelfTest.class); +// +// suite.addTestSuite(GridCacheDynamicLoadOnClientTest.class); +// suite.addTestSuite(GridCacheDynamicLoadOnClientPersistentTest.class); return suite; } From 05cf83803143d2003bdc0c3d86cddd503e7ce2d2 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Sun, 16 Dec 2018 23:11:36 +0300 Subject: [PATCH 21/25] debug --- .../GridDhtPartitionsExchangeFuture.java | 2 ++ .../dht/topology/GridDhtLocalPartition.java | 18 ++++++++++++++++++ .../GridDhtPartitionTopologyImpl.java | 3 +++ ...gniteCachePartitionLossPolicySelfTest.java | 19 ++++++++++++++++++- .../junits/common/GridCommonAbstractTest.java | 2 +- 5 files changed, 42 insertions(+), 2 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 ce62993e42f66..e5278ed04ed0a 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 @@ -1813,6 +1813,8 @@ private void sendLocalPartitions(ClusterNode node) throws IgniteCheckedException true); } else { + log.info("single message"); + msg = cctx.exchange().createPartitionsSingleMessage(exchangeId(), false, true, diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java index c1f742eeaf7df..2767d6efe435a 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java @@ -80,6 +80,18 @@ * Key partition. */ public class GridDhtLocalPartition extends GridCacheConcurrentMapImpl implements Comparable, GridReservable { + public static final ConcurrentMap> MEGAMAP = new ConcurrentHashMap<>(); + + public void putState(GridDhtPartitionState oldState, GridDhtPartitionState newState) { + + if (grp.hasCache("default")) { + String nodeId = ctx.localNodeId().toString(); + String nodeIdx = nodeId.substring(nodeId.length() - 3); + + MEGAMAP.computeIfAbsent(nodeIdx, (v) -> new ConcurrentHashMap<>()).computeIfAbsent(id, v -> new StringBuilder(oldState.toString())).append(" -> ").append(newState); + } + } + /** */ private static final GridCacheMapEntryFactory ENTRY_FACTORY = GridDhtCacheEntry::new; @@ -559,6 +571,8 @@ private boolean casState(long state, GridDhtPartitionState toState) { boolean update = this.state.compareAndSet(state, setPartState(state, toState)); if (update) { + putState(prevState, toState); + try { ctx.wal().log(new PartitionMetaStateRecord(grp.groupId(), id, toState, updateCounter())); } @@ -582,6 +596,8 @@ private boolean casState(long state, GridDhtPartitionState toState) { boolean update = this.state.compareAndSet(state, setPartState(state, toState)); if (update) { + putState(prevState, toState); + if (log.isDebugEnabled()) log.debug("Partition changed state [grp=" + grp.cacheOrGroupName() + ", p=" + id + ", prev=" + prevState + ", to=" + toState + "]"); @@ -665,6 +681,8 @@ public IgniteInternalFuture rent(boolean updateSeq) { delayedRenting = true; if (getReservations(state0) == 0 && casState(state0, RENTING)) { + log.info(id() + " - " + RENTING + " clear async"); + delayedRenting = false; // Evict asynchronously, as the 'rent' method may be called diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index f87ba903089ab..62a13a65b0376 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -1057,6 +1057,9 @@ else if (loc != null && state == RENTING && !showRenting) { if (part == null) continue; + if (part.state() == RENTING) + log.info(">xxx> observing RENTING " + part.id()); + map.put(i, part.state()); } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java index 56068f1a33ed4..a1e92a8b00752 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.Set; import java.util.TreeSet; +import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Semaphore; @@ -51,6 +52,7 @@ import org.apache.ignite.internal.processors.cache.distributed.CacheBlockOnReadAbstractTest.RunnableX; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsAbstractMessage; import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsFullMessage; +import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition; import org.apache.ignite.internal.util.typedef.F; import org.apache.ignite.internal.util.typedef.G; import org.apache.ignite.internal.util.typedef.P1; @@ -248,6 +250,8 @@ protected CacheConfiguration cacheConfiguration() { */ @Test public void testReadWriteSafeNoResendDelay() throws Exception { + GridDhtLocalPartition.MEGAMAP.clear(); + withoutResendDelay(this::checkReadWriteSafe); } @@ -618,7 +622,11 @@ private void checkLostPartition(boolean canWrite, boolean safe, TopologyChanger ignite(4).resetLostPartitions(singletonList(DEFAULT_CACHE_NAME)); - awaitPartitionMapExchange(true, true, null); + try { + awaitPartitionMapExchange(true, true, null); + } finally { + printMegaMap(); + } for (Ignite ig : G.allGrids()) { IgniteCache cache = ig.cache(DEFAULT_CACHE_NAME); @@ -645,6 +653,15 @@ private void checkLostPartition(boolean canWrite, boolean safe, TopologyChanger } } + private void printMegaMap() { + for (Map.Entry> e : GridDhtLocalPartition.MEGAMAP.entrySet()) { + log.info(">xxx> -------- " + e.getKey()); + + for (Map.Entry e0 : e.getValue().entrySet()) + log.info(">xxx>\t" + e0.getKey() + ": " + e0.getValue()); + } + } + /** * @param client Client flag. * @param canWrite Can write flag. diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java index 48a3b1802db95..585cda1d41943 100755 --- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java @@ -739,7 +739,7 @@ protected void awaitPartitionMapExchange( if (System.currentTimeMillis() - start > timeout) { U.dumpThreads(log); - if (printPartState) +// if (printPartState) printPartitionState(c); throw new IgniteException("Timeout of waiting for topology map update [" + From 069e2b6060bc343b00204a490a43c58da2761a84 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Mon, 17 Dec 2018 11:20:41 +0300 Subject: [PATCH 22/25] wip --- .../GridDhtPartitionTopologyImpl.java | 32 +++++++++---------- ...gniteCachePartitionLossPolicySelfTest.java | 14 ++++---- ...yWithJsr310Java8DateTimeApiFieldsTest.java | 3 ++ 3 files changed, 26 insertions(+), 23 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 62a13a65b0376..12dce63fc882c 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2156,10 +2156,10 @@ else if (plc != PartitionLossPolicy.IGNORE) { if (e0.getValue() != LOST) continue; - AffinityAssignment assignment = grp.affinity().cachedAffinity(resTopVer); - - if (!assignment.idealAssignment().get(e0.getKey()).contains(ctx.discovery().node(e.getKey()))) - continue; +// AffinityAssignment assignment = grp.affinity().cachedAffinity(resTopVer); +// +// if (!assignment.idealAssignment().get(e0.getKey()).contains(ctx.discovery().node(e.getKey()))) +// continue; e0.setValue(OWNING); @@ -2289,18 +2289,18 @@ else if (plc != PartitionLossPolicy.IGNORE) { } } - if (lostParts != null) { - for (Map.Entry e : node2part.entrySet()) { - if (e.getKey().equals(ctx.localNodeId())) - continue; - - for (Integer part : lostParts) { - GridDhtPartitionState state = e.getValue().get(part); - if (state != null && state.active()) - e.getValue().put(part, LOST); - } - } - } +// if (lostParts != null) { +// for (Map.Entry e : node2part.entrySet()) { +// if (e.getKey().equals(ctx.localNodeId())) +// continue; +// +// for (Integer part : lostParts) { +// GridDhtPartitionState state = e.getValue().get(part); +// if (state != null && state.active()) +// e.getValue().put(part, LOST); +// } +// } +// } node2part = new GridDhtPartitionFullMap(node2part, updateSeq.incrementAndGet()); } finally { diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java index a1e92a8b00752..223d4463a9f5b 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java @@ -237,13 +237,13 @@ protected CacheConfiguration cacheConfiguration() { // checkLostPartition(false, false, killSingleNode); // } // -// /** -// * @throws Exception if failed. -// */ -// @Test -// public void testReadWriteSafe() throws Exception { -// checkReadWriteSafe(); -// } + /** + * @throws Exception if failed. + */ + @Test + public void testReadWriteSafe() throws Exception { + checkReadWriteSafe(); + } /** * @throws Exception if failed. diff --git a/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.java b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.java index 1854978adfd95..6a144fcfc6089 100644 --- a/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.java +++ b/modules/indexing/src/test/java8/org/apache/ignite/internal/processors/query/h2/CacheQueryEntityWithJsr310Java8DateTimeApiFieldsTest.java @@ -31,6 +31,9 @@ import org.apache.ignite.cache.query.SqlFieldsQuery; import org.apache.ignite.cache.query.annotations.QuerySqlField; import org.apache.ignite.configuration.CacheConfiguration; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; /** * Tests queries against entities with JSR-310 Java 8 Date and Time API fields. From a747894624e988f5e7d4297dbdda06c917d07300 Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Mon, 17 Dec 2018 17:11:15 +0300 Subject: [PATCH 23/25] Eviction race fix. --- .../GridDhtPartitionTopologyImpl.java | 4 +-- ...gniteCachePartitionLossPolicySelfTest.java | 30 +++++++++---------- 2 files changed, 17 insertions(+), 17 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 12dce63fc882c..60b19ba9e0df4 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2386,7 +2386,7 @@ private boolean checkEvictions(long updateSeq, AffinityAssignment aff) { if (nodeIds.containsAll(F.nodeIds(affNodes))) { GridDhtPartitionState state0 = part.state(); - IgniteInternalFuture rentFut = part.rent(false); + IgniteInternalFuture rentFut = part.rent(true); rentingFutures.add(rentFut); @@ -2417,7 +2417,7 @@ private boolean checkEvictions(long updateSeq, AffinityAssignment aff) { if (locId.equals(n.id())) { GridDhtPartitionState state0 = part.state(); - IgniteInternalFuture rentFut = part.rent(false); + IgniteInternalFuture rentFut = part.rent(true); rentingFutures.add(rentFut); diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java index 223d4463a9f5b..cb0eede075282 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java @@ -159,14 +159,14 @@ protected CacheConfiguration cacheConfiguration() { isPersistenceEnabled = false; } -// /** {@inheritDoc} */ -// @Override protected void afterTest() throws Exception { -// stopAllGrids(); -// -// cleanPersistenceDir(); -// -// super.afterTest(); -// } + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + cleanPersistenceDir(); + + super.afterTest(); + } // // /** // * @throws Exception if failed. @@ -237,13 +237,13 @@ protected CacheConfiguration cacheConfiguration() { // checkLostPartition(false, false, killSingleNode); // } // - /** - * @throws Exception if failed. - */ - @Test - public void testReadWriteSafe() throws Exception { - checkReadWriteSafe(); - } +// /** +// * @throws Exception if failed. +// */ +// @Test +// public void testReadWriteSafe() throws Exception { +// checkReadWriteSafe(); +// } /** * @throws Exception if failed. From f1d0138f640872226f2afb4b70d4edafe8634f5f Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Mon, 17 Dec 2018 18:06:34 +0300 Subject: [PATCH 24/25] wip --- .../dht/topology/GridDhtPartitionTopologyImpl.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 60b19ba9e0df4..14b3366717065 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2173,11 +2173,11 @@ else if (plc != PartitionLossPolicy.IGNORE) { long updateCntr = locPart.updateCounter(); - if (hasOwner.contains(locPart.id())) { +// if (hasOwner.contains(locPart.id())) { // Set update counters to 0, for full rebalance. locPart.updateCounter(updateCntr, -updateCntr); locPart.initialUpdateCounter(0); - } +// } } } } From 7a22957b344119b7f15f31ed0c13b71cc9201ace Mon Sep 17 00:00:00 2001 From: pereslegin-pa Date: Mon, 17 Dec 2018 20:49:39 +0300 Subject: [PATCH 25/25] IGNITE-10058 wipwipwip --- .../cache/IgniteCacheOffheapManagerImpl.java | 2 ++ .../dht/topology/GridDhtLocalPartition.java | 14 ++++++++++++-- .../dht/topology/GridDhtPartitionTopologyImpl.java | 13 ++++++++++--- .../IgniteCachePartitionLossPolicySelfTest.java | 8 ++++++++ 4 files changed, 32 insertions(+), 5 deletions(-) diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java index 5997f57b8c2f5..5c4a8e5d2090d 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapManagerImpl.java @@ -1551,6 +1551,8 @@ void decrementSize(int cacheId) { /** {@inheritDoc} */ @Override public void updateSize(int cacheId, long delta) { +// U.dumpStack("update size " + ctx.cache().cacheDescriptor(cacheId).cacheName() + "< delta = " + delta); + storageSize.addAndGet(delta); if (grp.sharedGroup()) { diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java index 2767d6efe435a..fb09c02b33da5 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtLocalPartition.java @@ -872,7 +872,7 @@ public void awaitDestroy() { if (state() != EVICTED) return; - final long timeout = 10_000; + final long timeout = 30_000; for (;;) { try { @@ -1025,6 +1025,8 @@ public long updateCounter() { * @param val Update counter value. */ public void updateCounter(long val) { + log.info(" update cntr " + id() + ", cntr=" + val); + store.updateCounter(val); } @@ -1049,7 +1051,12 @@ public void initialUpdateCounter(long val) { * @return Update counter value before update. */ public long getAndIncrementUpdateCounter(long delta) { - return store.getAndIncrementUpdateCounter(delta); + long cntr = store.getAndIncrementUpdateCounter(delta); + +// if (cntr == 0) + log.info(" update cntr " + id() + ", cntr=" + cntr); + + return cntr; } /** @@ -1059,6 +1066,9 @@ public long getAndIncrementUpdateCounter(long delta) { * @param delta Delta. */ public void updateCounter(long start, long delta) { +// if (start == 0) +// log.info(" update cntr " + id() + ", start=" + start + ", delta=" + delta); + store.updateCounter(start, delta); } diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java index 14b3366717065..db7268ad0c9e9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/topology/GridDhtPartitionTopologyImpl.java @@ -2173,11 +2173,18 @@ else if (plc != PartitionLossPolicy.IGNORE) { long updateCntr = locPart.updateCounter(); -// if (hasOwner.contains(locPart.id())) { + if (hasOwner.contains(locPart.id())) { // Set update counters to 0, for full rebalance. + log.info(">xxx> " + grp.cacheOrGroupName() + " there is owner for " + locPart.id() + " reset counter 0"); + locPart.updateCounter(updateCntr, -updateCntr); - locPart.initialUpdateCounter(0); -// } + } + else + log.info(">xxx> " + grp.cacheOrGroupName() + " there is no owner for " + locPart.id() + ", keeping update cntr = " + updateCntr); + + assert locPart.initialUpdateCounter() == 0 : locPart.initialUpdateCounter(); + + locPart.initialUpdateCounter(0); } } } diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java index cb0eede075282..9b5bc6bf84cac 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCachePartitionLossPolicySelfTest.java @@ -620,8 +620,12 @@ private void checkLostPartition(boolean canWrite, boolean safe, TopologyChanger } } + log.info(">xxx> reset lost parts"); + ignite(4).resetLostPartitions(singletonList(DEFAULT_CACHE_NAME)); + log.info(">xxx> reset lost parts finished"); + try { awaitPartitionMapExchange(true, true, null); } finally { @@ -739,6 +743,8 @@ private void verifyCacheOps(boolean canWrite, boolean safe, Ignite ig) { // Check write. for (int i = 0; i < parts; i++) { try { + log.info("put " + i); + cache.put(i, i); assertTrue("Write in read-only mode should be forbidden: " + i, canWrite); @@ -747,6 +753,8 @@ private void verifyCacheOps(boolean canWrite, boolean safe, Ignite ig) { assertFalse("Writing to a lost partition should have failed: " + i, safe); } catch (CacheException e) { + log.info("(ex) put " + i); + if (canWrite) { assertTrue("Write exception should only be triggered in safe mode: " + e, safe); assertTrue("Write exception should only be triggered for a lost partition: " + e,