You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by dg...@apache.org on 2019/05/24 08:36:22 UTC

[ignite] branch master updated: IGNITE-11865 FailureProcessor treats tcp-comm-worker as blocked when it works on reestablishing connect to failed client node

This is an automated email from the ASF dual-hosted git repository.

dgovorukhin 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 d2a085a  IGNITE-11865 FailureProcessor treats tcp-comm-worker as blocked when it works on reestablishing connect to failed client node
d2a085a is described below

commit d2a085acd5d26a2e8d1184a359b481d999bbf6c3
Author: Sergey Chugunov <se...@gmail.com>
AuthorDate: Fri May 24 11:35:59 2019 +0300

    IGNITE-11865 FailureProcessor treats tcp-comm-worker as blocked when it works on reestablishing connect to failed client node
    
    Signed-off-by: Dmitriy Govorukhin <dm...@gmail.com>
---
 .../client/suite/IgniteClientTestSuite.java        |   3 +
 .../spi/communication/tcp/TcpCommunicationSpi.java |   5 +
 .../ignite/internal/IgniteClientFailuresTest.java  | 161 +++++++++++++++++++++
 3 files changed, 169 insertions(+)

diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
index 1534975..c48c541 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/client/suite/IgniteClientTestSuite.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.client.suite;
 
+import org.apache.ignite.internal.IgniteClientFailuresTest;
 import org.apache.ignite.internal.TaskEventSubjectIdSelfTest;
 import org.apache.ignite.internal.client.ClientDefaultCacheSelfTest;
 import org.apache.ignite.internal.client.ClientReconnectionSelfTest;
@@ -167,6 +168,8 @@ import org.junit.runners.Suite;
 
     // SSL params.
     ClientSslParametersTest.class,
+
+    IgniteClientFailuresTest.class
 })
 public class IgniteClientTestSuite {
 }
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 79e705f..29e0d21 100755
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -3595,6 +3595,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter implements Communicati
                         break;
                     }
                 }
+
+                CommunicationWorker commWorker0 = commWorker;
+
+                if (commWorker0 != null && commWorker0.runner() == Thread.currentThread())
+                    commWorker0.updateHeartbeat();
             }
 
             if (ses != null)
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientFailuresTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientFailuresTest.java
new file mode 100644
index 0000000..3f01702
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteClientFailuresTest.java
@@ -0,0 +1,161 @@
+/*
+ * 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;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.cluster.IgniteClusterEx;
+import org.apache.ignite.internal.managers.GridManagerAdapter;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.GridStringLogger;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.apache.ignite.testframework.junits.logger.GridTestLog4jLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ *
+ */
+public class IgniteClientFailuresTest extends GridCommonAbstractTest {
+    /** */
+    private boolean clientMode;
+
+    /** */
+    private GridStringLogger inMemoryLog;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(igniteInstanceName);
+
+        cfg.setClientMode(clientMode);
+
+        if (!clientMode) {
+            cfg.setClientFailureDetectionTimeout(10_000);
+
+            cfg.setSystemWorkerBlockedTimeout(5_000);
+
+            cfg.setGridLogger(inMemoryLog);
+        }
+
+        return cfg;
+    }
+
+    /** */
+    @Before
+    public void setupClientFailuresTest() {
+        stopAllGrids();
+    }
+
+    /** */
+    @After
+    public void tearDownClientFailuresTest() {
+        stopAllGrids();
+    }
+
+    /**
+     * Test verifies that FailureProcessor doesn't treat tcp-comm-worker thread as blocked when
+     * the thread handles situation of failed client node and thus doesn't print full thread dump into logs.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testNoMessagesFromFailureProcessor() throws Exception {
+        inMemoryLog = new GridStringLogger(false, new GridTestLog4jLogger());
+
+        inMemoryLog.logLength(1024 * 1024);
+
+        IgniteEx srv = startGrid(0);
+
+        clientMode = true;
+
+        IgniteEx client00 = startGrid("client00");
+
+        client00.getOrCreateCache(new CacheConfiguration<>("cache0"));
+
+        breakClient(client00);
+
+        boolean waitRes = GridTestUtils.waitForCondition(() -> {
+            IgniteClusterEx cl = srv.cluster();
+
+            return (cl.topology(cl.topologyVersion()).size() == 1);
+        }, 30_000);
+
+        assertTrue(waitRes);
+
+        assertFalse(inMemoryLog.toString().contains("name=tcp-comm-worker"));
+    }
+
+    /**
+     * Test verifies that when client node failed but not yet cleaned up from topology (because {@link IgniteConfiguration#clientFailureDetectionTimeout} has not been reached yet)
+     * it doesn't affect new client connected from the same address.
+     *
+     * @throws Exception If failed.
+     */
+    @Test
+    public void testFailedClientLeavesTopologyAfterTimeout() throws Exception {
+        IgniteEx srv0 = startGrid(0);
+
+        clientMode = true;
+
+        IgniteEx client00 = startGrid("client00");
+
+        Thread.sleep(5_000);
+
+        client00.getOrCreateCache(new CacheConfiguration<>("cache0"));
+
+        breakClient(client00);
+
+        final IgniteClusterEx cl = srv0.cluster();
+
+        assertEquals(2, cl.topology(cl.topologyVersion()).size());
+
+        IgniteEx client01 = startGrid("client01");
+
+        assertEquals(3, cl.topology(cl.topologyVersion()).size());
+
+        boolean waitRes = GridTestUtils.waitForCondition(() -> (cl.topology(cl.topologyVersion()).size() == 2),
+            20_000);
+
+        checkCacheOperations(client01.cache("cache0"));
+
+        assertTrue(waitRes);
+    }
+
+    /** */
+    private void checkCacheOperations(IgniteCache cache) {
+        for (int i = 0; i < 100; i++)
+            cache.put(i, i);
+
+        for (int i = 0; i < 100; i++)
+            assertEquals(i, cache.get(i));
+    }
+
+    /** */
+    private void breakClient(IgniteEx client) {
+        Object discoSpi = ((Object[])GridTestUtils.getFieldValue(client.context().discovery(), GridManagerAdapter.class, "spis"))[0];
+
+        Object commSpi = ((Object[])GridTestUtils.getFieldValue(client.context().io(), GridManagerAdapter.class, "spis"))[0];
+
+        ((TcpCommunicationSpi)commSpi).simulateNodeFailure();
+
+        ((TcpDiscoverySpi)discoSpi).simulateNodeFailure();
+    }
+}