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/12/05 13:13:38 UTC

[ignite] branch master updated: IGNITE-12409 Destroying a cache during cache load may lead to a hang - Fixes #7092.

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 98883f1  IGNITE-12409 Destroying a cache during cache load may lead to a hang - Fixes #7092.
98883f1 is described below

commit 98883f151a36134ce6efc0bdf65c899505bf730a
Author: sk0x50 <sl...@gmail.com>
AuthorDate: Thu Dec 5 16:12:53 2019 +0300

    IGNITE-12409 Destroying a cache during cache load may lead to a hang - Fixes #7092.
    
    Signed-off-by: Ivan Rakov <ir...@apache.org>
---
 .../cache/GridCachePartitionExchangeManager.java   |   4 +-
 .../processors/datastreamer/DataStreamerImpl.java  |  20 +-
 .../datastreamer/DataStreamerStopCacheTest.java    | 213 +++++++++++++++++++++
 .../ignite/testsuites/IgniteCacheTestSuite.java    |   2 +
 4 files changed, 234 insertions(+), 5 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index dd28286..0f9fba1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -160,11 +160,11 @@ import static org.apache.ignite.internal.processors.affinity.AffinityTopologyVer
 import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.CachePartitionPartialCountersMap.PARTIAL_COUNTERS_MAP_SINCE;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture.nextDumpTimeout;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloader.DFLT_PRELOAD_RESEND_TIMEOUT;
-import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_OPS_BLOCKED_DURATION;
 import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_DURATION;
-import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_OPS_BLOCKED_DURATION_HISTOGRAM;
 import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_DURATION_HISTOGRAM;
 import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_METRICS;
+import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_OPS_BLOCKED_DURATION;
+import static org.apache.ignite.internal.processors.metric.GridMetricManager.PME_OPS_BLOCKED_DURATION_HISTOGRAM;
 
 /**
  * Partition exchange manager.
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
index ccb0f1f..cdf15f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerImpl.java
@@ -65,9 +65,9 @@ import org.apache.ignite.internal.IgniteFutureTimeoutCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.IgniteNodeAttributes;
+import org.apache.ignite.internal.cluster.ClusterReadOnlyModeCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyServerNotFoundException;
-import org.apache.ignite.internal.cluster.ClusterReadOnlyModeCheckedException;
 import org.apache.ignite.internal.managers.communication.GridIoMessage;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.deployment.GridDeployment;
@@ -76,7 +76,9 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.affinity.GridAffinityProcessor;
 import org.apache.ignite.internal.processors.cache.CacheObject;
 import org.apache.ignite.internal.processors.cache.CacheObjectContext;
+import org.apache.ignite.internal.processors.cache.CacheStoppedException;
 import org.apache.ignite.internal.processors.cache.DynamicCacheDescriptor;
+import org.apache.ignite.internal.processors.cache.ExchangeActions;
 import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
@@ -87,6 +89,7 @@ import org.apache.ignite.internal.processors.cache.IgniteCacheFutureImpl;
 import org.apache.ignite.internal.processors.cache.IgniteCacheProxy;
 import org.apache.ignite.internal.processors.cache.KeyCacheObject;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtInvalidPartitionException;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.topology.GridDhtPartitionState;
@@ -846,8 +849,19 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
 
             AffinityTopologyVersion topVer;
 
-            if (!cctx.isLocal())
-                topVer = ctx.cache().context().exchange().lastTopologyFuture().get();
+            if (!cctx.isLocal()) {
+                GridDhtPartitionsExchangeFuture exchFut = ctx.cache().context().exchange().lastTopologyFuture();
+
+                if (!exchFut.isDone()) {
+                    ExchangeActions acts = exchFut.exchangeActions();
+
+                    if (acts != null && acts.cacheStopped(CU.cacheId(cacheName)))
+                        throw new CacheStoppedException(cacheName);
+                }
+
+                // It is safe to block here even if the cache gate is acquired.
+                topVer = exchFut.get();
+            }
             else
                 topVer = ctx.cache().context().exchange().readyAffinityVersion();
 
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerStopCacheTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerStopCacheTest.java
new file mode 100644
index 0000000..6204a64
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerStopCacheTest.java
@@ -0,0 +1,213 @@
+/*
+ * 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.datastreamer;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import javax.cache.Cache;
+import javax.cache.CacheException;
+import javax.cache.configuration.FactoryBuilder;
+import javax.cache.integration.CacheLoaderException;
+import javax.cache.integration.CacheWriterException;
+import javax.cache.integration.CompletionListener;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.cache.affinity.rendezvous.RendezvousAffinityFunction;
+import org.apache.ignite.cache.store.CacheStoreAdapter;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.TestRecordingCommunicationSpi;
+import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsSingleMessage;
+import org.apache.ignite.internal.util.typedef.X;
+import org.apache.ignite.resources.IgniteInstanceResource;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+import static org.apache.ignite.cache.CacheMode.PARTITIONED;
+
+/**
+ * Tests that closing cache during uploading does not cause grid hang.
+ */
+public class DataStreamerStopCacheTest extends GridCommonAbstractTest {
+    /**
+     * Default timeout for operations.
+     */
+    private static final long TIMEOUT = 10_000;
+
+    /**
+     * Number of partitions.
+     */
+    private static final int PART_NUM = 32;
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setCacheConfiguration(cacheConfiguration());
+
+        TestRecordingCommunicationSpi commSpi = new TestRecordingCommunicationSpi();
+
+        cfg.setCommunicationSpi(commSpi);
+
+        return cfg;
+    }
+
+    /**
+     *
+     */
+    @Before
+    public void before() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     *
+     */
+    @After
+    public void after() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @return Cache configuration.
+     */
+    private CacheConfiguration cacheConfiguration() {
+        CacheConfiguration ccfg = defaultCacheConfiguration();
+
+        ccfg.setCacheMode(PARTITIONED);
+        ccfg.setAtomicityMode(ATOMIC);
+        ccfg.setAffinity(new RendezvousAffinityFunction(false, PART_NUM));
+
+        ccfg.setCacheStoreFactory(FactoryBuilder.factoryOf(TestCacheStore.class));
+
+        return ccfg;
+    }
+
+    /**
+     * Tests that stopping a cache does not lead to a deadlock while loading data through DataStreamer.
+     *
+     * @throws Exception if failed.
+     */
+    @Test
+    public void testLoadAllAndCacheStop() throws Exception {
+        final AtomicReference<Exception> fail = new AtomicReference<>();
+
+        final IgniteEx crd = startGrid(0);
+        final IgniteEx node1 = startGrid(1);
+
+        IgniteCache<Integer, String> c = node1.getOrCreateCache(DEFAULT_CACHE_NAME);
+
+        awaitPartitionMapExchange();
+
+        Set<Integer> keys = new HashSet<>();
+
+        for (int i = 0; i < PART_NUM; ++i) {
+            if (node1.affinity(DEFAULT_CACHE_NAME).isPrimary(node1.localNode(), i)) {
+                keys.add(i);
+
+                break;
+            }
+        }
+
+        final CountDownLatch loadFinished = new CountDownLatch(1);
+
+        GridTestUtils.runAsync(() -> {
+            c.loadAll(keys, true, new CompletionListener() {
+                @Override public void onCompletion() {
+                    loadFinished.countDown();
+                }
+
+                @Override public void onException(Exception e) {
+                    fail.compareAndSet(null, e);
+
+                    loadFinished.countDown();
+                }
+            });
+        });
+
+        assertTrue(
+            "loadAll() has not finished in " + TIMEOUT + " millis",
+            loadFinished.await(TIMEOUT, TimeUnit.MILLISECONDS));
+
+        assertTrue("Expected CacheException is not thrown", X.hasCause(fail.get(), CacheException.class));
+    }
+
+    /**
+     * Test cache store implementation.
+     */
+    public static class TestCacheStore extends CacheStoreAdapter<Integer, Integer> {
+        /**
+         * Ignite instance.
+         */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public Integer load(Integer key) throws CacheLoaderException {
+            // Block loading the key on the second node (non-coordinator).
+            if (((IgniteEx)ignite).localNode().order() != 2)
+                return key;
+
+            // It is guaranteed that at this point cache gate is already acquired.
+            TestRecordingCommunicationSpi spi = TestRecordingCommunicationSpi.spi(ignite);
+
+            spi.blockMessages((node, msg) -> msg instanceof GridDhtPartitionsSingleMessage);
+
+            GridTestUtils.runAsync(() -> ignite.destroyCache(DEFAULT_CACHE_NAME));
+
+            try {
+                spi.waitForBlocked(1, TIMEOUT);
+            }
+            catch (InterruptedException e) {
+                throw new CacheLoaderException("Failed to wait partition map exchange in " + TIMEOUT +" millis", e);
+            }
+            finally {
+                spi.stopBlock();
+            }
+
+            return key;
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public void write(
+            Cache.Entry<? extends Integer, ? extends Integer> entry) throws CacheWriterException {
+            // No-op.
+        }
+
+        /**
+         * {@inheritDoc}
+         */
+        @Override public void delete(Object key) throws CacheWriterException {
+            // No-op.
+        }
+    }
+}
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 304565c..1f41a08 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -150,6 +150,7 @@ import org.apache.ignite.internal.processors.datastreamer.DataStreamerClientReco
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerImplSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerMultiThreadedSelfTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerMultinodeCreateCacheTest;
+import org.apache.ignite.internal.processors.datastreamer.DataStreamerStopCacheTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerTimeoutTest;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamerUpdateAfterLoadTest;
 import org.apache.ignite.testframework.GridTestUtils;
@@ -256,6 +257,7 @@ public class IgniteCacheTestSuite {
         GridTestUtils.addTestIfNeeded(suite, DataStreamerUpdateAfterLoadTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, DataStreamerMultiThreadedSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, DataStreamerMultinodeCreateCacheTest.class, ignoredTests);
+        GridTestUtils.addTestIfNeeded(suite, DataStreamerStopCacheTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, DataStreamerImplSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, DataStreamerTimeoutTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, DataStreamerClientReconnectAfterClusterRestartTest.class, ignoredTests);