You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ir...@apache.org on 2019/08/01 16:51:50 UTC

[ignite] branch master updated: IGNITE-5227 StackOverflowError in GridCacheMapEntry#checkOwnerChanged() - Fixes #6736.

This is an automated email from the ASF dual-hosted git repository.

irakov pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ignite.git


The following commit(s) were added to refs/heads/master by this push:
     new b6dbc9e  IGNITE-5227 StackOverflowError in GridCacheMapEntry#checkOwnerChanged() - Fixes #6736.
b6dbc9e is described below

commit b6dbc9e781de9001c68007d9853938e85b217b87
Author: mstepachev <ma...@gmail.com>
AuthorDate: Thu Aug 1 19:47:08 2019 +0300

    IGNITE-5227 StackOverflowError in GridCacheMapEntry#checkOwnerChanged() - Fixes #6736.
    
    Signed-off-by: Ivan Rakov <ir...@apache.org>
---
 .../processors/cache/GridCacheMapEntry.java        |  16 ++-
 .../distributed/GridDistributedCacheEntry.java     |  18 ++-
 .../cache/local/GridLocalCacheEntry.java           |  19 ++-
 .../cache/CacheLockCandidatesThreadTest.java       | 133 +++++++++++++++++++++
 4 files changed, 173 insertions(+), 13 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index f1b7ec7..194c97a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -4865,6 +4865,19 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     protected final void checkOwnerChanged(@Nullable CacheLockCandidates prevOwners,
         @Nullable CacheLockCandidates owners,
         CacheObject val) {
+        checkOwnerChanged(prevOwners, owners, val, null);
+    }
+
+    /**
+     * @param prevOwners Previous owners.
+     * @param owners Current owners.
+     * @param val Entry value.
+     * @param checkingCandidate flag to enable or disable check of candidate chain
+     */
+    protected final void checkOwnerChanged(@Nullable CacheLockCandidates prevOwners,
+        @Nullable CacheLockCandidates owners,
+        CacheObject val,
+        CacheLockCandidates checkingCandidate) {
         assert !lock.isHeldByCurrentThread();
 
         if (prevOwners != null && owners == null) {
@@ -4900,7 +4913,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
                 if (locked) {
                     cctx.mvcc().callback().onOwnerChanged(this, owner);
 
-                    if (owner.local())
+                    if (owner.local()
+                        && (checkingCandidate == null || !checkingCandidate.hasCandidate(owner.version())))
                         checkThreadChain(owner);
 
                     if (cctx.events().isRecordable(EVT_CACHE_OBJECT_LOCKED)) {
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 9b1d0ca..51245e8 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
@@ -642,7 +642,7 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
     /**
      * Rechecks if lock should be reassigned.
      */
-    public void recheck() {
+    public CacheLockCandidates recheck(GridCacheMvccCandidate checkingCandidate) {
         CacheLockCandidates prev = null;
         CacheLockCandidates owner = null;
 
@@ -675,7 +675,9 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
         }
 
         // This call must be made outside of synchronization.
-        checkOwnerChanged(prev, owner, val);
+        checkOwnerChanged(prev, owner, val, checkingCandidate);
+
+        return owner;
     }
 
     /** {@inheritDoc} */
@@ -753,10 +755,14 @@ public class GridDistributedCacheEntry extends GridCacheMapEntry {
                     GridDistributedCacheEntry e =
                         (GridDistributedCacheEntry)cctx0.cache().peekEx(cand.parent().key());
 
-                    if (e != null)
-                        e.recheck();
-
-                    break;
+                    if (e != null) {
+                        CacheLockCandidates newOnwer = e.recheck(owner);
+                        if(newOnwer == null || !newOnwer.hasCandidate(cand.version()))
+                            // the lock from the chain hasn't been acquired, no sense to check the rest of the chain
+                            break;
+                    }
+                    else
+                        break;
                 }
             }
         }
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
index e26174a..65f7c400 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCacheEntry.java
@@ -199,8 +199,9 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
 
     /**
      * Rechecks if lock should be reassigned.
+     * @return owner
      */
-    public void recheck() {
+    public CacheLockCandidates recheck(GridCacheMvccCandidate checkingCandidate) {
         CacheObject val;
         CacheLockCandidates prev = null;
         CacheLockCandidates owner = null;
@@ -225,7 +226,9 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
             unlockEntry();
         }
 
-        checkOwnerChanged(prev, owner, val);
+        checkOwnerChanged(prev, owner, val, checkingCandidate);
+
+        return owner;
     }
 
     /** {@inheritDoc} */
@@ -248,10 +251,14 @@ public class GridLocalCacheEntry extends GridCacheMapEntry {
 
                     // At this point candidate may have been removed and entry destroyed,
                     // so we check for null.
-                    if (e != null)
-                        e.recheck();
-
-                    break;
+                    if (e != null) {
+                        CacheLockCandidates newOwner = e.recheck(owner);
+                        if(newOwner == null || !newOwner.hasCandidate(cand.version()))
+                            // the lock from the chain hasn't been acquired, no sense to check the rest of the chain
+                            break;
+                    }
+                    else
+                        break;
                 }
             }
         }
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesThreadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesThreadTest.java
new file mode 100644
index 0000000..5062f94
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheLockCandidatesThreadTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.concurrent.Callable;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.locks.Lock;
+
+/**
+ * Tests locking of thread of candidates (see GG-17364)
+ */
+public class CacheLockCandidatesThreadTest extends GridCommonAbstractTest {
+    /** */
+    private static final String DEFAULT_CACHE_NAME = "default";
+
+    /**
+     * @throws Exception if failed.
+     */
+    @Test
+    public void testLockCandidatesThreadForLocalMode() throws Exception {
+        lockThreadOfCandidates(CacheMode.LOCAL);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    @Test
+    public void testLockCandidatesThreadForReplicatedMode() throws Exception {
+        lockThreadOfCandidates(CacheMode.REPLICATED);
+    }
+
+    /**
+     * @throws Exception if failed.
+     */
+    @Test
+    public void testLockCandidatesThreadForPartitionedMode() throws Exception {
+        lockThreadOfCandidates(CacheMode.PARTITIONED);
+    }
+
+    /**
+     * @param mode Mode.
+     */
+    private void lockThreadOfCandidates(CacheMode mode) throws Exception {
+        startGridsMultiThreaded(1);
+
+        grid(0).createCache(new CacheConfiguration<Integer, Integer>(DEFAULT_CACHE_NAME)
+            .setAtomicityMode(CacheAtomicityMode.TRANSACTIONAL)
+            .setCacheMode(mode));
+
+        try {
+            final CountDownLatch unlock = new CountDownLatch(1);
+            final CountDownLatch locked = new CountDownLatch(1);
+
+            final IgniteCache<Object, Object> cache = grid(0).cache(DEFAULT_CACHE_NAME);
+
+            final String triggerKey = "" + ThreadLocalRandom.current().nextInt();
+
+            System.out.println("Trigger: " + triggerKey);
+
+            cache.put(triggerKey, "val");
+
+            IgniteInternalFuture<Object> future = GridTestUtils.runAsync(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    Lock lock = cache.lock(triggerKey);
+                    try {
+                        lock.lock();
+
+                        System.out.println("Trigger is locked");
+
+                        locked.countDown();
+
+                        unlock.await();
+                    } finally {
+                        lock.unlock();
+
+                        System.out.println("Trigger is unlocked");
+                    }
+
+                    return null;
+                }
+            });
+
+            locked.await();
+
+            Map<String, String> map = new TreeMap<>();
+
+            map.put(triggerKey, "trigger-new-val");
+
+            for (int i = 0; i < 4_000; i++)
+                map.put("key-" + i, "value");
+
+            IgniteFuture<Void> f = grid(0).cache(DEFAULT_CACHE_NAME).putAllAsync(map);
+
+            Thread.sleep(200);
+
+            unlock.countDown();
+
+            future.get();
+            f.get();
+        }
+        finally {
+            stopAllGrids();
+        }
+    }
+}