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 2015/06/24 13:57:08 UTC

[42/50] incubator-ignite git commit: ignite-1045 properly handle flag DynamicCacheChangeRequest.clientStartOnly

ignite-1045 properly handle flag DynamicCacheChangeRequest.clientStartOnly


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/a6cda33a
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/a6cda33a
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/a6cda33a

Branch: refs/heads/ignite-648
Commit: a6cda33a8b76925d09c21262e88467421025fa77
Parents: c7ba154
Author: sboikov <sb...@gridgain.com>
Authored: Tue Jun 23 16:47:13 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Tue Jun 23 17:29:49 2015 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        |  46 +-
 ...eDynamicCacheStartNoExchangeTimeoutTest.java | 466 +++++++++++++++++++
 .../distributed/IgniteCache150ClientsTest.java  | 189 ++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |   1 +
 .../testsuites/IgniteCacheTestSuite4.java       |   1 +
 .../testsuites/IgniteClientTestSuite.java       |  38 ++
 6 files changed, 733 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
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 7c780b0..38a0d55 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
@@ -153,6 +153,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     /** Skip preload flag. */
     private boolean skipPreload;
 
+    /** */
+    private boolean clientOnlyExchange;
+
     /**
      * Dummy future created to trigger reassignments if partition
      * topology changed while preloading.
@@ -524,6 +527,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                         if (exchId.isLeft())
                             cctx.mvcc().removeExplicitNodeLocks(exchId.nodeId(), exchId.topologyVersion());
 
+                        rmtIds = Collections.emptyList();
+                        rmtNodes = Collections.emptyList();
+
                         onDone(exchId.topologyVersion());
 
                         skipPreload = cctx.kernalContext().clientNode();
@@ -532,8 +538,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                     }
                 }
 
-                if (cctx.kernalContext().clientNode()) {
-                    skipPreload = true;
+                clientOnlyExchange = clientNodeEvt || cctx.kernalContext().clientNode();
+
+                if (clientOnlyExchange) {
+                    skipPreload = cctx.kernalContext().clientNode();
 
                     for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
                         if (cacheCtx.isLocal())
@@ -551,23 +559,45 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                         initTopology(cacheCtx);
                     }
 
-                    if (oldestNode.get() != null) {
+                    if (oldest != null) {
                         rmtNodes = new ConcurrentLinkedQueue<>(CU.aliveRemoteServerNodesWithCaches(cctx,
                             exchId.topologyVersion()));
 
                         rmtIds = Collections.unmodifiableSet(new HashSet<>(F.nodeIds(rmtNodes)));
 
-                        ready.set(true);
-
                         initFut.onDone(true);
 
                         if (log.isDebugEnabled())
                             log.debug("Initialized future: " + this);
 
-                        sendPartitions();
+                        if (cctx.localNode().equals(oldest)) {
+                            for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                                boolean updateTop = !cacheCtx.isLocal() &&
+                                    exchId.topologyVersion().equals(cacheCtx.startTopologyVersion());
+
+                                if (updateTop) {
+                                    for (GridClientPartitionTopology top : cctx.exchange().clientTopologies()) {
+                                        if (top.cacheId() == cacheCtx.cacheId()) {
+                                            cacheCtx.topology().update(exchId, top.partitionMap(true));
+
+                                            break;
+                                        }
+                                    }
+
+                                }
+                            }
+
+                            onDone(exchId.topologyVersion());
+                        }
+                        else
+                            sendPartitions();
                     }
-                    else
+                    else {
+                        rmtIds = Collections.emptyList();
+                        rmtNodes = Collections.emptyList();
+
                         onDone(exchId.topologyVersion());
+                    }
 
                     return;
                 }
@@ -885,7 +915,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      */
     private void sendLocalPartitions(ClusterNode node, @Nullable GridDhtPartitionExchangeId id) throws IgniteCheckedException {
         GridDhtPartitionsSingleMessage m = new GridDhtPartitionsSingleMessage(id,
-            cctx.kernalContext().clientNode(),
+            clientOnlyExchange,
             cctx.versions().last());
 
         for (GridCacheContext cacheCtx : cctx.cacheContexts()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
new file mode 100644
index 0000000..5011e5f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartNoExchangeTimeoutTest.java
@@ -0,0 +1,466 @@
+/*
+ * 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;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.affinity.fair.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.extensions.communication.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+
+/**
+ *
+ */
+public class IgniteDynamicCacheStartNoExchangeTimeoutTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES = 4;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi) cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setCommunicationSpi(new TestCommunicationSpi());
+
+        if (gridName.equals(getTestGridName(NODES - 1)))
+            cfg.setClientMode(true);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        super.beforeTest();
+
+        startGrids(NODES);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultinodeCacheStart() throws Exception {
+        for (int i = 0; i < 10; i++) {
+            log.info("Iteration: " + i);
+
+            final String name = "cache-" + i;
+
+            final AtomicInteger idx = new AtomicInteger();
+
+            GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Ignite ignite = ignite(idx.getAndIncrement());
+
+                    CacheConfiguration ccfg = new CacheConfiguration();
+
+                    ccfg.setName(name);
+
+                    assertNotNull(ignite.getOrCreateCache(ccfg));
+
+                    return null;
+                }
+            }, 2, "create-cache").get(15_000);
+
+            awaitPartitionMapExchange();
+
+            checkCache(name);
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestNotAffinityNode1() throws Exception {
+        for (CacheConfiguration ccfg : cacheConfigurations())
+            oldestNotAffinityNode1(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void oldestNotAffinityNode1(final CacheConfiguration ccfg) throws Exception {
+        log.info("Test with cache: " + ccfg.getName());
+
+        IgniteEx ignite = grid(0);
+
+        assertEquals(1L, ignite.localNode().order());
+
+        ccfg.setNodeFilter(new TestFilterExcludeOldest());
+
+        assertNotNull(ignite.getOrCreateCache(ccfg));
+
+        awaitPartitionMapExchange();
+
+        checkCache(ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestNotAffinityNode2() throws Exception {
+        for (CacheConfiguration ccfg : cacheConfigurations())
+            oldestNotAffinityNode2(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void oldestNotAffinityNode2(final CacheConfiguration ccfg) throws Exception {
+        log.info("Test with cache: " + ccfg.getName());
+
+        IgniteEx ignite0 = grid(0);
+        IgniteEx ignite1 = grid(1);
+
+        assertEquals(1L, ignite0.localNode().order());
+
+        ccfg.setNodeFilter(new TestFilterExcludeOldest());
+
+        assertNotNull(ignite1.getOrCreateCache(ccfg));
+
+        assertNotNull(ignite0.cache(ccfg.getName()));
+
+        awaitPartitionMapExchange();
+
+        checkCache(ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNotAffinityNode1() throws Exception {
+        for (CacheConfiguration ccfg : cacheConfigurations())
+            notAffinityNode1(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void notAffinityNode1(final CacheConfiguration ccfg) throws Exception {
+        log.info("Test with cache: " + ccfg.getName());
+
+        IgniteEx ignite = grid(1);
+
+        assertEquals(2, ignite.localNode().order());
+
+        ccfg.setNodeFilter(new TestFilterExcludeNode(2));
+
+        assertNotNull(ignite.getOrCreateCache(ccfg));
+
+        awaitPartitionMapExchange();
+
+        checkCache(ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testNotAffinityNode2() throws Exception {
+        for (CacheConfiguration ccfg : cacheConfigurations())
+            notAffinityNode2(ccfg);
+    }
+
+    /**
+     * @param ccfg Cache configuration.
+     * @throws Exception If failed.
+     */
+    private void notAffinityNode2(final CacheConfiguration ccfg) throws Exception {
+        log.info("Test with cache: " + ccfg.getName());
+
+        IgniteEx ignite0 = grid(0);
+        IgniteEx ignite1 = grid(1);
+
+        assertEquals(2L, ignite1.localNode().order());
+
+        ccfg.setNodeFilter(new TestFilterExcludeNode(2));
+
+        assertNotNull(ignite0.getOrCreateCache(ccfg));
+
+        assertNotNull(ignite1.cache(ccfg.getName()));
+
+        awaitPartitionMapExchange();
+
+        checkCache(ccfg.getName());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestChanged1() throws Exception {
+        IgniteEx ignite0 = grid(0);
+
+        assertEquals(1L, ignite0.localNode().order());
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setNodeFilter(new TestFilterExcludeOldest());
+
+        assertNotNull(ignite0.getOrCreateCache(ccfg));
+
+        stopGrid(0);
+
+        IgniteEx client = grid(NODES - 1);
+
+        assertTrue(client.configuration().isClientMode());
+
+        assertNotNull(client.getOrCreateCache((String)null));
+
+        awaitPartitionMapExchange();
+
+        checkCache(null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestChanged2() throws Exception {
+        IgniteEx ignite0 = grid(0);
+
+        assertEquals(1L, ignite0.localNode().order());
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setNodeFilter(new TestFilterIncludeNode(3));
+
+        assertNotNull(ignite0.getOrCreateCache(ccfg));
+
+        stopGrid(0);
+
+        IgniteEx ingite1 = grid(1);
+
+        assertNotNull(ingite1.getOrCreateCache((String)null));
+
+        awaitPartitionMapExchange();
+
+        checkCache(null);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testOldestChanged3() throws Exception {
+        IgniteEx ignite0 = grid(0);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setNodeFilter(new TestFilterIncludeNode(3));
+
+        assertNotNull(ignite0.getOrCreateCache(ccfg));
+
+        stopGrid(0);
+
+        IgniteEx client = grid(NODES - 1);
+
+        assertTrue(client.configuration().isClientMode());
+
+        assertNotNull(client.getOrCreateCache((String)null));
+
+        awaitPartitionMapExchange();
+
+        checkCache(null);
+    }
+
+    /**
+     * @param name Cache name.
+     */
+    private void checkCache(@Nullable String name) {
+        int key = 0;
+
+        for (Ignite ignite : G.allGrids()) {
+            IgniteCache<Object, Object> cache = ignite.cache(name);
+
+            assertNotNull(cache);
+
+            for (int i = 0; i < 100; i++) {
+                cache.put(key, key);
+
+                assertEquals(key, cache.get(key));
+
+                key++;
+            }
+        }
+    }
+
+    /**
+     * @return Cache configurations.
+     */
+    private List<CacheConfiguration> cacheConfigurations() {
+        List<CacheConfiguration> res = new ArrayList<>();
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-1");
+            ccfg.setAtomicityMode(ATOMIC);
+            ccfg.setBackups(0);
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-2");
+            ccfg.setAtomicityMode(ATOMIC);
+            ccfg.setBackups(1);
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-3");
+            ccfg.setAtomicityMode(ATOMIC);
+            ccfg.setBackups(1);
+            ccfg.setAffinity(new FairAffinityFunction());
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-4");
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(0);
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-5");
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(1);
+
+            res.add(ccfg);
+        }
+
+        {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setName("cache-4");
+            ccfg.setAtomicityMode(TRANSACTIONAL);
+            ccfg.setBackups(1);
+            ccfg.setAffinity(new FairAffinityFunction());
+
+            res.add(ccfg);
+        }
+
+        return res;
+    }
+
+    /**
+     *
+     */
+    private static class TestFilterExcludeOldest implements IgnitePredicate<ClusterNode> {
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode node) {
+            return node.order() > 1;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestFilterExcludeNode implements IgnitePredicate<ClusterNode> {
+        /** */
+        private final long excludeOrder;
+
+        /**
+         * @param excludeOrder Node order to exclude.
+         */
+        public TestFilterExcludeNode(long excludeOrder) {
+            this.excludeOrder = excludeOrder;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode node) {
+            return node.order() != excludeOrder;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestFilterIncludeNode implements IgnitePredicate<ClusterNode> {
+        /** */
+        private final long includeOrder;
+
+        /**
+         * @param includeOrder Node order to exclude.
+         */
+        public TestFilterIncludeNode(long includeOrder) {
+            this.includeOrder = includeOrder;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(ClusterNode node) {
+            return node.order() == includeOrder;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class TestCommunicationSpi extends TcpCommunicationSpi {
+        /** {@inheritDoc} */
+        @Override public void sendMessage(ClusterNode node, Message msg)
+            throws IgniteSpiException {
+            Object msg0 = ((GridIoMessage)msg).message();
+
+            if (msg0 instanceof GridDhtPartitionsSingleRequest) // Sent in case of exchange timeout.
+                fail("Unexpected message: " + msg0);
+
+            super.sendMessage(node, msg);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
new file mode 100644
index 0000000..282c7c8
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCache150ClientsTest.java
@@ -0,0 +1,189 @@
+/*
+ * 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.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+import org.jsr166.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ *
+ */
+public class IgniteCache150ClientsTest extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int CACHES = 10;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setNetworkTimeout(30_000);
+        cfg.setConnectorConfiguration(null);
+        cfg.setPeerClassLoadingEnabled(false);
+        cfg.setTimeServerPortRange(200);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSocketWriteTimeout(200);
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setLocalPortRange(200);
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(0);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setMaxMissedClientHeartbeats(200);
+
+        cfg.setClientMode(!gridName.equals(getTestGridName(0)));
+
+        CacheConfiguration[] ccfgs = new CacheConfiguration[CACHES];
+
+        for (int i = 0 ; i < ccfgs.length; i++) {
+            CacheConfiguration ccfg = new CacheConfiguration();
+
+            ccfg.setCacheMode(PARTITIONED);
+            ccfg.setAtomicityMode(i % 2 == 0 ? ATOMIC : TRANSACTIONAL);
+            ccfg.setWriteSynchronizationMode(PRIMARY_SYNC);
+            ccfg.setBackups(1);
+
+            ccfg.setName("cache-" + i);
+
+            ccfgs[i] = ccfg;
+        }
+
+        cfg.setCacheConfiguration(ccfgs);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 10 * 60_000;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        super.afterTest();
+
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void test150Clients() throws Exception {
+        Ignite srv = startGrid(0);
+
+        assertFalse(srv.configuration().isClientMode());
+
+        final int CLIENTS = 150;
+
+        final AtomicInteger idx = new AtomicInteger(1);
+
+        final CountDownLatch latch = new CountDownLatch(CLIENTS);
+
+        final List<String> cacheNames = new ArrayList<>();
+
+        for (int i = 0; i < CACHES; i++)
+            cacheNames.add("cache-" + i);
+
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                boolean cnt = false;
+
+                try {
+                    Ignite ignite = startGrid(idx.getAndIncrement());
+
+                    assertTrue(ignite.configuration().isClientMode());
+                    assertTrue(ignite.cluster().localNode().isClient());
+
+                    latch.countDown();
+
+                    cnt = true;
+
+                    log.info("Started [node=" + ignite.name() + ", left=" + latch.getCount() + ']');
+
+                    ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
+
+                    while (latch.getCount() > 0) {
+                        Thread.sleep(1000);
+
+                        IgniteCache<Object, Object> cache = ignite.cache(cacheNames.get(rnd.nextInt(0, CACHES)));
+
+                        Integer key = rnd.nextInt(0, 100_000);
+
+                        cache.put(key, 0);
+
+                        assertNotNull(cache.get(key));
+                    }
+
+                    return null;
+                }
+                finally {
+                    if (!cnt)
+                        latch.countDown();
+                }
+            }
+        }, CLIENTS, "start-client");
+
+        fut.get();
+
+        log.info("Started all clients.");
+
+        checkNodes(CLIENTS + 1);
+    }
+
+    /**
+     * @param expCnt Expected number of nodes.
+     */
+    private void checkNodes(int expCnt) {
+        assertEquals(expCnt, G.allGrids().size());
+
+        long topVer = -1L;
+
+        for (Ignite ignite : G.allGrids()) {
+            log.info("Check node: " + ignite.name());
+
+            if (topVer == -1L)
+                topVer = ignite.cluster().topologyVersion();
+            else
+                assertEquals("Unexpected topology version for node: " + ignite.name(),
+                    topVer,
+                    ignite.cluster().topologyVersion());
+
+            assertEquals("Unexpected number of nodes for node: " + ignite.name(),
+                expCnt,
+                ignite.cluster().nodes().size());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
index e5d30b6..d60a0c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
@@ -464,6 +464,7 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
 
     /**
      * @param nearCache If {@code true} creates near cache on client.
+     * @param srvNode If {@code true} creates client cache on server node.
      * @throws Exception If failed.
      */
     private void clientOnlyCacheStart(boolean nearCache, boolean srvNode) throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/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 6295a4d..d13a99d 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
@@ -102,6 +102,7 @@ public class IgniteCacheTestSuite4 extends TestSuite {
         suite.addTestSuite(IgniteCacheConfigurationTemplateTest.class);
         suite.addTestSuite(IgniteCacheConfigurationDefaultTemplateTest.class);
         suite.addTestSuite(IgniteDynamicClientCacheStartSelfTest.class);
+        suite.addTestSuite(IgniteDynamicCacheStartNoExchangeTimeoutTest.class);
 
         suite.addTestSuite(GridCacheTxLoadFromStoreOnLockSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a6cda33a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientTestSuite.java
new file mode 100644
index 0000000..98f9181
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteClientTestSuite.java
@@ -0,0 +1,38 @@
+/*
+ * 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.testsuites;
+
+import junit.framework.*;
+import org.apache.ignite.internal.processors.cache.distributed.*;
+
+/**
+ *
+ */
+public class IgniteClientTestSuite extends TestSuite {
+    /**
+     * @return Test suite.
+     * @throws Exception If failed.
+     */
+    public static TestSuite suite() throws Exception {
+        TestSuite suite = new TestSuite("Ignite Client Test Suite");
+
+        suite.addTestSuite(IgniteCache150ClientsTest.class);
+
+        return suite;
+    }
+}