You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2016/09/08 15:41:00 UTC

[18/50] [abbrv] ignite git commit: ignite-2968 Deadlock detection for optimistic tx and near caches

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java
index abbefd0..4bf5a41 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionCrossCacheTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.transactions;
 
-import java.util.concurrent.ConcurrentMap;
+import java.util.Collection;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -26,6 +26,7 @@ import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheMode;
 import org.apache.ignite.configuration.CacheConfiguration;
 import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.NearCacheConfiguration;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.util.typedef.internal.U;
@@ -60,22 +61,6 @@ public class TxPessimisticDeadlockDetectionCrossCacheTest extends GridCommonAbst
             cfg.setDiscoverySpi(discoSpi);
         }
 
-        CacheConfiguration ccfg0 = defaultCacheConfiguration();
-
-        ccfg0.setName("cache0");
-        ccfg0.setCacheMode(CacheMode.PARTITIONED);
-        ccfg0.setBackups(1);
-        ccfg0.setNearConfiguration(null);
-
-        CacheConfiguration ccfg1 = defaultCacheConfiguration();
-
-        ccfg1.setName("cache1");
-        ccfg1.setCacheMode(CacheMode.PARTITIONED);
-        ccfg1.setBackups(1);
-        ccfg1.setNearConfiguration(null);
-
-        cfg.setCacheConfiguration(ccfg0, ccfg1);
-
         return cfg;
     }
 
@@ -96,70 +81,132 @@ public class TxPessimisticDeadlockDetectionCrossCacheTest extends GridCommonAbst
     /**
      * @throws Exception If failed.
      */
-    public void testDeadlock() throws Exception {
-        final CyclicBarrier barrier = new CyclicBarrier(2);
+    public void testDeadlockNoNear() throws Exception {
+        doTestDeadlock(false, false);
+    }
 
-        final AtomicInteger threadCnt = new AtomicInteger();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlockOneNear() throws Exception {
+        doTestDeadlock(false, true);
+    }
 
-        final AtomicBoolean deadlock = new AtomicBoolean();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlockAnotherNear() throws Exception {
+        doTestDeadlock(true, false);
+        doTestDeadlock(false, true);
+    }
 
-        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
-            @Override public void run() {
-                int threadNum = threadCnt.getAndIncrement();
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDeadlockBothNear() throws Exception {
+        doTestDeadlock(true, true);
+    }
 
-                Ignite ignite = ignite(0);
+    /**
+     * @param near0 Near flag for cache0.
+     * @param near1 Near flag for cache1.
+     */
+    private void doTestDeadlock(boolean near0, boolean near1) throws Exception {
+        IgniteCache<Integer, Integer> cache0 = null;
+        IgniteCache<Integer, Integer> cache1 = null;
 
-                IgniteCache<Integer, Integer> cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1));
+        try {
+            cache0 = getCache(ignite(0), "cache0", near0);
+            cache1 = getCache(ignite(0), "cache1", near1);
 
-                IgniteCache<Integer, Integer> cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0));
+            awaitPartitionMapExchange();
 
-                try (Transaction tx =
-                         ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 500, 0)
-                ) {
-                    int key1 = primaryKey(cache1);
+            final CyclicBarrier barrier = new CyclicBarrier(2);
 
-                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
-                        ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']');
+            final AtomicInteger threadCnt = new AtomicInteger();
 
-                    cache1.put(key1, 0);
+            final AtomicBoolean deadlock = new AtomicBoolean();
 
-                    barrier.await();
+            IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+                @Override public void run() {
+                    int threadNum = threadCnt.getAndIncrement();
 
-                    int key2 = primaryKey(cache2);
+                    Ignite ignite = ignite(0);
 
-                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
-                        ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']');
+                    IgniteCache<Integer, Integer> cache1 = ignite.cache("cache" + (threadNum == 0 ? 0 : 1));
 
-                    cache2.put(key2, 1);
+                    IgniteCache<Integer, Integer> cache2 = ignite.cache("cache" + (threadNum == 0 ? 1 : 0));
 
-                    tx.commit();
-                }
-                catch (Throwable e) {
-                    // At least one stack trace should contain TransactionDeadlockException.
-                    if (hasCause(e, TransactionTimeoutException.class) &&
-                        hasCause(e, TransactionDeadlockException.class)
-                        ) {
-                        if (deadlock.compareAndSet(false, true))
-                            U.error(log, "At least one stack trace should contain " +
-                                TransactionDeadlockException.class.getSimpleName(), e);
+                    try (Transaction tx =
+                             ignite.transactions().txStart(PESSIMISTIC, REPEATABLE_READ, 500, 0)
+                    ) {
+                        int key1 = primaryKey(cache1);
+
+                        log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
+                            ", tx=" + tx + ", key=" + key1 + ", cache=" + cache1.getName() + ']');
+
+                        cache1.put(key1, 0);
+
+                        barrier.await();
+
+                        int key2 = primaryKey(cache2);
+
+                        log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
+                            ", tx=" + tx + ", key=" + key2 + ", cache=" + cache2.getName() + ']');
+
+                        cache2.put(key2, 1);
+
+                        tx.commit();
+                    }
+                    catch (Throwable e) {
+                        // At least one stack trace should contain TransactionDeadlockException.
+                        if (hasCause(e, TransactionTimeoutException.class) &&
+                            hasCause(e, TransactionDeadlockException.class)
+                            ) {
+                            if (deadlock.compareAndSet(false, true))
+                                U.error(log, "At least one stack trace should contain " +
+                                    TransactionDeadlockException.class.getSimpleName(), e);
+                        }
                     }
                 }
-            }
-        }, 2, "tx-thread");
+            }, 2, "tx-thread");
 
-        fut.get();
+            fut.get();
 
-        assertTrue(deadlock.get());
+            assertTrue(deadlock.get());
 
-        for (int i = 0; i < NODES_CNT ; i++) {
-            Ignite ignite = ignite(i);
+            for (int i = 0; i < NODES_CNT ; i++) {
+                Ignite ignite = ignite(i);
 
-            IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm();
+                IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm();
 
-            ConcurrentMap<Long, TxDeadlockDetection.TxDeadlockFuture> futs =
-                GridTestUtils.getFieldValue(txMgr, IgniteTxManager.class, "deadlockDetectFuts");
+                Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
 
-            assertTrue(futs.isEmpty());
+                assertTrue(futs.isEmpty());
+            }
+        }
+        finally {
+            if (cache0 != null)
+                cache0.destroy();
+
+            if (cache1 != null)
+                cache1.destroy();
         }
     }
+
+    /**
+     * @param ignite Ignite.
+     * @param name Name.
+     * @param near Near.
+     */
+    private IgniteCache<Integer, Integer> getCache(Ignite ignite, String name, boolean near) {
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setName(name);
+        ccfg.setCacheMode(CacheMode.PARTITIONED);
+        ccfg.setBackups(1);
+        ccfg.setNearConfiguration(near ? new NearCacheConfiguration() : null);
+
+        return ignite.getOrCreateCache(ccfg);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
index ee1a989..83eb908 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxPessimisticDeadlockDetectionTest.java
@@ -25,7 +25,6 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CyclicBarrier;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
@@ -43,6 +42,7 @@ import org.apache.ignite.internal.IgniteKernal;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap;
 import org.apache.ignite.internal.processors.cache.GridCacheMapEntry;
+import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
@@ -135,7 +135,7 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
     /**
      * @throws Exception If failed.
      */
-    public void _testDeadlocksPartitionedNear() throws Exception {
+    public void testDeadlocksPartitionedNear() throws Exception {
         for (CacheWriteSynchronizationMode syncMode : CacheWriteSynchronizationMode.values()) {
             doTestDeadlocks(createCache(PARTITIONED, syncMode, true), NO_OP_TRANSFORMER);
             doTestDeadlocks(createCache(PARTITIONED, syncMode, true), WRAPPING_TRANSFORMER);
@@ -178,6 +178,7 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
      * @param cacheMode Cache mode.
      * @param syncMode Write sync mode.
      * @param near Near.
+     * @return Created cache.
      */
     @SuppressWarnings("unchecked")
     private IgniteCache createCache(CacheMode cacheMode, CacheWriteSynchronizationMode syncMode, boolean near) {
@@ -189,7 +190,19 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
         ccfg.setNearConfiguration(near ? new NearCacheConfiguration() : null);
         ccfg.setWriteSynchronizationMode(syncMode);
 
-        return ignite(0).getOrCreateCache(ccfg);
+        IgniteCache cache = ignite(0).createCache(ccfg);
+
+        if (near) {
+            for (int i = 0; i < NODES_CNT; i++) {
+                Ignite client = ignite(i + NODES_CNT);
+
+                assertTrue(client.configuration().isClientMode());
+
+                client.createNearCache(ccfg.getName(), new NearCacheConfiguration<>());
+            }
+        }
+
+        return cache;
     }
 
     /**
@@ -323,7 +336,14 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
             }
         }, loc ? 2 : txCnt, "tx-thread");
 
-        fut.get();
+        try {
+            fut.get();
+        }
+        catch (IgniteCheckedException e) {
+            U.error(null, "Unexpected exception", e);
+
+            fail();
+        }
 
         U.sleep(1000);
 
@@ -331,13 +351,17 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         assertNotNull(deadlockE);
 
+        boolean fail = false;
+
         // Check transactions, futures and entry locks state.
         for (int i = 0; i < NODES_CNT * 2; i++) {
             Ignite ignite = ignite(i);
 
             int cacheId = ((IgniteCacheProxy)ignite.cache(CACHE_NAME)).context().cacheId();
 
-            IgniteTxManager txMgr = ((IgniteKernal)ignite).context().cache().context().tm();
+            GridCacheSharedContext<Object, Object> cctx = ((IgniteKernal)ignite).context().cache().context();
+
+            IgniteTxManager txMgr = cctx.tm();
 
             Collection<IgniteInternalTx> activeTxs = txMgr.activeTransactions();
 
@@ -345,13 +369,16 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
                 Collection<IgniteTxEntry> entries = tx.allEntries();
 
                 for (IgniteTxEntry entry : entries) {
-                    if (entry.cacheId() == cacheId)
-                        fail("Transaction still exists: " + tx);
+                    if (entry.cacheId() == cacheId) {
+                        fail = true;
+
+                        U.error(log, "Transaction still exists: " + "\n" + tx.xidVersion() +
+                            "\n" + tx.nearXidVersion() + "\n nodeId=" + cctx.localNodeId() + "\n tx=" + tx);
+                    }
                 }
             }
 
-            ConcurrentMap<Long, TxDeadlockDetection.TxDeadlockFuture> futs =
-                GridTestUtils.getFieldValue(txMgr, IgniteTxManager.class, "deadlockDetectFuts");
+            Collection<IgniteInternalFuture<?>> futs = txMgr.deadlockDetectionFutures();
 
             assertTrue(futs.isEmpty());
 
@@ -371,6 +398,9 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
             }
         }
 
+        if (fail)
+            fail("Some transactions still exist");
+
         // Check deadlock report
         String msg = deadlockE.getMessage();
 
@@ -484,4 +514,4 @@ public class TxPessimisticDeadlockDetectionTest extends GridCommonAbstractTest {
             return id;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 8414461..14e5833 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -36,6 +36,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePart
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredAtomicFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridCachePartitionedNearDisabledOffHeapTieredMultiNodeFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedMultiNodeLongTxTimeoutFullApiTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicClientOnlyMultiNodeP2PDisabledFullApiSelfTest;
@@ -74,6 +75,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeCounterSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapMultiNodeFullApiSelfTest;
@@ -167,6 +169,8 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderMultiNodeP2PDisabledFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicNearEnabledMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCacheAtomicNearEnabledPrimaryWriteOrderMultiNodeFullApiSelfTest.class);
+        suite.addTestSuite(CachePartitionedMultiNodeLongTxTimeoutFullApiTest.class);
+        suite.addTestSuite(CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest.class);
 
         suite.addTestSuite(GridCachePartitionedNearDisabledMultiNodeFullApiSelfTest.class);
         suite.addTestSuite(GridCachePartitionedNearDisabledMultiNodeP2PDisabledFullApiSelfTest.class);

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java
index c057e55..5a1b1ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/TxDeadlockDetectionTestSuite.java
@@ -19,7 +19,10 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.transactions.DepthFirstSearchTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionNoHangsTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxDeadlockDetectionTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticDeadlockDetectionCrossCacheTest;
+import org.apache.ignite.internal.processors.cache.transactions.TxOptimisticDeadlockDetectionTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPessimisticDeadlockDetectionCrossCacheTest;
 import org.apache.ignite.internal.processors.cache.transactions.TxPessimisticDeadlockDetectionTest;
 
@@ -35,9 +38,12 @@ public class TxDeadlockDetectionTestSuite extends TestSuite {
         TestSuite suite = new TestSuite("Ignite Deadlock Detection Test Suite");
 
         suite.addTestSuite(DepthFirstSearchTest.class);
+        suite.addTestSuite(TxOptimisticDeadlockDetectionTest.class);
+        suite.addTestSuite(TxOptimisticDeadlockDetectionCrossCacheTest.class);
         suite.addTestSuite(TxPessimisticDeadlockDetectionTest.class);
         suite.addTestSuite(TxPessimisticDeadlockDetectionCrossCacheTest.class);
         suite.addTestSuite(TxDeadlockDetectionTest.class);
+        suite.addTestSuite(TxDeadlockDetectionNoHangsTest.class);
 
         return suite;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/0465874d/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala
index 22fb89d..1b55505 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/commands/cache/VisorCacheStopCommand.scala
@@ -19,6 +19,7 @@ package org.apache.ignite.visor.commands.cache
 
 import org.apache.ignite.cluster.{ClusterGroupEmptyException, ClusterNode}
 import org.apache.ignite.visor.visor._
+
 import org.apache.ignite.internal.visor.cache.VisorCacheStopTask
 import org.apache.ignite.internal.visor.util.VisorTaskUtils._
 
@@ -101,9 +102,7 @@ class VisorCacheStopCommand {
                 return
         }
 
-        val dflt = if (batchMode) "y" else "n"
-
-        ask(s"Are you sure you want to stop cache: ${escapeName(cacheName)}? (y/n) [$dflt]: ", dflt) match {
+        ask(s"Are you sure you want to stop cache: ${escapeName(cacheName)}? (y/n) [n]: ", "n") match {
             case "y" | "Y" =>
                 try {
                     executeRandom(grp, classOf[VisorCacheStopTask], cacheName)