You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2017/05/17 10:32:50 UTC

[11/17] ignite git commit: ignite-5075

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 <se...@inria.fr>
Authored: Wed May 17 01:07:21 2017 +0300
Committer: sboikov <se...@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);