You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dp...@apache.org on 2018/05/10 14:32:38 UTC

ignite git commit: IGNITE-7968 IgniteAtomicSequence.incrementAndGet throws ClusterTopologyException: Failed to acquire lock for keys - Fixes #3868.

Repository: ignite
Updated Branches:
  refs/heads/master e8e462663 -> 89aedb19f


IGNITE-7968 IgniteAtomicSequence.incrementAndGet throws ClusterTopologyException: Failed to acquire lock for keys - Fixes #3868.

Signed-off-by: dpavlov <dp...@apache.org>


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

Branch: refs/heads/master
Commit: 89aedb19f4b679d1702173f6061f676d48c173a1
Parents: e8e4626
Author: pvinokurov <vi...@gmail.com>
Authored: Thu May 10 17:32:11 2018 +0300
Committer: dpavlov <dp...@apache.org>
Committed: Thu May 10 17:32:11 2018 +0300

----------------------------------------------------------------------
 .../GridCacheAtomicSequenceImpl.java            | 12 +++----
 ...eAbstractDataStructuresFailoverSelfTest.java | 34 ++++++++++++++++++--
 2 files changed, 36 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/89aedb19/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
index fd4db4a..4e20267 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastructures/GridCacheAtomicSequenceImpl.java
@@ -26,18 +26,15 @@ import java.io.ObjectStreamException;
 import java.util.concurrent.Callable;
 import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReentrantLock;
-import org.apache.ignite.IgniteCacheRestartingException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.GridKernalContext;
+import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridNearTxLocal;
-import org.apache.ignite.internal.util.future.GridFutureAdapter;
-import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.internal.util.typedef.internal.A;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -223,7 +220,7 @@ public final class GridCacheAtomicSequenceImpl extends AtomicDataStructureProxy<
                 updateCall = internalUpdate(l, updated);
 
             try {
-                return updateCall.call();
+                return CU.retryTopologySafe(updateCall);
             }
             catch (IgniteCheckedException | IgniteException | IllegalStateException e) {
                 throw e;
@@ -359,7 +356,8 @@ public final class GridCacheAtomicSequenceImpl extends AtomicDataStructureProxy<
                     return curLocVal;
                 }
                 catch (Error | Exception e) {
-                    U.error(log, "Failed to get and add: " + this, e);
+                    if(!X.hasCause(e, ClusterTopologyCheckedException.class))
+                        U.error(log, "Failed to get and add: " + this, e);
 
                     throw e;
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/89aedb19/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
index 69a466d..9da1161 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
@@ -90,6 +90,9 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
     private static final String TRANSACTIONAL_CACHE_NAME = "tx_cache";
 
     /** */
+    private static final String CLIENT_INSTANCE_NAME = "client";
+
+    /** */
     private static final int TOP_CHANGE_CNT = 2;
 
     /** */
@@ -161,6 +164,16 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
     }
 
     /**
+     * Starts client node.
+     *
+     * @return client node.
+     * @throws Exception If failed.
+     */
+    protected IgniteEx startClient() throws Exception {
+        return startGrid(getConfiguration(CLIENT_INSTANCE_NAME).setClientMode(true));
+    }
+
+    /**
      * @throws Exception If failed.
      */
     public void testAtomicLongFailsWhenServersLeft() throws Exception {
@@ -1219,7 +1232,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
      * @throws Exception If failed.
      */
     public void testAtomicSequenceConstantTopologyChange() throws Exception {
-        doTestAtomicSequence(new ConstantTopologyChangeWorker(TOP_CHANGE_THREAD_CNT));
+        doTestAtomicSequence(new ConstantTopologyChangeWorker(TOP_CHANGE_THREAD_CNT, true));
     }
 
     /**
@@ -1236,7 +1249,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
      * @throws Exception If failed.
      */
     private void doTestAtomicSequence(ConstantTopologyChangeWorker topWorker) throws Exception {
-        try (IgniteAtomicSequence s = grid(0).atomicSequence(STRUCTURE_NAME, 1, true)) {
+        try (IgniteAtomicSequence s = startClient().atomicSequence(STRUCTURE_NAME, 1, true)) {
             IgniteInternalFuture<?> fut = topWorker.startChangingTopology(new IgniteClosure<Ignite, Object>() {
                 @Override public Object apply(Ignite ignite) {
                     assertTrue(ignite.atomicSequence(STRUCTURE_NAME, 1, false).get() > 0);
@@ -1314,6 +1327,9 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
         /** */
         private final int topChangeThreads;
 
+        /** Flag to enable circular topology change. */
+        private boolean circular;
+
         /**
          * @param topChangeThreads Number of topology change threads.
          */
@@ -1322,6 +1338,15 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
         }
 
         /**
+         * @param topChangeThreads Number of topology change threads.
+         * @param circular flag to enable circular topology change.
+         */
+        public ConstantTopologyChangeWorker(int topChangeThreads, boolean circular) {
+            this.topChangeThreads = topChangeThreads;
+            this.circular = circular;
+        }
+
+        /**
          * Starts changing cluster's topology.
          *
          * @param cb Callback to run after node start.
@@ -1349,7 +1374,10 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Ig
                                 cb.apply(g);
                             }
                             finally {
-                                stopGrid(idx);
+                                if(circular)
+                                    stopGrid(G.allGrids().get(0).configuration().getIgniteInstanceName());
+                                else
+                                    stopGrid(idx);
                             }
                         }
                     }