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);