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 B4705183A1 for ; Tue, 29 Dec 2015 09:37:56 +0000 (UTC) Received: (qmail 50011 invoked by uid 500); 29 Dec 2015 09:37:56 -0000 Delivered-To: apmail-ignite-commits-archive@ignite.apache.org Received: (qmail 49974 invoked by uid 500); 29 Dec 2015 09:37:56 -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 49965 invoked by uid 99); 29 Dec 2015 09:37:56 -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; Tue, 29 Dec 2015 09:37:56 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 4E8D0DFC8B; Tue, 29 Dec 2015 09:37:56 +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 Message-Id: <0234a533d00f40ff8fc2239359d760d6@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: ignite git commit: ignite-1.5 debug Date: Tue, 29 Dec 2015 09:37:56 +0000 (UTC) Repository: ignite Updated Branches: refs/heads/ignite-1537 9449ca208 -> 24c69e688 ignite-1.5 debug Project: http://git-wip-us.apache.org/repos/asf/ignite/repo Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/24c69e68 Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/24c69e68 Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/24c69e68 Branch: refs/heads/ignite-1537 Commit: 24c69e68824cc51491d86824bdc6fb3f951c2eea Parents: 9449ca2 Author: sboikov Authored: Tue Dec 29 12:37:47 2015 +0300 Committer: sboikov Committed: Tue Dec 29 12:37:47 2015 +0300 ---------------------------------------------------------------------- .../cache/transactions/IgniteTxAdapter.java | 3 - .../IgniteCacheSyncUpdateNodeFailureTest.java | 330 ---------------- .../IgniteCacheCommitDelayTxRecoveryTest.java | 372 +++++++++++++++++++ .../IgniteCacheTxRecoverySelfTestSuite.java | 3 + 4 files changed, 375 insertions(+), 333 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/ignite/blob/24c69e68/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java ---------------------------------------------------------------------- diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java index 41b0f76..cf3016f 100644 --- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java +++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java @@ -1098,9 +1098,6 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter } case MARKED_ROLLBACK: { - if (prev == COMMITTING) - System.exit(99); - valid = prev == ACTIVE || prev == PREPARING || prev == PREPARED; break; http://git-wip-us.apache.org/repos/asf/ignite/blob/24c69e68/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSyncUpdateNodeFailureTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSyncUpdateNodeFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSyncUpdateNodeFailureTest.java deleted file mode 100644 index eb937c0..0000000 --- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheSyncUpdateNodeFailureTest.java +++ /dev/null @@ -1,330 +0,0 @@ -package org.apache.ignite.internal.processors.cache.distributed; - -import org.apache.ignite.Ignite; -import org.apache.ignite.IgniteCache; -import org.apache.ignite.cache.CacheEntryProcessor; -import org.apache.ignite.cache.store.CacheStore; -import org.apache.ignite.cache.store.CacheStoreAdapter; -import org.apache.ignite.configuration.CacheConfiguration; -import org.apache.ignite.configuration.IgniteConfiguration; -import org.apache.ignite.internal.IgniteInternalFuture; -import org.apache.ignite.internal.util.typedef.internal.U; -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; - -import javax.cache.Cache; -import javax.cache.configuration.Factory; -import javax.cache.integration.CacheLoaderException; -import javax.cache.integration.CacheWriterException; -import javax.cache.processor.MutableEntry; - -import java.util.Collections; -import java.util.HashSet; -import java.util.Set; -import java.util.concurrent.Callable; - -import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; -import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; - -/** - * - */ -public class IgniteCacheSyncUpdateNodeFailureTest extends GridCommonAbstractTest { - /** */ - private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); - - /** {@inheritDoc} */ - @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { - IgniteConfiguration cfg = super.getConfiguration(gridName); - - ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); - - CacheConfiguration ccfg = new CacheConfiguration(); - - ccfg.setAtomicityMode(TRANSACTIONAL); - ccfg.setBackups(1); - ccfg.setWriteSynchronizationMode(FULL_SYNC); - //ccfg.setCacheStoreFactory(new TestStoreFactory()); - - cfg.setCacheConfiguration(ccfg); - - return cfg; - } - - /** {@inheritDoc} */ - @Override protected void afterTestsStopped() throws Exception { - stopAllGrids(); - - super.afterTestsStopped(); - } - - /** - * @throws Exception If failed. - */ - public void testFullSyncOnePhaseCommit() throws Exception { - startGrids(3); - - awaitPartitionMapExchange(); - - Ignite ignite = ignite(0); - - final IgniteCache cache = ignite.cache(null); - - final Integer key = nearKey(cache); - - final String backupNode = backupNode(key, null).name(); - - Ignite primary = primaryNode(key, null); - - IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { - @Override public Void call() throws Exception { - log.info("Start update."); - - cache.invoke(key, new TestEntryProcessor(Collections.singleton(backupNode))); - - Integer val = cache.get(key); - - log.info("End update, value: " + val); - - return null; - } - }, "put-thread"); - - U.sleep(500); - - primary.close(); - - fut.get(); - - U.sleep(5000); - - log.info("Value at the end: " + cache.get(key)); - - cache.put(key, 2); - - log.info("Value at the end: " + cache.get(key)); -// try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { -// cache.put(key, 0); -// -// tx.commit(); -// } - } - - /** - * TODO: test with one backup and store, with two backups. - * - * @throws Exception If failed. - */ - public void testFullSyncTxRecovery1() throws Exception { - startGrids(4); - - awaitPartitionMapExchange(); - - Ignite ignite = ignite(0); - - final IgniteCache cache = ignite.cache(null); - - final Integer key = nearKey(cache); - - Ignite primary = primaryNode(key, null); - - IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { - @Override public Void call() throws Exception { - log.info("Start update."); - - Set backups = new HashSet<>(); - - for (Ignite backup : backupNodes(key, null)) - backups.add(backup.name()); - - cache.invoke(key, new TestEntryProcessor(backups)); - - log.info("End update, do get"); - - Integer val = cache.get(key); - - log.info("End update, value: " + val); - - return null; - } - }, "put-thread"); - - U.sleep(500); - - primary.close(); - - U.sleep(5000); - - U.dumpThreads(log); - - fut.get(); - - U.sleep(5000); - - log.info("Value at the end: " + cache.get(key)); - } - - /** - * TODO: test with one backup and store, with two backups. - * - * @throws Exception If failed. - */ - public void testFullSyncTxRecovery2() throws Exception { - startGrids(4); - - awaitPartitionMapExchange(); - - final Ignite ignite = ignite(0); - - final IgniteCache cache = ignite.cache(null); - - final Integer key = backupKey(cache); - - Ignite primary = primaryNode(key, null); - - IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { - @Override public Void call() throws Exception { - log.info("Start update."); - - Set backups = new HashSet<>(); - - for (Ignite backup : backupNodes(key, null)) - backups.add(backup.name()); - - cache.invoke(key, new TestEntryProcessor(backups)); - - log.info("End update, do get"); - - Integer val = cache.get(key); - - log.info("End update, value: " + val); - - return null; - } - }, "put-thread"); - - U.sleep(500); - - primary.close(); - - U.sleep(5000); - - U.dumpThreads(log); - - fut.get(); - - U.sleep(5000); - - log.info("Value at the end: " + cache.get(key)); - } - - /** - * TODO: test with one backup and store, with two backups. - * - * @throws Exception If failed. - */ - public void testFullSyncTxRecovery3() throws Exception { - startGrids(3); - - awaitPartitionMapExchange(); - - final Ignite ignite = ignite(0); - - final IgniteCache cache = ignite.cache(null); - - final Integer key = nearKey(cache); - - Ignite primary = primaryNode(key, null); - - IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { - @Override public Void call() throws Exception { - log.info("Start update."); - - cache.invoke(key, new TestEntryProcessor(Collections.singleton(backupNode(key, null).name()))); - - log.info("End update, do get"); - - Integer val = cache.get(key); - - log.info("End update, value: " + val); - - return null; - } - }, "put-thread"); - - U.sleep(500); - - primary.close(); - - //U.sleep(5000); - //U.dumpThreads(log); - - fut.get(); - - U.sleep(5000); - - log.info("Value at the end: " + cache.get(key)); - } - - /** - * - */ - static class TestEntryProcessor implements CacheEntryProcessor { - /** */ - private Set nodeNames; - - /** - * @param nodeNames Node names where sleep will be called. - */ - public TestEntryProcessor(Set nodeNames) { - this.nodeNames = nodeNames; - } - - /** {@inheritDoc} */ - @Override public Void process(MutableEntry entry, Object... args) { - Ignite ignite = entry.unwrap(Ignite.class); - - if (nodeNames.contains(ignite.name())) { - try { - System.out.println(Thread.currentThread().getName() + " sleep."); - - Thread.sleep(10_000); // TODO use Latch - - System.out.println(Thread.currentThread().getName() + " end sleep."); - } - catch (InterruptedException e) { - throw new RuntimeException(e); - } - } - - entry.setValue(1); - - return null; - } - } - - /** - * - */ - private static class TestStoreFactory implements Factory { - /** {@inheritDoc} */ - @Override public CacheStore create() { - return new CacheStoreAdapter() { - @Override public Object load(Object key) throws CacheLoaderException { - return null; - } - - @Override public void write(Cache.Entry entry) throws CacheWriterException { - // No-op. - } - - @Override public void delete(Object key) throws CacheWriterException { - // No-op. - } - }; - } - } -} http://git-wip-us.apache.org/repos/asf/ignite/blob/24c69e68/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCommitDelayTxRecoveryTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCommitDelayTxRecoveryTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCommitDelayTxRecoveryTest.java new file mode 100644 index 0000000..f42c908 --- /dev/null +++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheCommitDelayTxRecoveryTest.java @@ -0,0 +1,372 @@ +/* + * 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.dht; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Callable; +import java.util.concurrent.CountDownLatch; +import javax.cache.Cache; +import javax.cache.configuration.Factory; +import javax.cache.integration.CacheLoaderException; +import javax.cache.integration.CacheWriterException; +import javax.cache.processor.MutableEntry; +import org.apache.ignite.Ignite; +import org.apache.ignite.IgniteCache; +import org.apache.ignite.cache.CacheEntryProcessor; +import org.apache.ignite.cache.store.CacheStore; +import org.apache.ignite.cache.store.CacheStoreAdapter; +import org.apache.ignite.configuration.CacheConfiguration; +import org.apache.ignite.configuration.IgniteConfiguration; +import org.apache.ignite.internal.IgniteInternalFuture; +import org.apache.ignite.internal.util.typedef.G; +import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi; +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; +import org.apache.ignite.transactions.Transaction; + +import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.ignite.cache.CacheAtomicityMode.TRANSACTIONAL; +import static org.apache.ignite.cache.CacheRebalanceMode.SYNC; +import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC; +import static org.apache.ignite.transactions.TransactionConcurrency.PESSIMISTIC; +import static org.apache.ignite.transactions.TransactionIsolation.REPEATABLE_READ; + +/** + * + */ +public class IgniteCacheCommitDelayTxRecoveryTest extends GridCommonAbstractTest { + /** */ + private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true); + + /** */ + private static final int SRVS = 4; + + /** */ + private static volatile boolean commit; + + /** */ + private static volatile CountDownLatch commitStartedLatch; + + /** */ + private static volatile CountDownLatch commitFinishLatch; + + /** */ + private boolean client; + + /** {@inheritDoc} */ + @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception { + IgniteConfiguration cfg = super.getConfiguration(gridName); + + ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER); + + ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1); + + cfg.setClientMode(client); + + return cfg; + } + + /** {@inheritDoc} */ + @Override protected void afterTestsStopped() throws Exception { + stopAllGrids(); + + super.afterTestsStopped(); + } + + /** {@inheritDoc} */ + @Override protected void afterTest() throws Exception { + stopAllGrids(); + + super.afterTest(); + } + + /** + * @throws Exception If failed. + */ + public void testRecovery1() throws Exception { + checkRecovery(1, false); + } + + /** + * @throws Exception If failed. + */ + public void testRecovery2() throws Exception { + checkRecovery(2, false); + } + + /** + * @throws Exception If failed. + */ + public void testRecoveryStoreEnabled1() throws Exception { + checkRecovery(1, true); + } + + /** + * @throws Exception If failed. + */ + public void testRecoveryStoreEnabled2() throws Exception { + checkRecovery(2, true); + } + + /** + * @param backups Number of cache backups. + * @param useStore If {@code true} tests cache with store configured. + * @throws Exception If failed. + */ + private void checkRecovery(int backups, boolean useStore) throws Exception { + startGridsMultiThreaded(SRVS, false); + + client = true; + + Ignite clientNode = startGrid(SRVS); + + assertTrue(clientNode.configuration().isClientMode()); + + client = false; + + Ignite srv = ignite(0); + + assertFalse(srv.configuration().isClientMode()); + + clientNode.createCache(cacheConfiguration(backups, useStore)); + + awaitPartitionMapExchange(); + + for (Boolean pessimistic : Arrays.asList(false, true)) { + checkRecovery(backupKey(srv.cache(null)), srv, pessimistic, useStore); + + checkRecovery(nearKey(srv.cache(null)), srv, pessimistic, useStore); + + checkRecovery(nearKey(clientNode.cache(null)), clientNode, pessimistic, useStore); + } + } + + /** + * @param key Key. + * @param ignite Node executing update. + * @param pessimistic If {@code true} uses pessimistic transaction. + * @param useStore {@code True} if store is used. + * @throws Exception If failed. + */ + private void checkRecovery(final Integer key, + final Ignite ignite, + final boolean pessimistic, + final boolean useStore) throws Exception { + Ignite primary = primaryNode(key, null); + + assertNotSame(ignite, primary); + + List backups = backupNodes(key, null); + + assertFalse(backups.isEmpty()); + + final Set backupNames = new HashSet<>(); + + for (Ignite node : backups) + backupNames.add(node.name()); + + log.info("Check recovery [key=" + key + + ", pessimistic=" + pessimistic + + ", primary=" + primary.name() + + ", backups=" + backupNames + + ", node=" + ignite.name() + ']'); + + final IgniteCache cache = ignite.cache(null); + + cache.put(key, 0); + + commitStartedLatch = new CountDownLatch(backupNames.size()); + commitFinishLatch = new CountDownLatch(1); + + commit = false; + + TestEntryProcessor.skipFirst = useStore ? ignite.name() : null; + + IgniteInternalFuture fut = GridTestUtils.runAsync(new Callable() { + @Override public Void call() throws Exception { + log.info("Start update."); + + if (pessimistic) { + try (Transaction tx = ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ)) { + cache.invoke(key, new TestEntryProcessor(backupNames)); + + commit = true; + + log.info("Start commit."); + + assertEquals(backupNames.size(), commitStartedLatch.getCount()); + + tx.commit(); + } + } + else { + commit = true; + + cache.invoke(key, new TestEntryProcessor(backupNames)); + } + + log.info("End update, execute get."); + + Integer val = cache.get(key); + + log.info("Get value: " + val); + + assertEquals(1, (Object)val); + + return null; + } + }, "update-thread"); + + assertTrue(commitStartedLatch.await(30, SECONDS)); + + log.info("Stop node: " + primary.name()); + + primary.close(); + + commitFinishLatch.countDown(); + + fut.get(); + + for (Ignite node : G.allGrids()) + assertEquals(1, node.cache(null).get(key)); + + cache.put(key, 2); + + for (Ignite node : G.allGrids()) + assertEquals(2, node.cache(null).get(key)); + + startGrid(primary.name()); + + for (Ignite node : G.allGrids()) + assertEquals(2, node.cache(null).get(key)); + + cache.put(key, 3); + + for (Ignite node : G.allGrids()) + assertEquals(3, node.cache(null).get(key)); + + awaitPartitionMapExchange(); + } + + /** + * + */ + static class TestEntryProcessor implements CacheEntryProcessor { + /** */ + private Set nodeNames; + + /** Skips first call for given node (used to skip call for store update). */ + private static String skipFirst; + + /** + * @param nodeNames Node names where sleep will be called. + */ + public TestEntryProcessor(Set nodeNames) { + this.nodeNames = nodeNames; + } + + /** {@inheritDoc} */ + @Override public Void process(MutableEntry entry, Object... args) { + Ignite ignite = entry.unwrap(Ignite.class); + + System.out.println(Thread.currentThread().getName() + " process [node=" + ignite.name() + + ", commit=" + commit + ", skipFirst=" + skipFirst + ']'); + + boolean skip = false; + + if (commit && ignite.name().equals(skipFirst)) { + skipFirst = null; + + skip = true; + } + + if (!skip && commit && nodeNames.contains(ignite.name())) { + try { + System.out.println(Thread.currentThread().getName() + " start process invoke."); + + assertTrue(commitStartedLatch != null && commitStartedLatch.getCount() > 0); + + commitStartedLatch.countDown(); + + assertTrue(commitFinishLatch.await(10, SECONDS)); + + System.out.println(Thread.currentThread().getName() + " end process invoke."); + } + catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + else + System.out.println(Thread.currentThread().getName() + " invoke set value."); + + entry.setValue(1); + + return null; + } + } + + /** + * @param backups Number of backups. + * @param useStore If {@code true} adds cache store. + * @return Cache configuration. + */ + private CacheConfiguration cacheConfiguration(int backups, boolean useStore) { + CacheConfiguration ccfg = new CacheConfiguration<>(); + + ccfg.setAtomicityMode(TRANSACTIONAL); + ccfg.setBackups(backups); + ccfg.setWriteSynchronizationMode(FULL_SYNC); + ccfg.setRebalanceMode(SYNC); + + if (useStore) { + ccfg.setCacheStoreFactory(new TestStoreFactory()); + + ccfg.setWriteThrough(true); + } + + return ccfg; + } + + /** + * + */ + private static class TestStoreFactory implements Factory> { + /** {@inheritDoc} */ + @Override public CacheStore create() { + return new CacheStoreAdapter() { + @Override public Object load(Object key) throws CacheLoaderException { + return null; + } + + @Override public void write(Cache.Entry entry) throws CacheWriterException { + // No-op. + } + + @Override public void delete(Object key) throws CacheWriterException { + // No-op. + } + }; + } + } +} http://git-wip-us.apache.org/repos/asf/ignite/blob/24c69e68/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java index d81efd9..7363c7c 100644 --- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java +++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTxRecoverySelfTestSuite.java @@ -21,6 +21,7 @@ import junit.framework.TestSuite; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCacheColocatedTxPessimisticOriginatingNodeFailureSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledTxOriginatingNodeFailureSelfTest; import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedTxOriginatingNodeFailureSelfTest; +import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCacheCommitDelayTxRecoveryTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedNearDisabledPrimaryNodeFailureRecoveryTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedPrimaryNodeFailureRecoveryTest; import org.apache.ignite.internal.processors.cache.distributed.dht.IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest; @@ -39,6 +40,8 @@ public class IgniteCacheTxRecoverySelfTestSuite extends TestSuite { public static TestSuite suite() throws Exception { TestSuite suite = new TestSuite("Cache tx recovery test suite"); + suite.addTestSuite(IgniteCacheCommitDelayTxRecoveryTest.class); + suite.addTestSuite(IgniteCachePartitionedPrimaryNodeFailureRecoveryTest.class); suite.addTestSuite(IgniteCachePartitionedNearDisabledPrimaryNodeFailureRecoveryTest.class); suite.addTestSuite(IgniteCachePartitionedTwoBackupsPrimaryNodeFailureRecoveryTest.class);