This is an automated email from the ASF dual-hosted git repository.
irakov pushed a commit to branch ignite-2.8
in repository https://gitbox.apache.org/repos/asf/ignite.git
The following commit(s) were added to refs/heads/ignite-2.8 by this push:
new 0aaf490 IGNITE-12409 Destroying a cache during cache load may lead to
a hang - Fixes #7092.
0aaf490 is described below
commit 0aaf490ba00db0870753431a276ca6551db81ed9
Author: sk0x50 <[email protected]>
AuthorDate: Tue Dec 10 15:33:29 2019 +0300
IGNITE-12409 Destroying a cache during cache load may lead to a hang -
Fixes #7092.
(cherry picked from commit 98883f151a36134ce6efc0bdf65c899505bf730a)
Signed-off-by: Ivan Rakov <[email protected]>
---
.../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);