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 6DED4178F8 for ; Thu, 16 Apr 2015 14:56:12 +0000 (UTC) Received: (qmail 1623 invoked by uid 500); 16 Apr 2015 14:56:12 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 1595 invoked by uid 500); 16 Apr 2015 14:56:12 -0000 Mailing-List: contact commits-help@ignite.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@ignite.incubator.apache.org Delivered-To: mailing list commits@ignite.incubator.apache.org Received: (qmail 1586 invoked by uid 99); 16 Apr 2015 14:56:12 -0000 Received: from nike.apache.org (HELO nike.apache.org) (192.87.106.230) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 16 Apr 2015 14:56:12 +0000 X-ASF-Spam-Status: No, hits=-2000.0 required=5.0 tests=ALL_TRUSTED,T_RP_MATCHES_RCVD X-Spam-Check-By: apache.org Received: from [140.211.11.3] (HELO mail.apache.org) (140.211.11.3) by apache.org (qpsmtpd/0.29) with SMTP; Thu, 16 Apr 2015 14:55:47 +0000 Received: (qmail 96841 invoked by uid 99); 16 Apr 2015 14:55:43 -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; Thu, 16 Apr 2015 14:55:43 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 3408BE108C; Thu, 16 Apr 2015 14:55:43 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: sboikov@apache.org To: commits@ignite.incubator.apache.org Message-Id: <0e476094a0594363be889e24490b095e@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: incubator-ignite git commit: # ignite-746-1 removed timeout for atomic futures Date: Thu, 16 Apr 2015 14:55:43 +0000 (UTC) X-Virus-Checked: Checked by ClamAV on apache.org Repository: incubator-ignite Updated Branches: refs/heads/ignite-746-1 216cec3b3 -> cac5072f1 # ignite-746-1 removed timeout for atomic futures Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/cac5072f Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/cac5072f Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/cac5072f Branch: refs/heads/ignite-746-1 Commit: cac5072f15ec2a3c57996ce7e6a2454524653f71 Parents: 216cec3 Author: sboikov Authored: Thu Apr 16 17:55:35 2015 +0300 Committer: sboikov Committed: Thu Apr 16 17:55:35 2015 +0300 ---------------------------------------------------------------------- .../GridCacheAtomicTimeoutSelfTest.java | 314 ------------------- .../IgniteCacheAtomicMessageRecoveryTest.java | 32 ++ .../IgniteCacheMessageRecoveryAbstractTest.java | 175 +++++++++++ .../IgniteCacheTxMessageRecoveryTest.java | 32 ++ .../ignite/testsuites/IgniteCacheTestSuite.java | 2 + 5 files changed, 241 insertions(+), 314 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cac5072f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java deleted file mode 100644 index bde9ec7..0000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java +++ /dev/null @@ -1,314 +0,0 @@ -/* - * 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.cache.affinity.*; -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.*; -import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*; -import org.apache.ignite.internal.util.nio.*; -import org.apache.ignite.internal.util.typedef.*; -import org.apache.ignite.internal.util.typedef.internal.*; -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.testframework.*; -import org.apache.ignite.testframework.junits.common.*; - -import java.util.*; - -import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*; -import static org.apache.ignite.cache.CacheAtomicityMode.*; -import static org.apache.ignite.cache.CacheMode.*; -import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*; - -/** - * Tests timeout exception when message gets lost. - */ -public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest { - /** Grid count. */ - public static final int GRID_CNT = 3; - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - TestCommunicationSpi commSpi = new TestCommunicationSpi(); - - cfg.setCommunicationSpi(commSpi); - - CacheConfiguration ccfg = defaultCacheConfiguration(); - - ccfg.setCacheMode(PARTITIONED); - ccfg.setAtomicityMode(ATOMIC); - ccfg.setBackups(1); - ccfg.setAtomicWriteOrderMode(PRIMARY); - ccfg.setNearConfiguration(null); - ccfg.setWriteSynchronizationMode(FULL_SYNC); - - cfg.setCacheConfiguration(ccfg); - - cfg.setCommunicationSpi(new TestCommunicationSpi()); - - cfg.setNetworkTimeout(3000); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void beforeTestsStarted() throws Exception { - startGrids(GRID_CNT); - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - } - - /** {@inheritDoc} */ - @Override protected void afterTest() throws Exception { - for (int i = 0; i < GRID_CNT; i++) { - final IgniteKernal grid = (IgniteKernal)grid(i); - - TestCommunicationSpi commSpi = (TestCommunicationSpi)grid.configuration().getCommunicationSpi(); - - commSpi.skipNearRequest = false; - commSpi.skipNearResponse = false; - commSpi.skipDhtRequest = false; - commSpi.skipDhtResponse = false; - - GridTestUtils.retryAssert(log, 10, 100, new CA() { - @Override public void apply() { - assertTrue(grid.internalCache().context().mvcc().atomicFutures().isEmpty()); - } - }); - } - } - - /** - * @throws Exception If failed. - */ - public void testNearUpdateRequestLost() throws Exception { - Ignite ignite = grid(0); - - TestCommunicationSpi commSpi = (TestCommunicationSpi)grid(0).configuration().getCommunicationSpi(); - - IgniteCache cache = ignite.cache(null); - IgniteCache cacheAsync = cache.withAsync(); - - int key = keyForTest(); - - cache.put(key, 0); - - commSpi.skipNearRequest = true; - - cacheAsync.put(key, 1); - - IgniteFuture fut = cacheAsync.future(); - - Map clients = U.field(commSpi, "clients"); - - GridTcpNioCommunicationClient client = (GridTcpNioCommunicationClient)clients.get(grid(1).localNode().id()); - - client.session().close().get(); - - try { - fut.get(); - - fail(); - } - catch (IgniteException e) { - if (!(e.getCause() instanceof CacheAtomicUpdateTimeoutCheckedException)) - throw e; - - // Expected exception. - } - } - - /** - * @throws Exception If failed. - */ - public void testNearUpdateResponseLost() throws Exception { - Ignite ignite = grid(0); - - IgniteCache cache = ignite.cache(null); - IgniteCache cacheAsync = cache.withAsync(); - - int key = keyForTest(); - - cache.put(key, 0); - - TestCommunicationSpi commSpi = (TestCommunicationSpi)grid(1).configuration().getCommunicationSpi(); - - commSpi.skipNearResponse = true; - - cacheAsync.put(key, 1); - - IgniteFuture fut = cacheAsync.future(); - - Map clients = U.field(commSpi, "clients"); - - GridTcpNioCommunicationClient client = (GridTcpNioCommunicationClient)clients.get(grid(0).localNode().id()); - - client.session().close().get(); - - try { - fut.get(); - - fail(); - } - catch (IgniteException e) { - if (!(e.getCause() instanceof CacheAtomicUpdateTimeoutCheckedException)) - throw e; - - // Expected exception. - } - } - - /** - * @throws Exception If failed. - */ - public void testDhtUpdateRequestLost() throws Exception { - Ignite ignite = grid(0); - - IgniteCache cache = ignite.cache(null); - IgniteCache cacheAsync = cache.withAsync(); - - int key = keyForTest(); - - cache.put(key, 0); - - TestCommunicationSpi commSpi = (TestCommunicationSpi)grid(1).configuration().getCommunicationSpi(); - - commSpi.skipDhtRequest = true; - - cacheAsync.put(key, 1); - - IgniteFuture fut = cacheAsync.future(); - - Map clients = U.field(commSpi, "clients"); - - GridTcpNioCommunicationClient client = (GridTcpNioCommunicationClient)clients.get(grid(2).localNode().id()); - - client.session().close().get(); - - try { - fut.get(); - - fail(); - } - catch (IgniteException e) { - assertTrue("Invalid exception thrown: " + e, X.hasCause(e, CacheAtomicUpdateTimeoutCheckedException.class) - || X.hasSuppressed(e, CacheAtomicUpdateTimeoutCheckedException.class)); - } - } - - /** - * @throws Exception If failed. - */ - public void testDhtUpdateResponseLost() throws Exception { - Ignite ignite = grid(0); - - IgniteCache cache = ignite.cache(null); - IgniteCache cacheAsync = cache.withAsync(); - - int key = keyForTest(); - - cache.put(key, 0); - - TestCommunicationSpi commSpi = (TestCommunicationSpi)grid(2).configuration().getCommunicationSpi(); - - commSpi.skipDhtResponse = true; - - cacheAsync.put(key, 1); - - IgniteFuture fut = cacheAsync.future(); - - Map clients = U.field(commSpi, "clients"); - - GridTcpNioCommunicationClient client = (GridTcpNioCommunicationClient)clients.get(grid(1).localNode().id()); - - client.session().close().get(); - - try { - fut.get(); - - fail(); - } - catch (IgniteException e) { - assertTrue("Invalid exception thrown: " + e, X.hasCause(e, CacheAtomicUpdateTimeoutCheckedException.class) - || X.hasSuppressed(e, CacheAtomicUpdateTimeoutCheckedException.class)); - } - } - - /** - * @return Key for test; - */ - private int keyForTest() { - int i = 0; - - Affinity aff = grid(0).affinity(null); - - while (!aff.isPrimary(grid(1).localNode(), i) || !aff.isBackup(grid(2).localNode(), i)) - i++; - - return i; - } - - /** - * Communication SPI that will count single partition update messages. - */ - private static class TestCommunicationSpi extends TcpCommunicationSpi { - /** */ - private boolean skipNearRequest; - - /** */ - private boolean skipNearResponse; - - /** */ - private boolean skipDhtRequest; - - /** */ - private boolean skipDhtResponse; - - /** {@inheritDoc} */ - @Override public void sendMessage(ClusterNode node, Message msg) - throws IgniteSpiException { - if (!skipMessage((GridIoMessage)msg)) - super.sendMessage(node, msg); - } - - /** - * Checks if message should be skipped. - * - * @param msg Message. - */ - private boolean skipMessage(GridIoMessage msg) { - return msg.message() instanceof GridNearAtomicUpdateRequest && skipNearRequest - || msg.message() instanceof GridNearAtomicUpdateResponse && skipNearResponse - || msg.message() instanceof GridDhtAtomicUpdateRequest && skipDhtRequest - || msg.message() instanceof GridDhtAtomicUpdateResponse && skipDhtResponse; - } - } - -} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cac5072f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryTest.java new file mode 100644 index 0000000..589374b --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryTest.java @@ -0,0 +1,32 @@ +/* + * 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.cache.*; + +import static org.apache.ignite.cache.CacheAtomicityMode.*; + +/** + * + */ +public class IgniteCacheAtomicMessageRecoveryTest extends IgniteCacheMessageRecoveryAbstractTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return ATOMIC; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cac5072f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java new file mode 100644 index 0000000..96abe5f --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java @@ -0,0 +1,175 @@ +/* + * 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.cache.*; +import org.apache.ignite.configuration.*; +import org.apache.ignite.internal.*; +import org.apache.ignite.internal.util.nio.*; +import org.apache.ignite.internal.util.typedef.*; +import org.apache.ignite.internal.util.typedef.internal.*; +import org.apache.ignite.spi.communication.tcp.*; +import org.apache.ignite.testframework.*; +import org.apache.ignite.testframework.junits.common.*; + +import java.util.*; +import java.util.concurrent.*; +import java.util.concurrent.atomic.*; + +import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*; +import static org.apache.ignite.cache.CacheMode.*; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*; + +/** + * Tests message delivery after reconnection. + */ +public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonAbstractTest { + /** Grid count. */ + public static final int GRID_CNT = 3; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + TcpCommunicationSpi commSpi = new TcpCommunicationSpi(); + + commSpi.setSocketWriteTimeout(1000); + + cfg.setCommunicationSpi(commSpi); + + CacheConfiguration ccfg = defaultCacheConfiguration(); + + ccfg.setCacheMode(PARTITIONED); + ccfg.setAtomicityMode(atomicityMode()); + ccfg.setBackups(1); + ccfg.setAtomicWriteOrderMode(PRIMARY); + ccfg.setNearConfiguration(null); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + + cfg.setCacheConfiguration(ccfg); + + return cfg; + } + + /** + * @return Cache atomicity mode. + */ + protected abstract CacheAtomicityMode atomicityMode(); + + /** {@inheritDoc} */ + @Override protected void beforeTestsStarted() throws Exception { + startGrids(GRID_CNT); + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + for (int i = 0; i < GRID_CNT; i++) { + final IgniteKernal grid = (IgniteKernal)grid(i); + + GridTestUtils.retryAssert(log, 10, 100, new CA() { + @Override public void apply() { + assertTrue(grid.internalCache().context().mvcc().atomicFutures().isEmpty()); + } + }); + } + } + + /** + * @throws Exception If failed. + */ + public void testMessageRecovery() throws Exception { + final Ignite ignite = grid(0); + + final IgniteCache cache = ignite.cache(null); + + Map map = new HashMap<>(); + + for (int i = 0; i < 1000; i++) + map.put(i, "0"); + + cache.putAll(map); + + final AtomicBoolean stop = new AtomicBoolean(); + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Object call() throws Exception { + Thread.currentThread().setName("update-thread"); + + ThreadLocalRandom rnd = ThreadLocalRandom.current(); + + int iter = 0; + + while (!stop.get()) { + Map map = new HashMap<>(); + + for (int i = 0; i < 100; i++) + map.put(rnd.nextInt(0, 1000), String.valueOf(i)); + + cache.putAll(map); + + if (++iter % 100 == 0) + log.info("Iteration: " + iter); + } + + return null; + } + }); + + try { + for (int i = 0; i < 30; i++) { + Thread.sleep(1000); + + closeSessions(); + } + } + finally { + stop.set(true); + } + + fut.get(); + } + + /** + * @throws Exception If failed. + */ + private void closeSessions() throws Exception { + Ignite ignite = ignite(ThreadLocalRandom.current().nextInt(0, GRID_CNT)); + + log.info("Close sessions for: " + ignite.name()); + + TcpCommunicationSpi commSpi = (TcpCommunicationSpi)ignite.configuration().getCommunicationSpi(); + + Map clients = U.field(commSpi, "clients"); + + assertTrue(clients.size() > 0); + + for (GridCommunicationClient client : clients.values()) { + GridTcpNioCommunicationClient client0 = (GridTcpNioCommunicationClient)client; + + GridNioSession ses = client0.session(); + + ses.close(); + } + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cac5072f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxMessageRecoveryTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxMessageRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxMessageRecoveryTest.java new file mode 100644 index 0000000..f26948a --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheTxMessageRecoveryTest.java @@ -0,0 +1,32 @@ +/* + * 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.cache.*; + +import static org.apache.ignite.cache.CacheAtomicityMode.*; + +/** + * + */ +public class IgniteCacheTxMessageRecoveryTest extends IgniteCacheMessageRecoveryAbstractTest { + /** {@inheritDoc} */ + @Override protected CacheAtomicityMode atomicityMode() { + return TRANSACTIONAL; + } +} http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cac5072f/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 5b066ff..298cd65 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 @@ -138,6 +138,8 @@ public class IgniteCacheTestSuite extends TestSuite { suite.addTestSuite(GridCacheMissingCommitVersionSelfTest.class); suite.addTestSuite(GridCacheEntrySetIterationPreloadingSelfTest.class); suite.addTestSuite(GridCacheMixedPartitionExchangeSelfTest.class); + suite.addTestSuite(IgniteCacheAtomicMessageRecoveryTest.class); + suite.addTestSuite(IgniteCacheTxMessageRecoveryTest.class); suite.addTestSuite(GridCacheOffHeapTieredEvictionAtomicSelfTest.class); suite.addTestSuite(GridCacheOffHeapTieredEvictionSelfTest.class); suite.addTestSuite(GridCacheOffHeapTieredAtomicSelfTest.class);