You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/08/21 11:04:52 UTC

[07/44] incubator-ignite git commit: # ignite-1265 set topology version for mvcc candidate

# ignite-1265 set topology version for mvcc candidate


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

Branch: refs/heads/ignite-gg-9615-1
Commit: b55365d316febf834cde553fbb55b33d42194069
Parents: 8b3fed8
Author: sboikov <sb...@gridgain.com>
Authored: Wed Aug 19 13:30:06 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Aug 19 13:53:41 2015 +0300

----------------------------------------------------------------------
 .../distributed/GridDistributedCacheEntry.java  |  7 ++
 .../distributed/dht/GridDhtTxPrepareFuture.java |  7 +-
 .../distributed/near/GridNearCacheEntry.java    |  6 ++
 .../distributed/near/GridNearLockFuture.java    |  4 +-
 .../cache/IgniteCacheInvokeReadThroughTest.java |  2 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |  4 +-
 .../near/GridCacheNearTxForceKeyTest.java       | 76 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite2.java       |  1 +
 8 files changed, 100 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
index bd72764..e007190 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheEntry.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache.distributed;
 
+import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -68,6 +69,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
      *
      * @param threadId Owning thread ID.
      * @param ver Lock version.
+     * @param topVer Topology version.
      * @param timeout Timeout to acquire lock.
      * @param reenter Reentry flag.
      * @param tx Transaction flag.
@@ -78,6 +80,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
     @Nullable public GridCacheMvccCandidate addLocal(
         long threadId,
         GridCacheVersion ver,
+        AffinityTopologyVersion topVer,
         long timeout,
         boolean reenter,
         boolean tx,
@@ -105,6 +108,9 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
 
             cand = mvcc.addLocal(this, threadId, ver, timeout, reenter, tx, implicitSingle);
 
+            if (cand != null)
+                cand.topologyVersion(topVer);
+
             owner = mvcc.anyOwner();
 
             boolean emptyAfter = mvcc.isEmpty();
@@ -732,6 +738,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
             return addLocal(
                 tx.threadId(),
                 tx.xidVersion(),
+                tx.topologyVersion(),
                 timeout,
                 false,
                 true,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index ad1023f..1539a2a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -797,6 +797,10 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
         return map;
     }
 
+    /**
+     * @param keysMap Keys to request.
+     * @return Keys request future.
+     */
     private IgniteInternalFuture<Object> forceRebalanceKeys(Map<Integer, Collection<KeyCacheObject>> keysMap) {
         if (F.isEmpty(keysMap))
             return null;
@@ -978,7 +982,8 @@ public final class GridDhtTxPrepareFuture extends GridCompoundFuture<IgniteInter
                         fut.onResult(e);
                     }
                     catch (IgniteCheckedException e) {
-                        fut.onResult(e);
+                        if (!cctx.kernalContext().isStopping())
+                            fut.onResult(e);
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
index 9e8d76b..194c68a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheEntry.java
@@ -433,6 +433,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
     @Override public GridCacheMvccCandidate addLocal(
         long threadId,
         GridCacheVersion ver,
+        AffinityTopologyVersion topVer,
         long timeout,
         boolean reenter,
         boolean tx,
@@ -441,6 +442,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
             null,
             threadId,
             ver,
+            topVer,
             timeout,
             reenter,
             tx,
@@ -454,6 +456,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
      * @param dhtNodeId DHT node ID.
      * @param threadId Owning thread ID.
      * @param ver Lock version.
+     * @param topVer Topology version.
      * @param timeout Timeout to acquire lock.
      * @param reenter Reentry flag.
      * @param tx Transaction flag.
@@ -465,6 +468,7 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
         @Nullable UUID dhtNodeId,
         long threadId,
         GridCacheVersion ver,
+        AffinityTopologyVersion topVer,
         long timeout,
         boolean reenter,
         boolean tx,
@@ -513,6 +517,8 @@ public class GridNearCacheEntry extends GridDistributedCacheEntry {
                 tx,
                 implicitSingle);
 
+            cand.topologyVersion(topVer);
+
             owner = mvcc.anyOwner();
 
             boolean emptyAfter = mvcc.isEmpty();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 3d28018..b7e0d73 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -307,6 +307,7 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             dhtNodeId,
             threadId,
             lockVer,
+            topVer,
             timeout,
             !inTx(),
             inTx(),
@@ -319,9 +320,6 @@ public final class GridNearLockFuture extends GridCompoundIdentityFuture<Boolean
             txEntry.cached(entry);
         }
 
-        if (c != null)
-            c.topologyVersion(topVer);
-
         synchronized (mux) {
             entries.add(entry);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
index b72540d..10ab1ab 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeReadThroughTest.java
@@ -34,7 +34,7 @@ import static org.apache.ignite.cache.CacheMode.*;
 public class IgniteCacheInvokeReadThroughTest extends IgniteCacheAbstractTest {
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-//        fail("https://issues.apache.org/jira/browse/IGNITE-114");
+        fail("https://issues.apache.org/jira/browse/IGNITE-114");
     }
 
     /** */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
index b6bc56e..d1d7c02 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
@@ -198,7 +198,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
             }
 
             // Test optimistic transaction.
-            GridTestUtils.assertThrows(log, new Callable<Object>() {
+            GridTestUtils.assertThrowsWithCause(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     try (Transaction tx = igniteNearOnly.transactions().txStart(OPTIMISTIC, REPEATABLE_READ)) {
                         nearOnly.put("key", "val");
@@ -208,7 +208,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
 
                     return null;
                 }
-            }, ClusterTopologyException.class, null);
+            }, ClusterTopologyCheckedException.class);
 
             // Test pessimistic transaction.
             GridTestUtils.assertThrowsWithCause(new Callable<Object>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java
new file mode 100644
index 0000000..44ef20d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxForceKeyTest.java
@@ -0,0 +1,76 @@
+/*
+ * 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.near;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+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.junits.common.*;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+
+/**
+ *
+ */
+public class GridCacheNearTxForceKeyTest 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.setWriteSynchronizationMode(FULL_SYNC);
+        ccfg.setRebalanceMode(ASYNC);
+        ccfg.setRebalanceDelay(5000);
+        ccfg.setBackups(0);
+        ccfg.setNearConfiguration(new NearCacheConfiguration());
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * Test provokes scenario when primary node sends force key request to node started transaction.
+     *
+     * @throws Exception If failed.
+     */
+    public void testNearTx() throws Exception {
+        Ignite ignite0 = startGrid(0);
+
+        IgniteCache<Integer, Integer> cache = ignite0.cache(null);
+
+        Ignite ignite1 = startGrid(1);
+
+        final Integer key = 2;
+
+        assertNull(cache.getAndPut(key, key));
+
+        assertTrue(ignite0.affinity(null).isPrimary(ignite1.cluster().localNode(), key));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b55365d3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index ec50399..495719f 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -142,6 +142,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
 
         suite.addTest(new TestSuite(IgniteCacheEntryProcessorNodeJoinTest.class));
         suite.addTest(new TestSuite(IgniteAtomicCacheEntryProcessorNodeJoinTest.class));
+        suite.addTest(new TestSuite(GridCacheNearTxForceKeyTest.class));
 
         return suite;
     }