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 2015/12/25 11:56:46 UTC

ignite git commit: ignite-1.5 Do not allow 'committing' -> 'marked_rollback' tx state change.

Repository: ignite
Updated Branches:
  refs/heads/ignite-1537 bd86925b1 -> 96c35103c


ignite-1.5 Do not allow 'committing' -> 'marked_rollback' tx state  change.


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/96c35103
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/96c35103
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/96c35103

Branch: refs/heads/ignite-1537
Commit: 96c35103cc5745446faa7b4862ab1d84bb93fc0b
Parents: bd86925
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 25 13:56:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 25 13:56:24 2015 +0300

----------------------------------------------------------------------
 .../cache/transactions/IgniteTxAdapter.java     |  2 +-
 .../IgniteCachePutRetryAbstractSelfTest.java    | 36 ++++++++++++++++++--
 ...gniteCachePutRetryTransactionalSelfTest.java | 21 ++++++++++++
 .../junits/common/GridCommonAbstractTest.java   | 34 ++++++++++++++++--
 4 files changed, 87 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/96c35103/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 53f4f56..c13e86b 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
@@ -1079,7 +1079,7 @@ public abstract class IgniteTxAdapter extends GridMetadataAwareAdapter
                 }
 
                 case MARKED_ROLLBACK: {
-                    valid = prev == ACTIVE || prev == PREPARING || prev == PREPARED || prev == COMMITTING;
+                    valid = prev == ACTIVE || prev == PREPARING || prev == PREPARED;
 
                     break;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/96c35103/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
index 4eb8a6b..7532354 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryAbstractSelfTest.java
@@ -30,6 +30,8 @@ import javax.cache.integration.CacheLoaderException;
 import javax.cache.integration.CacheWriterException;
 import javax.cache.processor.EntryProcessorResult;
 import javax.cache.processor.MutableEntry;
+
+import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCache;
 import org.apache.ignite.cache.CacheAtomicWriteOrderMode;
 import org.apache.ignite.cache.CacheAtomicityMode;
@@ -238,7 +240,7 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst
      * @param store If {@code true} uses cache with store.
      * @throws Exception If failed.
      */
-    private void checkRetry(Test test, TestMemoryMode memMode, boolean store) throws Exception {
+    protected final void checkRetry(Test test, TestMemoryMode memMode, boolean store) throws Exception {
         ignite(0).createCache(cacheConfiguration(memMode, store));
 
         final AtomicBoolean finished = new AtomicBoolean();
@@ -259,7 +261,7 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst
             }
         });
 
-        IgniteCache<Integer, Integer> cache = ignite(0).cache(null);
+        final IgniteCache<Integer, Integer> cache = ignite(0).cache(null);
 
         int iter = 0;
 
@@ -309,6 +311,31 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst
                     break;
                 }
 
+                case TX_PUT: {
+                    while (System.currentTimeMillis() < stopTime) {
+                        final Integer val = ++iter;
+
+                        Ignite ignite = ignite(0);
+
+                        for (int i = 0; i < keysCnt; i++) {
+                            final Integer key = i;
+
+                            doInTransaction(ignite, new Callable<Void>() {
+                                @Override public Void call() throws Exception {
+                                    cache.put(key, val);
+
+                                    return null;
+                                }
+                            });
+                        }
+
+                        for (int i = 0; i < keysCnt; i++)
+                            assertEquals(val, cache.get(i));
+                    }
+
+                    break;
+                }
+
                 case PUT_ALL: {
                     while (System.currentTimeMillis() < stopTime) {
                         Integer val = ++iter;
@@ -541,7 +568,10 @@ public abstract class IgniteCachePutRetryAbstractSelfTest extends GridCommonAbst
         INVOKE,
 
         /** */
-        INVOKE_ALL
+        INVOKE_ALL,
+
+        /** */
+        TX_PUT
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/96c35103/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
index 7655464..9204bc8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCachePutRetryTransactionalSelfTest.java
@@ -101,6 +101,20 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
     /**
      * @throws Exception If failed.
      */
+    public void testExplicitTransactionRetriesSingleValue() throws Exception {
+        checkRetry(Test.TX_PUT, TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testExplicitTransactionRetriesSingleValueStoreEnabled() throws Exception {
+        checkRetry(Test.TX_PUT, TestMemoryMode.HEAP, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testExplicitTransactionRetries() throws Exception {
         explicitTransactionRetries(TestMemoryMode.HEAP, false);
     }
@@ -108,6 +122,13 @@ public class IgniteCachePutRetryTransactionalSelfTest extends IgniteCachePutRetr
     /**
      * @throws Exception If failed.
      */
+    public void testExplicitTransactionRetriesSingleOperation() throws Exception {
+        explicitTransactionRetries(TestMemoryMode.HEAP, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testExplicitTransactionRetriesStoreEnabled() throws Exception {
         explicitTransactionRetries(TestMemoryMode.HEAP, true);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/96c35103/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index e0ffc60..949290e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -978,7 +978,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     /**
      * @param key Key.
      * @param cacheName Cache name.
-     * @return Ignite instance which has primary cache for given key.
+     * @return Ignite instance which has backup cache for given key.
      */
     protected Ignite backupNode(Object key, String cacheName) {
         List<Ignite> allGrids = Ignition.allGrids();
@@ -1001,8 +1001,38 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     }
 
     /**
+     * @param key Key.
+     * @param cacheName Cache name.
+     * @return Ignite instances which has backup cache for given key.
+     */
+    protected List<Ignite> backupNodes(Object key, String cacheName) {
+        List<Ignite> allGrids = Ignition.allGrids();
+
+        assertFalse("There are no alive nodes.", F.isEmpty(allGrids));
+
+        Ignite ignite = allGrids.get(0);
+
+        Affinity<Object> aff = ignite.affinity(cacheName);
+
+        Collection<ClusterNode> nodes = aff.mapKeyToPrimaryAndBackups(key);
+
+        assertTrue("Expected more than one node for key [key=" + key + ", nodes=" + nodes +']', nodes.size() > 1);
+
+        Iterator<ClusterNode> it = nodes.iterator();
+
+        it.next(); // Skip primary.
+
+        List<Ignite> backups = new ArrayList<>(nodes.size() - 1);
+
+        while (it.hasNext())
+            backups.add(grid(it.next()));
+
+        return backups;
+    }
+
+    /**
      * In ATOMIC cache with CLOCK mode if key is updated from different nodes at same time
-     * only one update wins others are ignored (can happen in test event when updates are executed from
+     * only one update wins others are ignored (can happen in test even when updates are executed from
      * different nodes sequentially), this delay is used to avoid lost updates.
      *
      * @param cache Cache.