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-973-2 Commit: a6cda33a8b76925d09c21262e88467421025fa77 Parents: c7ba154 Author: sboikov <sboi...@gridgain.com> Authored: Tue Jun 23 16:47:13 2015 +0300 Committer: sboikov <sboi...@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; + } +}