You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2017/03/21 11:29:52 UTC

[03/33] ignite git commit: IGNITE-4147 - Throw exception on connecting node to cluster with different SSL configuration

IGNITE-4147 - Throw exception on connecting node to cluster with different SSL configuration


Project: http://git-wip-us.apache.org/repos/asf/ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/ignite/commit/1f881aa7
Tree: http://git-wip-us.apache.org/repos/asf/ignite/tree/1f881aa7
Diff: http://git-wip-us.apache.org/repos/asf/ignite/diff/1f881aa7

Branch: refs/heads/ignite-4565-ddl
Commit: 1f881aa70a3894af01135f4cc5e341a8130462c2
Parents: 8874f99
Author: dkarachentsev <dk...@gridgain.com>
Authored: Fri Feb 17 12:34:41 2017 +0300
Committer: dkarachentsev <dk...@gridgain.com>
Committed: Fri Feb 17 12:34:41 2017 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  30 ++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  19 ++
 .../TcpDiscoverySslSecuredUnsecuredTest.java    | 185 +++++++++++++++++++
 .../IgniteSpiDiscoverySelfTestSuite.java        |   2 +
 4 files changed, 235 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/1f881aa7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 932e7d1..95e2cda 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -20,6 +20,7 @@ package org.apache.ignite.spi.discovery.tcp;
 import java.io.BufferedInputStream;
 import java.io.IOException;
 import java.io.InputStream;
+import java.io.StreamCorruptedException;
 import java.net.InetSocketAddress;
 import java.net.Socket;
 import java.net.SocketTimeoutException;
@@ -44,6 +45,7 @@ import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.net.ssl.SSLException;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteException;
@@ -587,6 +589,8 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         int connectAttempts = 1;
 
+        int sslConnectAttempts = 3;
+
         UUID locNodeId = getLocalNodeId();
 
         IgniteSpiOperationTimeoutHelper timeoutHelper = new IgniteSpiOperationTimeoutHelper(spi);
@@ -662,6 +666,22 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                 errs.add(e);
 
+                if (X.hasCause(e, SSLException.class)) {
+                    if (--sslConnectAttempts == 0)
+                        throw new IgniteSpiException("Unable to establish secure connection. " +
+                            "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e);
+
+                    continue;
+                }
+
+                if (X.hasCause(e, StreamCorruptedException.class)) {
+                    if (--sslConnectAttempts == 0)
+                        throw new IgniteSpiException("Unable to establish plain connection. " +
+                            "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e);
+
+                    continue;
+                }
+
                 if (timeoutHelper.checkFailureTimeoutReached(e))
                     break;
 
@@ -1593,7 +1613,15 @@ class ClientImpl extends TcpDiscoveryImpl {
 
             joinCnt++;
 
-            T2<SocketStream, Boolean> joinRes = joinTopology(false, spi.joinTimeout);
+            T2<SocketStream, Boolean> joinRes;
+            try {
+                joinRes = joinTopology(false, spi.joinTimeout);
+            }
+            catch (IgniteSpiException e) {
+                joinError(e);
+
+                return;
+            }
 
             if (joinRes == null) {
                 if (join)

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f881aa7/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index d462ac2..4600be0 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -24,6 +24,7 @@ import java.io.InputStream;
 import java.io.ObjectStreamException;
 import java.io.OutputStream;
 import java.io.Serializable;
+import java.io.StreamCorruptedException;
 import java.net.ConnectException;
 import java.net.InetAddress;
 import java.net.InetSocketAddress;
@@ -1109,6 +1110,8 @@ class ServerImpl extends TcpDiscoveryImpl {
 
         int connectAttempts = 1;
 
+        int sslConnectAttempts = 3;
+
         boolean joinReqSent;
 
         UUID locNodeId = getLocalNodeId();
@@ -1220,6 +1223,22 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 errs.add(e);
 
+                if (X.hasCause(e, SSLException.class)) {
+                    if (--sslConnectAttempts == 0)
+                        throw new IgniteException("Unable to establish secure connection. " +
+                            "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e);
+
+                    continue;
+                }
+
+                if (X.hasCause(e, StreamCorruptedException.class)) {
+                    if (--sslConnectAttempts == 0)
+                        throw new IgniteException("Unable to establish plain connection. " +
+                            "Was remote cluster configured with SSL? [rmtAddr=" + addr + ", errMsg=\"" + e.getMessage() + "\"]", e);
+
+                    continue;
+                }
+
                 if (timeoutHelper.checkFailureTimeoutReached(e))
                     break;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f881aa7/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java
new file mode 100644
index 0000000..ca34f77
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSecuredUnsecuredTest.java
@@ -0,0 +1,185 @@
+/*
+ * 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.spi.discovery.tcp;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.StreamCorruptedException;
+import java.net.Socket;
+import java.util.concurrent.Callable;
+import javax.net.ssl.SSLException;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Tests cases when node connects to cluster with different SSL configuration.
+ * Exception with meaningful message should be thrown.
+ */
+public class TcpDiscoverySslSecuredUnsecuredTest extends GridCommonAbstractTest {
+    /** */
+    private volatile TcpDiscoverySpi spi;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(final String gridName) throws Exception {
+        final IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setClientMode(gridName.contains("client"));
+
+        if (gridName.contains("ssl"))
+            cfg.setSslContextFactory(GridTestUtils.sslFactory());
+
+        if (spi != null) {
+            final TcpDiscoveryIpFinder finder = ((TcpDiscoverySpi)cfg.getDiscoverySpi()).getIpFinder();
+
+            spi.setIpFinder(finder);
+
+            cfg.setDiscoverySpi(spi);
+        }
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSecuredUnsecuredServerConnection() throws Exception {
+        checkConnection("plain-server", "ssl-server");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUnsecuredSecuredServerConnection() throws Exception {
+        checkConnection("ssl-server", "plain-server");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSecuredClientUnsecuredServerConnection() throws Exception {
+        checkConnection("plain-server", "ssl-client");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testUnsecuredClientSecuredServerConnection() throws Exception {
+        checkConnection("ssl-server", "plain-client");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPlainServerNodesRestart() throws Exception {
+        checkNodesRestart("plain-server-1", "plain-server-2");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSslServerNodesRestart() throws Exception {
+        checkNodesRestart("ssl-server-1", "ssl-server-2");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testPlainClientNodesRestart() throws Exception {
+        checkNodesRestart("plain-server", "plain-client");
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testSslClientNodesRestart() throws Exception {
+        checkNodesRestart("ssl-server", "ssl-client");
+    }
+
+    /**
+     * @param name1 First grid name.
+     * @param name2 Second grid name.
+     * @throws Exception If failed.
+     */
+    private void checkNodesRestart(String name1, String name2) throws Exception {
+        startGrid(name1);
+
+        spi = new FailDiscoverySpi(!name1.contains("ssl"));
+
+        startGrid(name2);
+    }
+
+    /**
+     * @param name1 First grid name.
+     * @param name2 Second grid name.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ThrowableNotThrown")
+    private void checkConnection(final String name1, final String name2) throws Exception {
+        startGrid(name1);
+
+        GridTestUtils.assertThrows(null, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                startGrid(name2);
+
+                return null;
+            }
+        }, IgniteCheckedException.class, null);
+    }
+
+    /**
+     *
+     */
+    private class FailDiscoverySpi extends TcpDiscoverySpi {
+        /** */
+        private int cnt = 1;
+
+        /** */
+        private final boolean plain;
+
+        /**
+         * @param plain Plain conection flag.
+         */
+        private FailDiscoverySpi(final boolean plain) {
+            this.plain = plain;
+        }
+
+        /** {@inheritDoc} */
+        @Override protected <T> T readMessage(final Socket sock, @Nullable final InputStream in,
+            final long timeout) throws IOException, IgniteCheckedException {
+            if (cnt-- > 0) {
+                if (plain)
+                    throw new StreamCorruptedException("Test exception");
+                else
+                    throw new SSLException("Test SSL exception");
+            }
+
+            return super.readMessage(sock, in, timeout);
+        }
+    }
+
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/1f881aa7/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
index 548e1a5..e6b39f7 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiDiscoverySelfTestSuite.java
@@ -35,6 +35,7 @@ import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiConfigSelfTest;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiFailureTimeoutSelfTest;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiSelfTest;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpiStartStopSelfTest;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSecuredUnsecuredTest;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySslSelfTest;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinderSelfTest;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinderSelfTest;
@@ -89,6 +90,7 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
 
         // SSL.
         suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class));
+        suite.addTest(new TestSuite(TcpDiscoverySslSecuredUnsecuredTest.class));
 
         return suite;
     }