ignite-5075
Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/664b7492 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/664b7492 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/664b7492 Branch: refs/heads/ignite-5075 Commit: 664b7492a7eb4c37847d42345abdd575500062e8 Parents: d282808 Author: sboikov <semen.boi...@inria.fr> Authored: Wed May 17 01:07:21 2017 +0300 Committer: sboikov <semen.boi...@inria.fr> Committed: Wed May 17 01:07:21 2017 +0300 ---------------------------------------------------------------------- .../processors/cache/ClusterCachesInfo.java | 111 ++++++----- .../CacheDiscoveryDataConcurrentJoinTest.java | 199 +++++++++++++++++++ .../testsuites/IgniteCacheTestSuite4.java | 14 +- 3 files changed, 260 insertions(+), 64 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/664b7492/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java index da36470..7d1e1a6 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/ClusterCachesInfo.java @@ -98,6 +98,8 @@ class ClusterCachesInfo { */ void onStart(CacheJoinNodeDiscoveryData joinDiscoData) { this.joinDiscoData = joinDiscoData; + + processJoiningNode(joinDiscoData, ctx.localNodeId()); } /** @@ -528,52 +530,6 @@ class ClusterCachesInfo { */ void onDiscoveryEvent(int type, ClusterNode node, AffinityTopologyVersion topVer) { if (type == EVT_NODE_JOINED && !ctx.isDaemon()) { - if (node.id().equals(ctx.discovery().localNode().id())) { - if (gridData == null) { // First node starts. - assert registeredCaches.isEmpty(); - assert registeredTemplates.isEmpty(); - assert joinDiscoData != null || !ctx.state().active(); - } - - assert locJoinStartCaches == null; - - locJoinStartCaches = new ArrayList<>(); - - if (!disconnectedState() && joinDiscoData != null) { - processJoiningNode(joinDiscoData, node.id()); - - for (DynamicCacheDescriptor desc : registeredCaches.values()) { - CacheConfiguration cfg = desc.cacheConfiguration(); - - CacheJoinNodeDiscoveryData.CacheInfo locCfg = joinDiscoData.caches().get(cfg.getName()); - - NearCacheConfiguration nearCfg = null; - - if (locCfg != null) { - nearCfg = locCfg.config().getNearConfiguration(); - - DynamicCacheDescriptor desc0 = new DynamicCacheDescriptor(ctx, - locCfg.config(), - desc.cacheType(), - desc.template(), - desc.deploymentId(), - desc.schema()); - - desc0.startTopologyVersion(desc.startTopologyVersion()); - desc0.receivedFromStartVersion(desc.receivedFromStartVersion()); - desc0.clientCacheStartVersion(desc.clientCacheStartVersion()); - desc0.receivedFrom(desc.receivedFrom()); - desc0.staticallyConfigured(desc.staticallyConfigured()); - - desc = desc0; - } - - if (locCfg != null || joinDiscoData.startCaches() || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter())) - locJoinStartCaches.add(new T2<>(desc, nearCfg)); - } - } - } - for (DynamicCacheDescriptor desc : registeredCaches.values()) { if (node.id().equals(desc.receivedFrom())) desc.receivedFromStartVersion(topVer); @@ -583,6 +539,14 @@ class ClusterCachesInfo { if (node.id().equals(desc.receivedFrom())) desc.receivedFromStartVersion(topVer); } + + if (node.id().equals(ctx.discovery().localNode().id())) { + if (gridData == null) { // First node starts. + assert joinDiscoData != null || !ctx.state().active(); + + initStartCachesForLocalJoin(true); + } + } } } @@ -660,9 +624,7 @@ class ClusterCachesInfo { desc.receivedFrom(cacheData.receivedFrom()); desc.staticallyConfigured(cacheData.staticallyConfigured()); - DynamicCacheDescriptor old = registeredTemplates.put(cacheData.cacheConfiguration().getName(), desc); - - assert old == null; + registeredTemplates.put(cacheData.cacheConfiguration().getName(), desc); } for (CacheData cacheData : cachesData.caches().values()) { @@ -679,9 +641,7 @@ class ClusterCachesInfo { desc.receivedFrom(cacheData.receivedFrom()); desc.staticallyConfigured(cacheData.staticallyConfigured()); - DynamicCacheDescriptor old = registeredCaches.put(cacheData.cacheConfiguration().getName(), desc); - - assert old == null; + registeredCaches.put(cacheData.cacheConfiguration().getName(), desc); ctx.discovery().setCacheFilter( cfg.getName(), @@ -700,6 +660,53 @@ class ClusterCachesInfo { } gridData = cachesData; + + if (!disconnectedState()) + initStartCachesForLocalJoin(false); + } + + /** + * @param firstNode {@code True} if first node in cluster starts. + */ + private void initStartCachesForLocalJoin(boolean firstNode) { + assert locJoinStartCaches == null; + + locJoinStartCaches = new ArrayList<>(); + + if (joinDiscoData != null) { + for (DynamicCacheDescriptor desc : registeredCaches.values()) { + if (firstNode && !joinDiscoData.caches().containsKey(desc.cacheName())) + continue; + + CacheConfiguration cfg = desc.cacheConfiguration(); + + CacheJoinNodeDiscoveryData.CacheInfo locCfg = joinDiscoData.caches().get(cfg.getName()); + + NearCacheConfiguration nearCfg = null; + + if (locCfg != null) { + nearCfg = locCfg.config().getNearConfiguration(); + + DynamicCacheDescriptor desc0 = new DynamicCacheDescriptor(ctx, + locCfg.config(), + desc.cacheType(), + desc.template(), + desc.deploymentId(), + desc.schema()); + + desc0.startTopologyVersion(desc.startTopologyVersion()); + desc0.receivedFromStartVersion(desc.receivedFromStartVersion()); + desc0.clientCacheStartVersion(desc.clientCacheStartVersion()); + desc0.receivedFrom(desc.receivedFrom()); + desc0.staticallyConfigured(desc.staticallyConfigured()); + + desc = desc0; + } + + if (locCfg != null || joinDiscoData.startCaches() || CU.affinityNode(ctx.discovery().localNode(), cfg.getNodeFilter())) + locJoinStartCaches.add(new T2<>(desc, nearCfg)); + } + } } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/664b7492/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDiscoveryDataConcurrentJoinTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDiscoveryDataConcurrentJoinTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDiscoveryDataConcurrentJoinTest.java new file mode 100644 index 0000000..2f11c86 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/CacheDiscoveryDataConcurrentJoinTest.java @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.ignite.internal.processors.cache.distributed; + +import org.apache.ignite.Ignite; +import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction; +import org.apache.ignite.cluster.ClusterNode; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.util.GridAtomicInteger; +import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi; +import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder; +import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage; +import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryJoinRequestMessage; +import org.apache.ignite.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; +import java.util.concurrent.Callable; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; + +/** + * + */ +@SuppressWarnings("unchecked") +public class CacheDiscoveryDataConcurrentJoinTest extends GridCommonAbstractTest { + /** */ + private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true); + + /** Iteration. */ + private static final int ITERATIONS = 3; + + /** */ + private boolean client; + + /** */ + private ThreadLocal<Integer> staticCaches = new ThreadLocal<>(); + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpDiscoverySpi testSpi = new TcpDiscoverySpi() { + /** */ + private boolean delay = true; + + @Override protected void startMessageProcess(TcpDiscoveryAbstractMessage msg) { + if (getTestIgniteInstanceName(0).equals(ignite.name())) { + if (msg instanceof TcpDiscoveryJoinRequestMessage) { + TcpDiscoveryJoinRequestMessage msg0 = (TcpDiscoveryJoinRequestMessage)msg; + + if (delay) { + log.info("Delay join processing: " + msg0); + + delay = false; + + doSleep(5000); + } + } + } + + super.startMessageProcess(msg); + } + }; + + testSpi.setIpFinder(ipFinder); + testSpi.setJoinTimeout(60_000); + + cfg.setDiscoverySpi(testSpi); + + cfg.setClientMode(client); + + Integer caches = staticCaches.get(); + + if (caches != null) { + cfg.setCacheConfiguration(cacheConfigurations(caches).toArray(new CacheConfiguration[caches])); + + staticCaches.remove(); + } + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected long getTestTimeout() { + return 10 * 60 * 1000L; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentJoin() throws Exception { + for (int iter = 0; iter < ITERATIONS; iter++) { + log.info("Iteration: " + iter); + + final int NODES = 6; + final int MAX_CACHES = 10; + + final GridAtomicInteger caches = new GridAtomicInteger(); + + startGrid(0); + + final AtomicInteger idx = new AtomicInteger(1); + + GridTestUtils.runMultiThreaded(new Callable<Void>() { + @Override public Void call() throws Exception { + int c = ThreadLocalRandom.current().nextInt(MAX_CACHES) + 1; + + staticCaches.set(c); + + startGrid(idx.getAndIncrement()); + + caches.setIfGreater(c); + + return null; + } + }, NODES - 1, "start-node"); + + assertTrue(caches.get() > 0); + + for (int i = 0; i < NODES; i++) { + Ignite node = ignite(i); + + for (int c = 0; c < caches.get(); c++) { + Collection<ClusterNode> nodes = node.cluster().forCacheNodes("cache-" + c).nodes(); + + assertEquals(NODES, nodes.size()); + + checkCache(node, "cache-" + c); + } + } + + stopAllGrids(); + } + } + + /** + * @param caches Number of caches. + * @return Cache configurations. + */ + private Collection<CacheConfiguration> cacheConfigurations(int caches) { + List<CacheConfiguration> ccfgs = new ArrayList<>(); + + for (int i = 0; i < caches; i++) + ccfgs.add(cacheConfiguration("cache-" + i)); + + return ccfgs; + } + + /** + * @param cacheName Cache name. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(String cacheName) { + CacheConfiguration ccfg = new CacheConfiguration(cacheName); + + ccfg.setName(cacheName); + ccfg.setAtomicityMode(TRANSACTIONAL); + ccfg.setAffinity(new RendezvousAffinityFunction(false, 16)); + + return ccfg; + } + /** + * @param node Node. + * @param cacheName Cache name. + */ + private void checkCache(Ignite node, final String cacheName) { + assertNotNull(((IgniteKernal)node).context().cache().cache(cacheName)); + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/664b7492/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java index 64ee3fb..4a5d2d8 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite4.java @@ -82,18 +82,7 @@ import org.apache.ignite.internal.processors.cache.IgniteStartCacheInTransaction import org.apache.ignite.internal.processors.cache.IgniteStartCacheInTransactionSelfTest; import org.apache.ignite.internal.processors.cache.IgniteSystemCacheOnClientTest; import org.apache.ignite.internal.processors.cache.MarshallerCacheJobRunNodeRestartTest; -import org.apache.ignite.internal.processors.cache.distributed.CacheAffinityEarlyTest; -import org.apache.ignite.internal.processors.cache.distributed.CacheAtomicPrimarySyncBackPressureTest; -import org.apache.ignite.internal.processors.cache.distributed.CacheGetFutureHangsSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.CacheNoValueClassOnServerNodeTest; -import org.apache.ignite.internal.processors.cache.distributed.CacheStartOnJoinTest; -import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheCreatePutMultiNodeSelfTest; -import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheCreatePutTest; -import org.apache.ignite.internal.processors.cache.distributed.IgniteCachePrimarySyncTest; -import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheReadFromBackupTest; -import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheSingleGetMessageTest; -import org.apache.ignite.internal.processors.cache.distributed.IgniteTxCachePrimarySyncTest; -import org.apache.ignite.internal.processors.cache.distributed.IgniteTxCacheWriteSynchronizationModesMultithreadedTest; +import org.apache.ignite.internal.processors.cache.distributed.*; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheDhtTxPreloadSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheLockFailoverSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheMultiTxLockSelfTest; @@ -224,6 +213,7 @@ public class IgniteCacheTestSuite4 extends TestSuite { suite.addTestSuite(IgniteCacheCreatePutMultiNodeSelfTest.class); suite.addTestSuite(IgniteCacheCreatePutTest.class); suite.addTestSuite(CacheStartOnJoinTest.class); + suite.addTestSuite(CacheDiscoveryDataConcurrentJoinTest.class); suite.addTestSuite(GridCacheTxLoadFromStoreOnLockSelfTest.class);