Return-Path: X-Original-To: apmail-ignite-commits-archive@minotaur.apache.org Delivered-To: apmail-ignite-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 4184418533 for ; Mon, 18 Jan 2016 05:56:25 +0000 (UTC) Received: (qmail 26325 invoked by uid 500); 18 Jan 2016 05:56:25 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 26285 invoked by uid 500); 18 Jan 2016 05:56:25 -0000 Mailing-List: contact commits-help@ignite.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.apache.org Delivered-To: mailing list commits@ignite.apache.org Received: (qmail 26272 invoked by uid 99); 18 Jan 2016 05:56:25 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 18 Jan 2016 05:56:25 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id D6850E058E; Mon, 18 Jan 2016 05:56:24 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.apache.org Date: Mon, 18 Jan 2016 05:56:24 -0000 Message-Id: <59f3c487a8c249d3a7eb41072573b681@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/2] ignite git commit: ignite-2386 Fixed DiscoverManager to do not increase minor topology version if exchange is not triggered. Repository: ignite Updated Branches: refs/heads/ignite-1811 a5ee64ca1 -> e81f4c499 ignite-2386 Fixed DiscoverManager to do not increase minor topology version if exchange is not triggered. Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/d8814178 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/d8814178 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/d8814178 Branch: refs/heads/ignite-1811 Commit: d8814178c2f8e93a76336069772cc3aec42fda46 Parents: 3a0cb51 Author: sboikov Authored: Fri Jan 15 15:15:02 2016 +0300 Committer: sboikov Committed: Fri Jan 15 15:15:02 2016 +0300 ---------------------------------------------------------------------- .../managers/discovery/CustomEventListener.java | 4 +- .../discovery/DiscoveryCustomMessage.java | 9 -- .../discovery/GridDiscoveryManager.java | 33 ++--- .../cache/DynamicCacheChangeBatch.java | 5 - .../processors/cache/GridCacheProcessor.java | 42 +++++-- .../continuous/AbstractContinuousMessage.java | 5 - .../continuous/GridContinuousProcessor.java | 13 +- ...niteDynamicCacheStartStopConcurrentTest.java | 119 +++++++++++++++++++ .../ignite/testframework/GridTestUtils.java | 27 +++++ .../testsuites/IgniteCacheTestSuite4.java | 2 + 10 files changed, 200 insertions(+), 59 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomEventListener.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomEventListener.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomEventListener.java index 8db4e67..ab143fb 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomEventListener.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/CustomEventListener.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.managers.discovery; import org.apache.ignite.cluster.ClusterNode; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; /** * Listener interface. @@ -27,7 +26,6 @@ public interface CustomEventListener { /** * @param snd Sender. * @param msg Message. - * @param topVer Current topology version. */ - public void onCustomEvent(ClusterNode snd, T msg, AffinityTopologyVersion topVer); + public void onCustomEvent(ClusterNode snd, T msg); } \ No newline at end of file http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java index 2ff40bf..d85075e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/DiscoveryCustomMessage.java @@ -18,7 +18,6 @@ package org.apache.ignite.internal.managers.discovery; import java.io.Serializable; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.lang.IgniteUuid; import org.jetbrains.annotations.Nullable; @@ -32,14 +31,6 @@ public interface DiscoveryCustomMessage extends Serializable { public IgniteUuid id(); /** - * Whether or not minor version of topology should be increased on message receive. - * - * @return {@code true} if minor topology version should be increased. - * @see AffinityTopologyVersion#minorTopVer - */ - public boolean incrementMinorTopologyVersion(); - - /** * Called when custom message has been handled by all nodes. * * @return Ack message or {@code null} if ack is not required. http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java index 23a85e4..29e85dd 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java @@ -474,21 +474,11 @@ public class GridDiscoveryManager extends GridManagerAdapter { if (type == EVT_NODE_METRICS_UPDATED) verChanged = false; - else if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) { - assert customMsg != null; - - if (customMsg.incrementMinorTopologyVersion()) { - minorTopVer++; - - verChanged = true; - } - else - verChanged = false; - } else { if (type != EVT_NODE_SEGMENTED && type != EVT_CLIENT_NODE_DISCONNECTED && - type != EVT_CLIENT_NODE_RECONNECTED) { + type != EVT_CLIENT_NODE_RECONNECTED && + type != DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) { minorTopVer = 0; verChanged = true; @@ -497,8 +487,6 @@ public class GridDiscoveryManager extends GridManagerAdapter { verChanged = false; } - final AffinityTopologyVersion nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer); - if (type == EVT_NODE_FAILED || type == EVT_NODE_LEFT) { for (DiscoCache c : discoCacheHist.values()) c.updateAlives(node); @@ -506,14 +494,27 @@ public class GridDiscoveryManager extends GridManagerAdapter { updateClientNodes(node.id()); } + final AffinityTopologyVersion nextTopVer; + if (type == DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT) { + assert customMsg != null; + + boolean incMinorTopVer = ctx.cache().onCustomEvent(customMsg, + new AffinityTopologyVersion(topVer, minorTopVer)); + + if (incMinorTopVer) { + minorTopVer++; + + verChanged = true; + } + for (Class cls = customMsg.getClass(); cls != null; cls = cls.getSuperclass()) { List> list = customEvtLsnrs.get(cls); if (list != null) { for (CustomEventListener lsnr : list) { try { - lsnr.onCustomEvent(node, customMsg, nextTopVer); + lsnr.onCustomEvent(node, customMsg); } catch (Exception e) { U.error(log, "Failed to notify direct custom event listener: " + customMsg, e); @@ -523,6 +524,8 @@ public class GridDiscoveryManager extends GridManagerAdapter { } } + nextTopVer = new AffinityTopologyVersion(topVer, minorTopVer); + // Put topology snapshot into discovery history. // There is no race possible between history maintenance and concurrent discovery // event notifications, since SPI notifies manager about all events from this listener. http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java index c7af64f..e10e5aa 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/DynamicCacheChangeBatch.java @@ -83,11 +83,6 @@ public class DynamicCacheChangeBatch implements DiscoveryCustomMessage { } /** {@inheritDoc} */ - @Override public boolean incrementMinorTopologyVersion() { - return true; - } - - /** {@inheritDoc} */ @Nullable @Override public DiscoveryCustomMessage ackMessage() { return null; } http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java index eb6d98e..16dfa7f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java @@ -69,8 +69,8 @@ import org.apache.ignite.internal.IgniteComponentType; import org.apache.ignite.internal.IgniteInternalFuture; import org.apache.ignite.internal.IgniteNodeAttributes; import org.apache.ignite.internal.IgniteTransactionsEx; -import org.apache.ignite.internal.managers.discovery.CustomEventListener; import org.apache.ignite.internal.binary.BinaryMarshaller; +import org.apache.ignite.internal.managers.discovery.DiscoveryCustomMessage; import org.apache.ignite.internal.processors.GridProcessorAdapter; import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.datastructures.CacheDataStructuresManager; @@ -580,15 +580,6 @@ public class GridCacheProcessor extends GridProcessorAdapter { "Deployment mode for cache is not CONTINUOUS or SHARED."); } - ctx.discovery().setCustomEventListener(DynamicCacheChangeBatch.class, - new CustomEventListener() { - @Override public void onCustomEvent(ClusterNode snd, - DynamicCacheChangeBatch msg, - AffinityTopologyVersion topVer) { - onCacheChangeRequested(msg, topVer); - } - }); - Set internalCaches = internalCachesNames(); CacheConfiguration[] cfgs = ctx.config().getCacheConfiguration(); @@ -2424,12 +2415,28 @@ public class GridCacheProcessor extends GridProcessorAdapter { } /** - * Callback invoked from discovery thread when cache deployment request is received. + * Callback invoked from discovery thread when discovery custom message is received. * + * @param msg Customer message. + * @param topVer Current topology version. + * @return {@code True} if minor topology version should be increased. + */ + public boolean onCustomEvent(DiscoveryCustomMessage msg, + AffinityTopologyVersion topVer) { + return msg instanceof DynamicCacheChangeBatch && onCacheChangeRequested((DynamicCacheChangeBatch) msg, topVer); + } + + /** * @param batch Change request batch. * @param topVer Current topology version. + * @return {@code True} if minor topology version should be increased. */ - private void onCacheChangeRequested(DynamicCacheChangeBatch batch, AffinityTopologyVersion topVer) { + private boolean onCacheChangeRequested(DynamicCacheChangeBatch batch, + AffinityTopologyVersion topVer) { + AffinityTopologyVersion newTopVer = null; + + boolean incMinorTopVer = false; + for (DynamicCacheChangeRequest req : batch.requests()) { if (req.template()) { CacheConfiguration ccfg = req.startCacheConfiguration(); @@ -2486,7 +2493,12 @@ public class GridCacheProcessor extends GridProcessorAdapter { DynamicCacheDescriptor startDesc = new DynamicCacheDescriptor(ctx, ccfg, req.cacheType(), false, req.deploymentId()); - startDesc.startTopologyVersion(topVer); + if (newTopVer == null) { + newTopVer = new AffinityTopologyVersion(topVer.topologyVersion(), + topVer.minorTopologyVersion() + 1); + } + + startDesc.startTopologyVersion(newTopVer); DynamicCacheDescriptor old = registeredCaches.put(maskNull(ccfg.getName()), startDesc); @@ -2562,7 +2574,11 @@ public class GridCacheProcessor extends GridProcessorAdapter { } req.exchangeNeeded(needExchange); + + incMinorTopVer |= needExchange; } + + return incMinorTopVer; } /** http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java index d6e25f1..8aa683e 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/AbstractContinuousMessage.java @@ -51,11 +51,6 @@ public abstract class AbstractContinuousMessage implements DiscoveryCustomMessag } /** {@inheritDoc} */ - @Override public boolean incrementMinorTopologyVersion() { - return false; - } - - /** {@inheritDoc} */ @Override public boolean isMutable() { return false; } http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java index 9bc9a38..cb028f3 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java @@ -52,7 +52,6 @@ import org.apache.ignite.internal.managers.deployment.GridDeploymentInfoBean; import org.apache.ignite.internal.managers.discovery.CustomEventListener; import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener; import org.apache.ignite.internal.processors.GridProcessorAdapter; -import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; import org.apache.ignite.internal.processors.cache.GridCacheAdapter; import org.apache.ignite.internal.processors.cache.GridCacheContext; import org.apache.ignite.internal.processors.cache.query.continuous.CacheContinuousQueryHandler; @@ -193,8 +192,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter { ctx.discovery().setCustomEventListener(StartRoutineDiscoveryMessage.class, new CustomEventListener() { @Override public void onCustomEvent(ClusterNode snd, - StartRoutineDiscoveryMessage msg, - AffinityTopologyVersion topVer) { + StartRoutineDiscoveryMessage msg) { if (!snd.id().equals(ctx.localNodeId()) && !ctx.isStopping()) processStartRequest(snd, msg); } @@ -203,8 +201,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter { ctx.discovery().setCustomEventListener(StartRoutineAckDiscoveryMessage.class, new CustomEventListener() { @Override public void onCustomEvent(ClusterNode snd, - StartRoutineAckDiscoveryMessage msg, - AffinityTopologyVersion topVer) { + StartRoutineAckDiscoveryMessage msg) { StartFuture fut = startFuts.remove(msg.routineId()); if (fut != null) { @@ -250,8 +247,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter { ctx.discovery().setCustomEventListener(StopRoutineDiscoveryMessage.class, new CustomEventListener() { @Override public void onCustomEvent(ClusterNode snd, - StopRoutineDiscoveryMessage msg, - AffinityTopologyVersion topVer) { + StopRoutineDiscoveryMessage msg) { if (!snd.id().equals(ctx.localNodeId())) { UUID routineId = msg.routineId(); @@ -270,8 +266,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter { ctx.discovery().setCustomEventListener(StopRoutineAckDiscoveryMessage.class, new CustomEventListener() { @Override public void onCustomEvent(ClusterNode snd, - StopRoutineAckDiscoveryMessage msg, - AffinityTopologyVersion topVer) { + StopRoutineAckDiscoveryMessage msg) { StopFuture fut = stopFuts.remove(msg.routineId()); if (fut != null) http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartStopConcurrentTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartStopConcurrentTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartStopConcurrentTest.java new file mode 100644 index 0000000..a208b07 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteDynamicCacheStartStopConcurrentTest.java @@ -0,0 +1,119 @@ +/* + * 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.Ignite; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteKernal; +import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion; +import org.apache.ignite.lang.IgniteInClosure; +import org.apache.ignite.lang.IgniteRunnable; +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.testframework.GridTestUtils; +import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest; + +/** + * + */ +public class IgniteDynamicCacheStartStopConcurrentTest extends GridCommonAbstractTest { + /** */ + private static final 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); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + super.beforeTestsStarted(); + + startGridsMultiThreaded(NODES); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** + * @throws Exception If failed. + */ + public void testConcurrentStartStop() throws Exception { + checkTopologyVersion(new AffinityTopologyVersion(NODES, 0)); + + int minorVer = 0; + + for (int i = 0; i < 5; i++) { + log.info("Iteration: " + i); + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer idx) { + Ignite ignite = ignite(idx); + + ignite.getOrCreateCache(new CacheConfiguration<>()); + } + }, NODES, "cache-thread"); + + minorVer++; + + checkTopologyVersion(new AffinityTopologyVersion(NODES, minorVer)); + + ignite(0).compute().affinityRun(null, 1, new IgniteRunnable() { + @Override public void run() { + // No-op. + } + }); + + GridTestUtils.runMultiThreaded(new IgniteInClosure() { + @Override public void apply(Integer idx) { + Ignite ignite = ignite(idx); + + ignite.destroyCache(null); + } + }, NODES, "cache-thread"); + + minorVer++; + + checkTopologyVersion(new AffinityTopologyVersion(NODES, minorVer)); + } + } + + /** + * @param topVer Expected version. + */ + private void checkTopologyVersion(AffinityTopologyVersion topVer) { + for (int i = 0; i < NODES; i++) { + IgniteKernal ignite = (IgniteKernal)ignite(i); + + assertEquals(ignite.name(), topVer, ignite.context().discovery().topologyVersionEx()); + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java index 7116227..26a8994 100644 --- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java +++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java @@ -93,6 +93,7 @@ import org.apache.ignite.internal.util.typedef.T2; import org.apache.ignite.internal.util.typedef.X; import org.apache.ignite.internal.util.typedef.internal.LT; import org.apache.ignite.internal.util.typedef.internal.U; +import org.apache.ignite.lang.IgniteInClosure; import org.apache.ignite.lang.IgnitePredicate; import org.apache.ignite.plugin.extensions.communication.Message; import org.apache.ignite.spi.swapspace.inmemory.GridTestSwapSpaceSpi; @@ -586,6 +587,32 @@ public final class GridTestUtils { } /** + * @param call Closure that receives thread index. + * @param threadNum Number of threads. + * @param threadName Thread names. + * @return Execution time in milliseconds. + * @throws Exception If failed. + */ + public static long runMultiThreaded(final IgniteInClosure call, int threadNum, String threadName) + throws Exception { + List> calls = new ArrayList<>(threadNum); + + for (int i = 0; i < threadNum; i++) { + final int idx = i; + + calls.add(new Callable() { + @Override public Void call() throws Exception { + call.apply(idx); + + return null; + } + }); + } + + return runMultiThreaded(calls, threadName); + } + + /** * Runs callable object in specified number of threads. * * @param call Callable. http://git-wip-us.apache.org/repos/asf/ignite/blob/d8814178/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 1b8eeda..04d0881 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 @@ -71,6 +71,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheTxStoreValueTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheFilterTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartNoExchangeTimeoutTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartSelfTest; +import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheStartStopConcurrentTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicCacheWithConfigStartSelfTest; import org.apache.ignite.internal.processors.cache.IgniteDynamicClientCacheStartSelfTest; import org.apache.ignite.internal.processors.cache.IgniteExchangeFutureHistoryTest; @@ -204,6 +205,7 @@ public class IgniteCacheTestSuite4 extends TestSuite { suite.addTestSuite(IgniteDynamicCacheStartSelfTest.class); suite.addTestSuite(IgniteDynamicCacheWithConfigStartSelfTest.class); suite.addTestSuite(IgniteCacheDynamicStopSelfTest.class); + suite.addTestSuite(IgniteDynamicCacheStartStopConcurrentTest.class); suite.addTestSuite(IgniteCacheConfigurationTemplateTest.class); suite.addTestSuite(IgniteCacheConfigurationDefaultTemplateTest.class); suite.addTestSuite(IgniteDynamicClientCacheStartSelfTest.class);