You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by al...@apache.org on 2019/05/23 14:01:53 UTC

[ignite] branch master updated: IGNITE-11671 Thin client: Client may hang when connected to a starting server - Fixes #6550.

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

alexpl 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 3c56d0a  IGNITE-11671 Thin client: Client may hang when connected to a starting server - Fixes #6550.
3c56d0a is described below

commit 3c56d0a00354bf31589c232b8e4888fef58b058e
Author: Aleksey Plekhanov <pl...@gmail.com>
AuthorDate: Thu May 23 17:00:50 2019 +0300

    IGNITE-11671 Thin client: Client may hang when connected to a starting server - Fixes #6550.
---
 .../processors/odbc/ClientListenerProcessor.java   | 12 ++-
 .../ignite/internal/util/nio/GridNioServer.java    | 10 +++
 .../ignite/client/ConnectToStartingNodeTest.java   | 90 ++++++++++++++++++++++
 .../org/apache/ignite/client/ClientTestSuite.java  |  3 +-
 4 files changed, 111 insertions(+), 4 deletions(-)

diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java
index c988f33..8e72a28 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/ClientListenerProcessor.java
@@ -164,8 +164,6 @@ public class ClientListenerProcessor extends GridProcessorAdapter {
                             .idleTimeout(idleTimeout > 0 ? idleTimeout : Long.MAX_VALUE)
                             .build();
 
-                        srv0.start();
-
                         srv = srv0;
 
                         ctx.ports().registerPort(port, IgnitePortProtocol.TCP, getClass());
@@ -200,6 +198,14 @@ public class ClientListenerProcessor extends GridProcessorAdapter {
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public void onKernalStart(boolean active) throws IgniteCheckedException {
+        super.onKernalStart(active);
+
+        if (srv != null)
+            srv.start();
+    }
+
     /**
      * Register an Ignite MBean for managing clients connections.
      */
@@ -592,4 +598,4 @@ public class ClientListenerProcessor extends GridProcessorAdapter {
             return sb.append(']').toString();
         }
     }
-}
\ No newline at end of file
+}
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index 33f328b..4584c87 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -2880,6 +2880,16 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
+        @Override public void cancel() {
+            super.cancel();
+
+            // If accept worker never was started then explicitly close selector, otherwise selector will be closed
+            // in finally block when workers thread will be stopped.
+            if (runner() == null)
+                closeSelector();
+        }
+
+        /** {@inheritDoc} */
         @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
             Throwable err = null;
 
diff --git a/modules/core/src/test/java/org/apache/ignite/client/ConnectToStartingNodeTest.java b/modules/core/src/test/java/org/apache/ignite/client/ConnectToStartingNodeTest.java
new file mode 100644
index 0000000..f79928d
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/client/ConnectToStartingNodeTest.java
@@ -0,0 +1,90 @@
+/*
+ * 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.client;
+
+import java.util.concurrent.Callable;
+import java.util.concurrent.CyclicBarrier;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.Ignition;
+import org.apache.ignite.configuration.ClientConfiguration;
+import org.apache.ignite.configuration.ClientConnectorConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.spi.IgniteSpiException;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+import org.junit.Test;
+
+/**
+ * Checks that connection with starting node will be established correctly.
+ */
+public class ConnectToStartingNodeTest extends GridCommonAbstractTest {
+    /** Client connector address. */
+    private static final String CLIENT_CONN_ADDR = "127.0.0.1:" + ClientConnectorConfiguration.DFLT_PORT;
+
+    /** Barrier to suspend discovery SPI start. */
+    private final CyclicBarrier barrier = new CyclicBarrier(2);
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String igniteInstanceName) throws Exception {
+        return super.getConfiguration(igniteInstanceName).setDiscoverySpi(new TcpDiscoverySpi() {
+            @Override public void spiStart(@Nullable String igniteInstanceName) throws IgniteSpiException {
+                try {
+                    barrier.await();
+                }
+                catch (Exception ignore) {
+                    // No-op.
+                }
+
+                super.spiStart(igniteInstanceName);
+
+                try {
+                    barrier.await();
+                }
+                catch (Exception ignore) {
+                    // No-op.
+                }
+            }
+        });
+    }
+
+    /**
+     * Test that client can't connect to server before discovery SPI start.
+     */
+    @Test
+    public void testClientConnectBeforeDiscoveryStart() throws Exception {
+        IgniteInternalFuture<Ignite> futStartGrid = GridTestUtils.runAsync((Callable<Ignite>)this::startGrid);
+
+        barrier.await();
+
+        IgniteInternalFuture<IgniteClient> futStartClient = GridTestUtils.runAsync(
+            () -> Ignition.startClient(new ClientConfiguration().setAddresses(CLIENT_CONN_ADDR)));
+
+        // Server doesn't accept connection before discovery SPI started.
+        assertFalse(GridTestUtils.waitForCondition(futStartClient::isDone, 500L));
+
+        barrier.await();
+
+        futStartGrid.get();
+
+        // Server accept connection after discovery SPI started.
+        assertTrue(GridTestUtils.waitForCondition(futStartClient::isDone, 500L));
+    }
+}
diff --git a/modules/indexing/src/test/java/org/apache/ignite/client/ClientTestSuite.java b/modules/indexing/src/test/java/org/apache/ignite/client/ClientTestSuite.java
index b83a813..0558b0a 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/client/ClientTestSuite.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/client/ClientTestSuite.java
@@ -35,7 +35,8 @@ import org.junit.runners.Suite;
     FunctionalQueryTest.class,
     IgniteBinaryQueryTest.class,
     SslParametersTest.class,
-    ConnectionTest.class
+    ConnectionTest.class,
+    ConnectToStartingNodeTest.class
 })
 public class ClientTestSuite {
     // No-op.