You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sk...@apache.org on 2021/04/15 19:30:05 UTC
[ignite] branch master updated: IGNITE-14331 Fixed an issue when
Data Streamer flushing could hang due to a connectivity problem. Fixes
#8955
This is an automated email from the ASF dual-hosted git repository.
sk0x50 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 9195498 IGNITE-14331 Fixed an issue when Data Streamer flushing could hang due to a connectivity problem. Fixes #8955
9195498 is described below
commit 91954983040d69287ac50421a9c78ad70f172862
Author: Slava Koptilin <sl...@gmail.com>
AuthorDate: Thu Apr 15 22:29:20 2021 +0300
IGNITE-14331 Fixed an issue when Data Streamer flushing could hang due to a connectivity problem. Fixes #8955
---
.../processors/datastreamer/DataStreamerImpl.java | 37 +++---
.../DataStreamerCommunicationSpiExceptionTest.java | 143 +++++++++++++++++++++
.../ignite/testsuites/IgniteCacheTestSuite.java | 2 +
3 files changed, 165 insertions(+), 17 deletions(-)
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 67d9d5b..9b1f624 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
@@ -1539,7 +1539,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
/** Node. */
private final ClusterNode node;
- /** Active futures. */
+ /** Active futures for a local updates. */
private final Collection<IgniteInternalFuture<Object>> locFuts;
/** Buffered entries. */
@@ -1551,7 +1551,7 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
/** ID generator. */
private final AtomicLong idGen = new AtomicLong();
- /** Active futures. */
+ /** Active futures related to the remote node. */
private final ConcurrentMap<Long, GridFutureAdapter<Object>> reqs;
/** */
@@ -1999,26 +1999,29 @@ public class DataStreamerImpl<K, V> implements IgniteDataStreamer<K, V>, Delayed
if (log.isDebugEnabled())
log.debug("Sent request to node [nodeId=" + node.id() + ", req=" + req + ']');
}
- catch (ClusterTopologyCheckedException e) {
- GridFutureAdapter<Object> fut0 = ((GridFutureAdapter<Object>)fut);
-
- fut0.onDone(e);
- }
catch (IgniteCheckedException e) {
+ // This request was not sent probably.
+ // Anyway it does not make sense to track it.
+ reqs.remove(reqId);
+
GridFutureAdapter<Object> fut0 = ((GridFutureAdapter<Object>)fut);
- if (X.hasCause(e, IgniteClientDisconnectedCheckedException.class, IgniteClientDisconnectedException.class))
+ if (e instanceof ClusterTopologyCheckedException)
fut0.onDone(e);
else {
- try {
- if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
- fut0.onDone(e);
- else
- fut0.onDone(new ClusterTopologyCheckedException("Failed to send request (node has left): "
- + node.id()));
- }
- catch (IgniteClientDisconnectedCheckedException e0) {
- fut0.onDone(e0);
+ if (X.hasCause(e, IgniteClientDisconnectedCheckedException.class, IgniteClientDisconnectedException.class))
+ fut0.onDone(e);
+ else {
+ try {
+ if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
+ fut0.onDone(e);
+ else
+ fut0.onDone(new ClusterTopologyCheckedException("Failed to send request (node has left): "
+ + node.id()));
+ }
+ catch (IgniteClientDisconnectedCheckedException e0) {
+ fut0.onDone(e0);
+ }
}
}
}
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerCommunicationSpiExceptionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerCommunicationSpiExceptionTest.java
new file mode 100644
index 0000000..03bac2e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamerCommunicationSpiExceptionTest.java
@@ -0,0 +1,143 @@
+/*
+ * 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.UUID;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.IgniteDataStreamer;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.managers.communication.GridIoMessage;
+import org.apache.ignite.lang.IgniteInClosure;
+import org.apache.ignite.lang.IgniteUuid;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.junit.Test;
+
+import static org.apache.ignite.cache.CachePeekMode.ALL;
+
+/**
+ * Tests behavior of DataStreamer when communication channel fails to send a data streamer request due to some reason,
+ * for example, handshake problems.
+ */
+public class DataStreamerCommunicationSpiExceptionTest extends GridCommonAbstractTest {
+ /** Data size. */
+ public static final int DATA_SIZE = 50;
+
+ /** {@inheritDoc} */
+ @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+ IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+ cfg.setDataStreamerThreadPoolSize(1);
+
+ cfg.setCommunicationSpi(new TestCommunicationSpi());
+
+ return cfg;
+ }
+
+ /** {@inheritDoc} */
+ @Override protected void afterTest() throws Exception {
+ super.afterTest();
+
+ stopAllGrids();
+ }
+
+ /**
+ * Tests that flushing data streamer does not hang due to SPI exception on communication layer.
+ * @throws Exception If failed.
+ */
+ @Test
+ public void testSpiOperationTimeoutException() throws Exception {
+ startGrids(2);
+
+ Ignite client = startClientGrid(3);
+
+ IgniteCache<IgniteUuid, Integer> cache = client.createCache(new CacheConfiguration<>(DEFAULT_CACHE_NAME));
+
+ awaitPartitionMapExchange();
+
+ TestCommunicationSpi.spi(client).victim(grid(0).cluster().localNode().id());
+
+ int threadBufSize = 10;
+
+ try (IgniteDataStreamer<Integer, Integer> streamer = client.dataStreamer(DEFAULT_CACHE_NAME)) {
+ streamer.perThreadBufferSize(threadBufSize);
+
+ for (int i = 0; i < DATA_SIZE; i++)
+ streamer.addData(i, i);
+
+ streamer.flush();
+
+ int sz = cache.size(ALL);
+ assertEquals(
+ "Unexpected cache size (data was not flushed) [expected=" + DATA_SIZE + ", actual=" + sz + ']',
+ DATA_SIZE,
+ sz);
+ }
+ }
+
+ /**
+ * Test communication SPI.
+ */
+ public static class TestCommunicationSpi extends TcpCommunicationSpi {
+ /** Node id. */
+ private volatile UUID victim;
+
+ /** Indicates that one data streamer request was blocked. */
+ private final AtomicBoolean firstBlocked = new AtomicBoolean(false);
+
+ /** {@inheritDoc} */
+ @Override public void sendMessage(
+ ClusterNode node,
+ Message msg,
+ IgniteInClosure<IgniteException> ackC
+ ) throws IgniteSpiException {
+ boolean dataStreamerReq = ((GridIoMessage)msg).message() instanceof DataStreamerRequest;
+
+ if (node.id().equals(victim) && dataStreamerReq && firstBlocked.compareAndSet(false, true))
+ throw new IgniteSpiException("Test Spi Exception");
+
+ super.sendMessage(node, msg, ackC);
+ }
+
+ /**
+ * Sets node identifier.
+ *
+ * This {@code id} is used for blocking the first data streamer request to this node and throws test exception.
+ */
+ public void victim(UUID id) {
+ victim = id;
+ }
+
+ /**
+ * Returns instance of TestCommunicationSpi configured for the given {@code node}.
+ *
+ * @param node Ignite instance.
+ * @return Communication SPI.
+ */
+ public static TestCommunicationSpi spi(Ignite node) {
+ return (TestCommunicationSpi)node.configuration().getCommunicationSpi();
+ }
+ }
+}
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 9eac106..31bc4d9 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
@@ -176,6 +176,7 @@ import org.apache.ignite.internal.processors.cache.query.continuous.CacheEntryPr
import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorPersistenceSelfTest;
import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessorSelfTest;
import org.apache.ignite.internal.processors.datastreamer.DataStreamerClientReconnectAfterClusterRestartTest;
+import org.apache.ignite.internal.processors.datastreamer.DataStreamerCommunicationSpiExceptionTest;
import org.apache.ignite.internal.processors.datastreamer.DataStreamerImplSelfTest;
import org.apache.ignite.internal.processors.datastreamer.DataStreamerMultiThreadedSelfTest;
import org.apache.ignite.internal.processors.datastreamer.DataStreamerMultinodeCreateCacheTest;
@@ -287,6 +288,7 @@ public class IgniteCacheTestSuite {
GridTestUtils.addTestIfNeeded(suite, DataStreamerImplSelfTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, DataStreamerTimeoutTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, DataStreamerClientReconnectAfterClusterRestartTest.class, ignoredTests);
+ GridTestUtils.addTestIfNeeded(suite, DataStreamerCommunicationSpiExceptionTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, GridCacheEntryMemorySizeSelfTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, GridCacheClearAllSelfTest.class, ignoredTests);
GridTestUtils.addTestIfNeeded(suite, GridCacheObjectToStringSelfTest.class, ignoredTests);