You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by ag...@apache.org on 2018/11/28 11:53:20 UTC

[49/50] [abbrv] ignite git commit: IGNITE-10298 Cover possible deadlock in case of caches start and frequent checkpoints. - Fixes #5517.

IGNITE-10298 Cover possible deadlock in case of caches start and frequent checkpoints. - Fixes #5517.

Signed-off-by: Pavel Kovalenko <jo...@gmail.com>


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

Branch: refs/heads/ignite-9720
Commit: fe8c8cc5816872ba58bbd2401698b9e847788ea0
Parents: 2418d87
Author: Pavel Kovalenko <jo...@gmail.com>
Authored: Wed Nov 28 13:32:50 2018 +0300
Committer: Pavel Kovalenko <jo...@gmail.com>
Committed: Wed Nov 28 13:32:50 2018 +0300

----------------------------------------------------------------------
 .../GridCacheDatabaseSharedManager.java         |  20 ++-
 ...PdsCacheStartStopWithFreqCheckpointTest.java | 177 +++++++++++++++++++
 .../ignite/testsuites/IgnitePdsTestSuite2.java  |   3 +
 3 files changed, 198 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/fe8c8cc5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
index 9a083f8..eb52ee6 100755
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/persistence/GridCacheDatabaseSharedManager.java
@@ -250,6 +250,9 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     /** Checkpoint thread. Needs to be volatile because it is created in exchange worker. */
     private volatile Checkpointer checkpointer;
 
+    /** Checkpointer thread instance. */
+    private volatile IgniteThread checkpointerThread;
+
     /** For testing only. */
     private volatile boolean checkpointsEnabled = true;
 
@@ -427,6 +430,15 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
 
     /**
      * For test use only.
+     *
+     * @return Checkpointer thread instance.
+     */
+    public IgniteThread checkpointerThread() {
+        return checkpointerThread;
+    }
+
+    /**
+     * For test use only.
      */
     public IgniteInternalFuture<Void> enableCheckpoints(boolean enable) {
         GridFutureAdapter<Void> fut = new GridFutureAdapter<>();
@@ -2014,7 +2026,11 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
     @Override public void onStateRestored(AffinityTopologyVersion topVer) throws IgniteCheckedException {
         long time = System.currentTimeMillis();
 
-        new IgniteThread(cctx.igniteInstanceName(), "db-checkpoint-thread", checkpointer).start();
+        IgniteThread cpThread = new IgniteThread(cctx.igniteInstanceName(), "db-checkpoint-thread", checkpointer);
+
+        cpThread.start();
+
+        checkpointerThread = cpThread;
 
         CheckpointProgressSnapshot chp = checkpointer.wakeupForCheckpoint(0, "node started");
 
@@ -4648,7 +4664,7 @@ public class GridCacheDatabaseSharedManager extends IgniteCacheDatabaseSharedMan
             pageStore.ensure(grp.groupId(), p);
 
             if (pageStore.pages(grp.groupId(), p) <= 1) {
-                log.info("Partition [id=" + p + ", state=N/A (only file header) ]");
+                log.info("Partition [grp=" + grp.cacheOrGroupName() + ", id=" + p + ", state=N/A (only file header) ]");
 
                 continue;
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/fe8c8cc5/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheStartStopWithFreqCheckpointTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheStartStopWithFreqCheckpointTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheStartStopWithFreqCheckpointTest.java
new file mode 100644
index 0000000..8e00d88
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/persistence/IgnitePdsCacheStartStopWithFreqCheckpointTest.java
@@ -0,0 +1,177 @@
+/*
+ * 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.persistence;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.CacheRebalanceMode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.DataRegionConfiguration;
+import org.apache.ignite.configuration.DataStorageConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.configuration.WALMode;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Assert;
+
+/**
+ *
+ */
+public class IgnitePdsCacheStartStopWithFreqCheckpointTest extends GridCommonAbstractTest {
+    /** Caches. */
+    private static final int CACHES = 10;
+
+    /** Cache name. */
+    private static final String CACHE_NAME = "test";
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setConsistentId(igniteInstanceName);
+
+        DataStorageConfiguration dsCfg = new DataStorageConfiguration()
+            .setWalMode(WALMode.LOG_ONLY)
+            .setCheckpointFrequency(1000)
+            .setDefaultDataRegionConfiguration(
+                new DataRegionConfiguration()
+                    .setMaxSize(512 * 1024 * 1024)
+                    .setPersistenceEnabled(true)
+            );
+
+        cfg.setDataStorageConfiguration(dsCfg);
+
+        CacheConfiguration[] ccfgs = new CacheConfiguration[CACHES];
+
+        for (int i = 0; i < ccfgs.length; i++)
+            ccfgs[i] = cacheConfiguration(i);
+
+        cfg.setCacheConfiguration(ccfgs);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    private CacheConfiguration cacheConfiguration(int cacheIdx) {
+        return new CacheConfiguration(CACHE_NAME + cacheIdx)
+            .setCacheMode(CacheMode.REPLICATED)
+            .setBackups(0)
+            .setRebalanceMode(CacheRebalanceMode.NONE);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        cleanPersistenceDir();
+    }
+
+    /**
+     * Test checkpoint deadlock during caches start/stop process and frequent checkpoints is set.
+     *
+     * @throws Exception If failed.
+     */
+    public void testCheckpointDeadlock() throws Exception {
+        IgniteEx crd = startGrid(0);
+
+        crd.cluster().active(true);
+
+        for (int cacheId = 0; cacheId < CACHES; cacheId++) {
+            IgniteCache<Object, Object> cache = crd.getOrCreateCache(CACHE_NAME + cacheId);
+
+            for (int key = 0; key < 4096; key++)
+                cache.put(key, key);
+        }
+
+        forceCheckpoint();
+
+        final AtomicBoolean stopFlag = new AtomicBoolean();
+
+        IgniteInternalFuture<?> cacheStartStopFut = GridTestUtils.runAsync(() -> {
+            while (!stopFlag.get()) {
+                List<String> cacheNames = new ArrayList<>();
+                for (int i = 0; i < CACHES / 2; i++)
+                    cacheNames.add(CACHE_NAME + i);
+
+                try {
+                    // Stop cache without destroy.
+                    crd.context().cache().dynamicDestroyCaches(cacheNames, false, false, false).get();
+                }
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException("Failed to destroy cache", e);
+                }
+
+                List<CacheConfiguration> cachesToStart = new ArrayList<>();
+                for (int i = 0; i < CACHES / 2; i++)
+                    cachesToStart.add(cacheConfiguration(i));
+
+                crd.getOrCreateCaches(cachesToStart);
+            }
+        });
+
+        U.sleep(60_000);
+
+        log.info("Stopping caches start/stop process.");
+
+        stopFlag.set(true);
+
+        try {
+            cacheStartStopFut.get(30, TimeUnit.SECONDS);
+        }
+        catch (IgniteFutureTimeoutCheckedException e) {
+            U.dumpThreads(log);
+
+            log.warning("Caches start/stop future hangs. Interrupting checkpointer...");
+
+            interruptCheckpointer(crd);
+
+            // Should succeed.
+            cacheStartStopFut.get();
+
+            Assert.assertTrue("Checkpoint and exchange is probably in deadlock (see thread dump above for details).", false);
+        }
+    }
+
+    /**
+     * Interrupts checkpoint thread for given node.
+     *
+     * @param node Node.
+     */
+    private void interruptCheckpointer(IgniteEx node) {
+        GridCacheDatabaseSharedManager dbMgr = (GridCacheDatabaseSharedManager) node.context().cache().context().database();
+
+        dbMgr.checkpointerThread().interrupt();
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/fe8c8cc5/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
index c12f515..baaf882 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgnitePdsTestSuite2.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.persistence.IgniteDataStorageMetricsSelfTest;
+import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCacheStartStopWithFreqCheckpointTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsCorruptedStoreTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsExchangeDuringCheckpointTest;
 import org.apache.ignite.internal.processors.cache.persistence.IgnitePdsPageSizesTest;
@@ -117,6 +118,8 @@ public class IgnitePdsTestSuite2 extends TestSuite {
         suite.addTestSuite(IgniteWalFlushFsyncWithDedicatedWorkerSelfTest.class);
 
         suite.addTestSuite(IgniteWalFlushFsyncWithMmapBufferSelfTest.class);
+
+        suite.addTestSuite(IgnitePdsCacheStartStopWithFreqCheckpointTest.class);
     }
 
     /**