Repository: incubator-ignite Updated Branches: refs/heads/sprint-1 c3557e4cf -> b8c313583
IGNITE-CANCELLED Cancel user's request when grid stopping. Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/d5182f50 Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/d5182f50 Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/d5182f50 Branch: refs/heads/sprint-1 Commit: d5182f506cd0cce4e200448cb06103d199c60963 Parents: 188a18d Author: nikolay_tikhonov <ntikho...@gridgain.com> Authored: Sat Feb 14 17:35:22 2015 +0300 Committer: nikolay_tikhonov <ntikho...@gridgain.com> Committed: Sat Feb 14 17:35:22 2015 +0300 ---------------------------------------------------------------------- .../ignite/internal/GridKernalGateway.java | 7 + .../ignite/internal/GridKernalGatewayImpl.java | 16 + .../apache/ignite/internal/IgniteKernal.java | 23 +- .../processors/cache/GridCacheMvccManager.java | 15 + .../processors/cache/GridCacheProcessor.java | 8 + .../internal/GridUpdateNotifierSelfTest.java | 4 + .../IgniteCacheAbstractStopBusySelfTest.java | 371 +++++++++++++++++++ .../IgniteCacheAtomicStopBusySelfTest.java | 59 +++ ...gniteCacheTransactionalStopBusySelfTest.java | 53 +++ .../ignite/testsuites/IgniteCacheTestSuite.java | 2 + 10 files changed, 557 insertions(+), 1 deletion(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java index e3fdd15..09164d9 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGateway.java @@ -127,5 +127,12 @@ public interface GridKernalGateway { * Gets user stack trace through the first call of grid public API. */ public String userStackTrace(); + + /** + * @param timeout Timeout. + * @return {@code True} if write lock has been acquired. + * @throws InterruptedException If interrupted. + */ + public boolean tryWriteLock(long timeout) throws InterruptedException; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java index cd2f219..5982005 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalGatewayImpl.java @@ -121,6 +121,22 @@ public class GridKernalGatewayImpl implements GridKernalGateway, Serializable { Thread.currentThread().interrupt(); } + /** {@inheritDoc} */ + @Override public boolean tryWriteLock(long timeout) throws InterruptedException { + boolean acquired = rwLock.tryWriteLock(timeout, TimeUnit.MILLISECONDS); + + if (acquired) { + if (stackTrace == null) + stackTrace = stackTrace(); + + enterThreadLocals(); + + return true; + } + + return false; + } + /** * Retrieves user stack trace. * http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java index 51ff4a8..c253182 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java @@ -1732,6 +1732,7 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit } IgniteEmailProcessorAdapter email = ctx.email(); + GridCacheProcessor cacheProcessor = ctx.cache(); List<GridComponent> comps = ctx.components(); @@ -1761,7 +1762,27 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit if (metricsLogTimer != null) metricsLogTimer.cancel(); - gw.writeLock(); + boolean interrupted = false; + + while (true) { + try { + if (gw.tryWriteLock(10)) + break; + } + catch (InterruptedException e) { + // Preserve interrupt status & ignore. + // Note that interrupted flag is cleared. + interrupted = true; + } + finally { + // Cleanup even on successful acquire. + if (cacheProcessor != null) + cacheProcessor.cancelUserOperations(); + } + } + + if (interrupted) + Thread.currentThread().interrupt(); try { assert gw.getState() == STARTED || gw.getState() == STARTING; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java index 3c24cab..d125c02 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java @@ -271,6 +271,21 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K, } /** + * Cancels all client futures. + */ + public void cancelClientFutures() { + IgniteException e = new IgniteException("Operation has been cancelled (grid is stopping)."); + + for (Collection<GridCacheFuture<?>> futures : futs.values()) { + for (GridCacheFuture<?> future : futures) + ((GridFutureAdapter)future).onDone(e); + } + + for (GridCacheAtomicFuture<?, ?> future : atomicFuts.values()) + ((GridFutureAdapter)future).onDone(e); + } + + /** * @param from From version. * @return To version. */ http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/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 a4ea863..d641ef9 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 @@ -1689,6 +1689,14 @@ public class GridCacheProcessor extends GridProcessorAdapter { } /** + * Cancel all user operations. + */ + public void cancelUserOperations() { + for (GridCacheAdapter<?, ?> cache : caches.values()) + cache.ctx.mvcc().cancelClientFutures(); + } + + /** * @return All internal cache instances. */ public Collection<GridCacheAdapter<?, ?>> internalCaches() { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java index efcc00e..8cc0b28 100644 --- a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java +++ b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java @@ -87,5 +87,9 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest { @Override public String userStackTrace() { return null; } + + @Override public boolean tryWriteLock(long timeout) { + return false; + } }; } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java new file mode 100644 index 0000000..ea9c859 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java @@ -0,0 +1,371 @@ +/* + * 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.*; +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.near.*; +import org.apache.ignite.internal.util.typedef.*; +import org.apache.ignite.internal.util.typedef.internal.*; +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.io.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; + +/** + * + */ +public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbstractTest { + /** */ + public static final int CLN_GRD = 0; + + /** */ + public static final int SRV_GRD = 1; + + /** */ + public static final String CACHE_NAME = "StopTest"; + + /** */ + public static final TcpDiscoveryIpFinder finder = new TcpDiscoveryVmIpFinder(true); + + /** */ + private AtomicBoolean suspended = new AtomicBoolean(false); + + /** */ + protected AtomicReference<Class> bannedMessage = new AtomicReference<>(); + + /** + * @return Cache distribution mode. + */ + protected CacheDistributionMode cacheDistributionMode() { + return CacheDistributionMode.PARTITIONED_ONLY; + } + + /** + * @return Cache mode. + */ + protected CacheMode cacheMode(){ + return CacheMode.PARTITIONED; + } + + /** + * @return Cache atomicity mode. + */ + protected CacheAtomicityMode atomicityMode(){ + return CacheAtomicityMode.TRANSACTIONAL; + } + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + CacheConfiguration cacheCfg = cacheConfiguration(CACHE_NAME); + + TestTpcCommunicationSpi commSpi = new TestTpcCommunicationSpi(); + + commSpi.setLocalPort(GridTestUtils.getNextCommPort(getClass())); + + commSpi.setTcpNoDelay(true); + + if (gridName.endsWith(String.valueOf(CLN_GRD))) + cacheCfg.setDistributionMode(CacheDistributionMode.CLIENT_ONLY); + + cacheCfg.setPreloadMode(CachePreloadMode.SYNC); + + cacheCfg.setWriteSynchronizationMode(CacheWriteSynchronizationMode.FULL_SYNC); + + cacheCfg.setBackups(1); + + cfg.setCommunicationSpi(commSpi); + + TcpDiscoverySpi spi = new TcpDiscoverySpi(); + + spi.setIpFinder(finder); + + cfg.setDiscoverySpi(spi); + + cfg.setCacheConfiguration(cacheCfg); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void beforeTest() throws Exception { + beforeTestsStarted(); + + startGrid(SRV_GRD); + + startGrid(CLN_GRD); + + for (int i = 0; i < 10; ++i) { + if (clientNode().cluster().nodes().size() == 2) + break; + + TimeUnit.MILLISECONDS.sleep(100L); + } + + assert clientNode().cluster().nodes().size() == 2; + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + suspended.set(false); + + bannedMessage.set(null); + + afterTestsStopped(); + + stopGrid(SRV_GRD); + + stopGrid(CLN_GRD); + + assert G.allGrids().isEmpty(); + } + + /** + * @throws Exception If failed. + */ + public void testPut() throws Exception { + executeTest(new Callable<Integer>() { + /** {@inheritDoc} */ + @Override public Integer call() throws Exception { + info("Start operation."); + + Integer val = (Integer)clientCache().getAndPut(1, 999); + + info("Stop operation."); + + return val; + } + }); + } + + /** + * @throws Exception If failed. + */ + public void testRemove() throws Exception { + executeTest(new Callable<Integer>() { + /** {@inheritDoc} */ + @Override public Integer call() throws Exception { + info("Start operation."); + + Integer val = (Integer)clientCache().getAndRemove(1); + + info("Stop operation."); + + return val; + } + }); + } + + /** + * @throws Exception If failed. + */ + public void testPutAsync() throws Exception { + executeTest(new Callable<Object>() { + /** {@inheritDoc} */ + @Override public Object call() throws Exception { + info("Start operation."); + + IgniteCache<Object, Object> cache = clientCache().withAsync(); + + cache.getAndPut(1, 1); + + info("Stop operation."); + + return cache.future().get(); + } + }); + } + + /** + * @throws Exception If failed. + */ + public void testGet() throws Exception { + bannedMessage.set(GridNearGetRequest.class); + + executeTest(new Callable<Integer>() { + /** {@inheritDoc} */ + @Override + public Integer call() throws Exception { + info("Start operation."); + + Integer put = (Integer) clientCache().get(1); + + info("Stop operation."); + + return put; + } + }); + } + + /** + * + * @param call Closure executing cache operation. + * @throws Exception If failed. + */ + private <T> void executeTest(Callable<T> call) throws Exception { + suspended.set(true); + + IgniteInternalFuture<T> fut = GridTestUtils.runAsync(call); + + Thread stopThread = new Thread(new StopRunnable()); + + stopThread.start(); + + stopThread.join(10000L); + + suspended.set(false); + + assert !stopThread.isAlive(); + + Exception e = null; + + try { + fut.get(); + } + catch (IgniteCheckedException gridException){ + e = gridException; + } + + assertNotNull(e); + + StringWriter sw = new StringWriter(); + PrintWriter pw = new PrintWriter(sw); + + e.printStackTrace(pw); + + assertTrue(sw.toString().contains("grid is stopping")); + } + + /** + * @throws Exception If failed. + */ + public void testPutBatch() throws Exception { + assert !suspended.get(); + + IgniteInternalFuture<Void> fut = GridTestUtils.runAsync(new Callable<Void>() { + /** {@inheritDoc} */ + @Override public Void call() throws Exception { + for (int i = 0; i < 1_000_000; i++) + clientCache().put(i, i); + + return null; + } + }); + + Thread stopThread = new Thread(new StopRunnable()); + + U.sleep(100); + + stopThread.start(); + + stopThread.join(10000L); + + assert !stopThread.isAlive(); + + Exception e = null; + + try { + fut.get(); + } + catch (IgniteCheckedException gridException){ + e = gridException; + } + + assertNotNull(e); + } + + /** + * @return Client cache. + */ + private Ignite clientNode() { + return grid(CLN_GRD); + } + + /** + * @return Client cache. + */ + private IgniteCache<Object, Object> clientCache() { + return grid(CLN_GRD).jcache(CACHE_NAME); + } + + /** + * @param cacheName Cache name. + * @return Cache configuration. + * @throws Exception In case of error. + */ + private CacheConfiguration cacheConfiguration(@Nullable String cacheName) throws Exception { + CacheConfiguration cfg = defaultCacheConfiguration(); + + cfg.setCacheMode(cacheMode()); + + cfg.setAtomicityMode(atomicityMode()); + + cfg.setDistributionMode(cacheDistributionMode()); + + cfg.setName(cacheName); + + return cfg; + } + + /** + * + */ + private class TestTpcCommunicationSpi extends TcpCommunicationSpi { + /** {@inheritDoc} */ + @Override public void sendMessage(ClusterNode node, MessageAdapter msg) throws IgniteSpiException { + if (suspended.get()) { + assert bannedMessage.get() != null; + + if (msg instanceof GridIoMessage + && ((GridIoMessage)msg).message().getClass().equals(bannedMessage.get())) + return; + } + + super.sendMessage(node, msg); + } + } + + /** + * + */ + private class StopRunnable implements Runnable { + /** {@inheritDoc} */ + @Override public void run() { + info("Stopping grid..."); + + stopGrid(CLN_GRD, true); + + info("Grid stopped."); + } + } +} \ No newline at end of file http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java new file mode 100644 index 0000000..bae1601 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAtomicStopBusySelfTest.java @@ -0,0 +1,59 @@ +/* + * 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.cache.*; +import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*; + +/** + * Stopped node when client operations are executing. + */ +public class IgniteCacheAtomicStopBusySelfTest extends IgniteCacheAbstractStopBusySelfTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return CacheAtomicityMode.ATOMIC; + } + + /** {@inheritDoc} */ + @Override public void testPut() throws Exception { + bannedMessage.set(GridNearAtomicUpdateRequest.class); + + super.testPut(); + } + + /** {@inheritDoc} */ + @Override public void testPutBatch() throws Exception { + bannedMessage.set(GridNearAtomicUpdateRequest.class); + + super.testPut(); + } + + /** {@inheritDoc} */ + @Override public void testPutAsync() throws Exception { + bannedMessage.set(GridNearAtomicUpdateRequest.class); + + super.testPut(); + } + + /** {@inheritDoc} */ + @Override public void testRemove() throws Exception { + bannedMessage.set(GridNearAtomicUpdateRequest.class); + + super.testPut(); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTransactionalStopBusySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTransactionalStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTransactionalStopBusySelfTest.java new file mode 100644 index 0000000..5ae69d7 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheTransactionalStopBusySelfTest.java @@ -0,0 +1,53 @@ +/* + * 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.internal.processors.cache.distributed.near.*; + +/** + * Stopped node when client operations are executing. + */ +public class IgniteCacheTransactionalStopBusySelfTest extends IgniteCacheAbstractStopBusySelfTest { + /** {@inheritDoc} */ + @Override public void testPut() throws Exception { + bannedMessage.set(GridNearLockRequest.class); + + super.testPut(); + } + + /** {@inheritDoc} */ + @Override public void testPutBatch() throws Exception { + bannedMessage.set(GridNearLockRequest.class); + + super.testPut(); + } + + /** {@inheritDoc} */ + @Override public void testPutAsync() throws Exception { + bannedMessage.set(GridNearLockRequest.class); + + super.testPut(); + } + + /** {@inheritDoc} */ + @Override public void testRemove() throws Exception { + bannedMessage.set(GridNearLockRequest.class); + + super.testPut(); + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5182f50/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java index d3f7e3e..6884871 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java @@ -115,6 +115,8 @@ public class IgniteCacheTestSuite extends TestSuite { suite.addTestSuite(GridCacheTtlManagerSelfTest.class); suite.addTestSuite(GridCacheLifecycleAwareSelfTest.class); suite.addTestSuite(GridCacheStopSelfTest.class); + suite.addTestSuite(IgniteCacheAtomicStopBusySelfTest.class); + suite.addTestSuite(IgniteCacheTransactionalStopBusySelfTest.class); suite.addTestSuite(GridCacheAtomicNearCacheSelfTest.class); suite.addTestSuite(GridCacheStorePutxSelfTest.class); suite.addTestSuite(GridCacheOffHeapMultiThreadedUpdateSelfTest.class);