You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vk...@apache.org on 2015/07/27 22:10:28 UTC

[03/50] [abbrv] incubator-ignite git commit: Cherry pick form IGNITE-323.

Cherry pick form IGNITE-323.


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

Branch: refs/heads/ignite-104
Commit: 1fd31892c7a87ecd1c5584b45aecd0b30e3411fb
Parents: 9cab55a
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Tue Jul 21 21:26:09 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Tue Jul 21 21:26:09 2015 +0300

----------------------------------------------------------------------
 .../configuration/ConnectorConfiguration.java   |  32 ++
 .../configuration/IgniteConfiguration.java      |  28 ++
 .../apache/ignite/internal/IgniteKernal.java    |   3 +-
 .../client/ssl/GridSslBasicContextFactory.java  |   3 +
 .../client/ssl/GridSslContextFactory.java       |   5 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |  17 +-
 .../util/nio/GridNioSessionMetaKey.java         |   5 +-
 .../util/nio/ssl/BlockingSslHandler.java        | 473 +++++++++++++++++++
 .../internal/util/nio/ssl/GridNioSslFilter.java |   2 +
 .../util/nio/ssl/GridNioSslHandler.java         |  12 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 158 ++++++-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   5 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  34 +-
 .../apache/ignite/ssl/SslContextFactory.java    | 455 ++++++++++++++++++
 .../org/apache/ignite/ssl/package-info.java     |  22 +
 .../GridAbstractCommunicationSelfTest.java      |  13 +
 .../tcp/GridTcpCommunicationSpiSslSelfTest.java |  38 ++
 .../discovery/AbstractDiscoverySelfTest.java    |  13 +
 .../tcp/TcpDiscoverySpiSslSelfTest.java         |  28 ++
 .../discovery/tcp/TcpDiscoverySslSelfTest.java  |  42 ++
 .../ignite/testframework/GridTestUtils.java     |  20 +
 .../ignite/testframework/junits/IgniteMock.java |  13 +
 .../IgniteSpiDiscoverySelfTestSuite.java        |   3 +
 23 files changed, 1394 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java
index 970c54b..a518cd8 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/ConnectorConfiguration.java
@@ -19,8 +19,12 @@ package org.apache.ignite.configuration;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.client.ssl.*;
+import org.apache.ignite.ssl.*;
+
 import org.jetbrains.annotations.*;
 
+import javax.cache.configuration.*;
+import javax.net.ssl.*;
 import java.net.*;
 
 /**
@@ -99,6 +103,9 @@ public class ConnectorConfiguration {
     /** SSL context factory for rest binary server. */
     private GridSslContextFactory sslCtxFactory;
 
+    /** SSL context factory for rest binary server. */
+    private Factory<SSLContext> sslFactory;
+
     /** Port range */
     private int portRange = DFLT_PORT_RANGE;
 
@@ -429,7 +436,9 @@ public class ConnectorConfiguration {
      *
      * @return SslContextFactory instance.
      * @see GridSslContextFactory
+     * @deprecated Use {@link #getSslFactory()} instead.
      */
+    @Deprecated
     public GridSslContextFactory getSslContextFactory() {
         return sslCtxFactory;
     }
@@ -440,12 +449,35 @@ public class ConnectorConfiguration {
      * {@link #setSslEnabled(boolean)} is set to {@code true}.
      *
      * @param sslCtxFactory Instance of {@link GridSslContextFactory}
+     * @deprecated Use {@link #setSslFactory(Factory)} instead.
      */
+    @Deprecated
     public void setSslContextFactory(GridSslContextFactory sslCtxFactory) {
         this.sslCtxFactory = sslCtxFactory;
     }
 
     /**
+     * Gets context factory that will be used for creating a secure socket layer of rest binary server.
+     *
+     * @return SSL context factory instance.
+     * @see SslContextFactory
+     */
+    public Factory<SSLContext> getSslFactory() {
+        return sslFactory;
+    }
+
+    /**
+     * Sets instance of {@link Factory<SSLContext>} that will be used to create an instance of {@code SSLContext}
+     * for Secure Socket Layer on TCP binary protocol. This factory will only be used if
+     * {@link #setSslEnabled(boolean)} is set to {@code true}.
+     *
+     * @param sslFactory Instance of {@link Factory<SSLContext>}
+     */
+    public void setSslFactory(Factory<SSLContext> sslFactory) {
+        this.sslFactory = sslFactory;
+    }
+
+    /**
      * Gets number of ports to try if configured port is already in use.
      *
      * @return Number of ports to try.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 2d36c7a..54ef2ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -52,6 +52,7 @@ import org.apache.ignite.spi.loadbalancing.*;
 import org.apache.ignite.spi.loadbalancing.roundrobin.*;
 import org.apache.ignite.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
+import org.apache.ignite.ssl.*;
 
 import javax.cache.configuration.*;
 import javax.cache.event.*;
@@ -59,6 +60,7 @@ import javax.cache.expiry.*;
 import javax.cache.integration.*;
 import javax.cache.processor.*;
 import javax.management.*;
+import javax.net.ssl.*;
 import java.lang.management.*;
 import java.util.*;
 
@@ -400,6 +402,9 @@ public class IgniteConfiguration {
     /** Cache store session listeners. */
     private Factory<CacheStoreSessionListener>[] storeSesLsnrs;
 
+    /** SSL connection factory. */
+    private Factory<SSLContext> sslCtxFactory;
+
     /**
      * Creates valid grid configuration with all default values.
      */
@@ -480,6 +485,7 @@ public class IgniteConfiguration {
         segResolvers = cfg.getSegmentationResolvers();
         sndRetryCnt = cfg.getNetworkSendRetryCount();
         sndRetryDelay = cfg.getNetworkSendRetryDelay();
+        sslCtxFactory = cfg.getSslContextFactory();
         storeSesLsnrs = cfg.getCacheStoreSessionListenerFactories();
         svcCfgs = cfg.getServiceConfiguration();
         sysPoolSize = cfg.getSystemThreadPoolSize();
@@ -1310,6 +1316,28 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Sets SSL context factory that will be used for creating a secure socket  layer.
+     *
+     * @param sslCtxFactory Ssl context factory.
+     * @see SslContextFactory
+     */
+    public IgniteConfiguration setSslContextFactory(Factory<SSLContext> sslCtxFactory) {
+        this.sslCtxFactory = sslCtxFactory;
+
+        return this;
+    }
+
+    /**
+     * Returns SSL context factory that will be used for creating a secure socket layer.
+     *
+     * @return SSL connection factory.
+     * @see SslContextFactory
+     */
+    public Factory<SSLContext> getSslContextFactory() {
+        return sslCtxFactory;
+    }
+
+    /**
      * Should return fully configured event SPI implementation. If not provided,
      * {@link MemoryEventStorageSpi} will be used.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 024dc7b..8a246dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -2065,7 +2065,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         assert log != null;
 
         if (log.isInfoEnabled())
-            log.info("Security status [authentication=" + onOff(ctx.security().enabled()) + ']');
+            log.info("Security status [authentication=" + onOff(ctx.security().enabled())
+                + ", communication encrypted=" + onOff(ctx.config().getSslContextFactory() != null) + ']');
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslBasicContextFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslBasicContextFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslBasicContextFactory.java
index 92c34e0..d9a1f9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslBasicContextFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslBasicContextFactory.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.client.ssl;
 
 import org.apache.ignite.internal.util.typedef.internal.*;
 
+import javax.cache.configuration.*;
 import javax.net.ssl.*;
 import java.io.*;
 import java.security.*;
@@ -37,7 +38,9 @@ import java.util.*;
  *     factory.setTrustManagers(GridSslBasicContextFactory.getDisabledTrustManager());
  *     // Rest of initialization.
  * </pre>
+ * @deprecated Use {@link Factory<SSLContext>} instead.
  */
+@Deprecated
 public class GridSslBasicContextFactory implements GridSslContextFactory {
     /** Default key store type. */
     public static final String DFLT_STORE_TYPE = "JKS";

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslContextFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslContextFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslContextFactory.java
index 8a9cd13..7ae1995 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslContextFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/ssl/GridSslContextFactory.java
@@ -17,14 +17,17 @@
 
 package org.apache.ignite.internal.client.ssl;
 
+import javax.cache.configuration.*;
 import javax.net.ssl.*;
 
 /**
  * This interface provides creation of SSL context both for server and client use.
  * <p>
  * Usually, it is enough to configure context from a particular key and trust stores, this functionality is provided
- * in {@link org.apache.ignite.internal.client.ssl.GridSslBasicContextFactory}.
+ * in {@link GridSslBasicContextFactory}.
+ * @deprecated Use {@link Factory<SSLContext>} instead.
  */
+@Deprecated
 public interface GridSslContextFactory {
     /**
      * Creates SSL context based on factory settings.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
index 9b6f353..c37c17d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
@@ -35,6 +35,7 @@ import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.spi.*;
 import org.jetbrains.annotations.*;
 
+import javax.cache.configuration.*;
 import javax.net.ssl.*;
 import java.io.*;
 import java.net.*;
@@ -106,13 +107,23 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
             SSLContext sslCtx = null;
 
             if (cfg.isSslEnabled()) {
-                GridSslContextFactory factory = cfg.getSslContextFactory();
+                Factory<SSLContext> igniteFactory = ctx.config().getSslContextFactory();
 
-                if (factory == null)
+                Factory<SSLContext> factory = cfg.getSslFactory();
+
+                // This factory deprecated and will be removed.
+                GridSslContextFactory depFactory = cfg.getSslContextFactory();
+
+                if (factory == null && depFactory == null && igniteFactory == null)
                     // Thrown SSL exception instead of IgniteCheckedException for writing correct warning message into log.
                     throw new SSLException("SSL is enabled, but SSL context factory is not specified.");
 
-                sslCtx = factory.createSslContext();
+                if (factory != null)
+                    sslCtx = factory.create();
+                else if (depFactory != null)
+                    sslCtx = depFactory.createSslContext();
+                else
+                    sslCtx = igniteFactory.create();
             }
 
             int lastPort = cfg.getPort() + cfg.getPortRange() - 1;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java
index d7eb2f3..004c327 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionMetaKey.java
@@ -42,7 +42,10 @@ public enum GridNioSessionMetaKey {
     MARSHALLER_ID,
 
     /** Message writer. */
-    MSG_WRITER;
+    MSG_WRITER,
+
+    /** SSL engine. */
+    SSL_ENGINE;
 
     /** Maximum count of NIO session keys in system. */
     public static final int MAX_KEYS_CNT = 64;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
new file mode 100644
index 0000000..eee90d8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/BlockingSslHandler.java
@@ -0,0 +1,473 @@
+/*
+ * 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.util.nio.ssl;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.nio.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import javax.net.ssl.*;
+import javax.net.ssl.SSLEngineResult.*;
+import java.io.*;
+import java.nio.*;
+import java.nio.channels.*;
+
+import static javax.net.ssl.SSLEngineResult.HandshakeStatus.*;
+import static javax.net.ssl.SSLEngineResult.Status.*;
+import static org.apache.ignite.internal.util.nio.ssl.GridNioSslFilter.*;
+
+/**
+ *
+ */
+public class BlockingSslHandler {
+    /** Logger. */
+    private IgniteLogger log;
+
+    /** */
+    private SocketChannel ch;
+
+    /** */
+    private GridFutureAdapter<ByteBuffer> fut;
+
+    /** SSL engine. */
+    private SSLEngine sslEngine;
+
+    /** Handshake completion flag. */
+    private boolean handshakeFinished;
+
+    /** Engine handshake status. */
+    private HandshakeStatus handshakeStatus;
+
+    /** Output buffer into which encrypted data will be written. */
+    private ByteBuffer outNetBuf;
+
+    /** Input buffer from which SSL engine will decrypt data. */
+    private ByteBuffer inNetBuf;
+
+    /** Empty buffer used in handshake procedure.  */
+    private ByteBuffer handshakeBuf = ByteBuffer.allocate(0);
+
+    /** Application buffer. */
+    private ByteBuffer appBuf;
+
+    /**
+     * @param sslEngine SSLEngine.
+     * @param ch Socket channel.
+     * @param fut Future.
+     * @param log Logger.
+     */
+    public BlockingSslHandler(SSLEngine sslEngine, SocketChannel ch, GridFutureAdapter<ByteBuffer> fut,
+        IgniteLogger log) throws SSLException {
+        this.ch = ch;
+        this.fut = fut;
+        this.log = log;
+
+        this.sslEngine = sslEngine;
+
+        // Allocate a little bit more so SSL engine would not return buffer overflow status.
+        int netBufSize = sslEngine.getSession().getPacketBufferSize() + 50;
+
+        outNetBuf = ByteBuffer.allocate(netBufSize);
+        inNetBuf = ByteBuffer.allocate(netBufSize);
+
+        // Initially buffer is empty.
+        outNetBuf.position(0);
+        outNetBuf.limit(0);
+
+        appBuf = allocateAppBuff();
+
+        handshakeStatus = sslEngine.getHandshakeStatus();
+
+        sslEngine.setUseClientMode(true);
+
+        if (log.isDebugEnabled())
+            log.debug("Started SSL session [netBufSize=" + netBufSize + ", appBufSize=" + appBuf.capacity() + ']');
+    }
+
+    /**
+     * Performs handshake procedure with remote peer.
+     *
+     * @throws GridNioException If filter processing has thrown an exception.
+     * @throws SSLException If failed to process SSL data.
+     */
+    public boolean handshake() throws IgniteCheckedException, SSLException {
+        if (log.isDebugEnabled())
+            log.debug("Entered handshake. Handshake status: " + handshakeStatus + '.');
+
+        sslEngine.beginHandshake();
+
+        handshakeStatus = sslEngine.getHandshakeStatus();
+
+        boolean loop = true;
+
+        while (loop) {
+            switch (handshakeStatus) {
+                case NOT_HANDSHAKING:
+                case FINISHED: {
+                    handshakeFinished = true;
+
+                    if (fut != null) {
+                        appBuf.flip();
+
+                        fut.onDone(appBuf);
+                    }
+
+                    loop = false;
+
+                    break;
+                }
+
+                case NEED_TASK: {
+                    handshakeStatus = runTasks();
+
+                    break;
+                }
+
+                case NEED_UNWRAP: {
+                    Status status = unwrapHandshake();
+
+                    handshakeStatus = sslEngine.getHandshakeStatus();
+
+                    if (status == BUFFER_UNDERFLOW && sslEngine.isInboundDone())
+                        // Either there is no enough data in buffer or session was closed.
+                        loop = false;
+
+                    break;
+                }
+
+                case NEED_WRAP: {
+                    // If the output buffer has remaining data, clear it.
+                    if (outNetBuf.hasRemaining())
+                        U.warn(log, "Output net buffer has unsent bytes during handshake (will clear). ");
+
+                    outNetBuf.clear();
+
+                    SSLEngineResult res = sslEngine.wrap(handshakeBuf, outNetBuf);
+
+                    outNetBuf.flip();
+
+                    handshakeStatus = res.getHandshakeStatus();
+
+                    if (log.isDebugEnabled())
+                        log.debug("Wrapped handshake data [status=" + res.getStatus() + ", handshakeStatus=" +
+                        handshakeStatus + ']');
+
+                    writeNetBuffer();
+
+                    break;
+                }
+
+                default: {
+                    throw new IllegalStateException("Invalid handshake status in handshake method [handshakeStatus=" +
+                        handshakeStatus + ']');
+                }
+            }
+        }
+
+        if (log.isDebugEnabled())
+            log.debug("Leaved handshake. Handshake status:" + handshakeStatus + '.');
+
+        return handshakeFinished;
+    }
+
+    /**
+     * Encrypts data to be written to the network.
+     *
+     * @param src data to encrypt.
+     * @throws SSLException on errors.
+     * @return Output buffer with encrypted data.
+     */
+    public ByteBuffer encrypt(ByteBuffer src) throws SSLException {
+        assert handshakeFinished;
+
+        // The data buffer is (must be) empty, we can reuse the entire
+        // buffer.
+        outNetBuf.clear();
+
+        // Loop until there is no more data in src
+        while (src.hasRemaining()) {
+            int outNetRemaining = outNetBuf.capacity() - outNetBuf.position();
+
+            if (outNetRemaining < src.remaining() * 2) {
+                outNetBuf = expandBuffer(outNetBuf, Math.max(
+                    outNetBuf.position() + src.remaining() * 2, outNetBuf.capacity() * 2));
+
+                if (log.isDebugEnabled())
+                    log.debug("Expanded output net buffer: " + outNetBuf.capacity());
+            }
+
+            SSLEngineResult res = sslEngine.wrap(src, outNetBuf);
+
+            if (log.isDebugEnabled())
+                log.debug("Encrypted data [status=" + res.getStatus() + ", handshakeStaus=" +
+                    res.getHandshakeStatus() + ']');
+
+            if (res.getStatus() == OK) {
+                if (res.getHandshakeStatus() == NEED_TASK)
+                    runTasks();
+            }
+            else
+                throw new SSLException("Failed to encrypt data (SSL engine error) [status=" + res.getStatus() +
+                    ", handshakeStatus=" + res.getHandshakeStatus() + ']');
+        }
+
+        outNetBuf.flip();
+
+        return outNetBuf;
+    }
+
+    /**
+     * Called by SSL filter when new message was received.
+     *
+     * @param buf Received message.
+     * @throws GridNioException If exception occurred while forwarding events to underlying filter.
+     * @throws SSLException If failed to process SSL data.
+     */
+    public ByteBuffer decode(ByteBuffer buf) throws IgniteCheckedException, SSLException {
+        inNetBuf.clear();
+
+        if (buf.limit() > inNetBuf.remaining()) {
+            inNetBuf = expandBuffer(inNetBuf, inNetBuf.capacity() + buf.limit() * 2);
+
+            appBuf = expandBuffer(appBuf, inNetBuf.capacity() * 2);
+
+            if (log.isDebugEnabled())
+                log.debug("Expanded buffers [inNetBufCapacity=" + inNetBuf.capacity() + ", appBufCapacity=" +
+                    appBuf.capacity() + ']');
+        }
+
+        // append buf to inNetBuffer
+        inNetBuf.put(buf);
+
+        if (!handshakeFinished)
+            handshake();
+        else
+            unwrapData();
+
+        if (isInboundDone()) {
+            int newPosition = buf.position() - inNetBuf.position();
+
+            if (newPosition >= 0) {
+                buf.position(newPosition);
+
+                // If we received close_notify but not all bytes has been read by SSL engine, print a warning.
+                if (buf.hasRemaining())
+                    U.warn(log, "Got unread bytes after receiving close_notify message (will ignore).");
+            }
+
+            inNetBuf.clear();
+        }
+
+        appBuf.flip();
+
+        return appBuf;
+    }
+
+    /**
+     * @return {@code True} if inbound data stream has ended, i.e. SSL engine received
+     * <tt>close_notify</tt> message.
+     */
+    boolean isInboundDone() {
+        return sslEngine.isInboundDone();
+    }
+
+    /**
+     * Unwraps user data to the application buffer.
+     *
+     * @throws SSLException If failed to process SSL data.
+     * @throws GridNioException If failed to pass events to the next filter.
+     */
+    private void unwrapData() throws IgniteCheckedException, SSLException {
+        if (log.isDebugEnabled())
+            log.debug("Unwrapping received data.");
+
+        // Flip buffer so we can read it.
+        inNetBuf.flip();
+
+        SSLEngineResult res = unwrap0();
+
+        // prepare to be written again
+        inNetBuf.compact();
+
+        checkStatus(res);
+
+        renegotiateIfNeeded(res);
+    }
+
+    /**
+     * Runs all tasks needed to continue SSL work.
+     *
+     * @return Handshake status after running all tasks.
+     */
+    private HandshakeStatus runTasks() {
+        Runnable runnable;
+
+        while ((runnable = sslEngine.getDelegatedTask()) != null) {
+            if (log.isDebugEnabled())
+                log.debug("Running SSL engine task: " + runnable + '.');
+
+            runnable.run();
+        }
+
+        if (log.isDebugEnabled())
+            log.debug("Finished running SSL engine tasks. HandshakeStatus: " + sslEngine.getHandshakeStatus());
+
+        return sslEngine.getHandshakeStatus();
+    }
+
+
+    /**
+     * Unwraps handshake data and processes it.
+     *
+     * @return Status.
+     * @throws SSLException If SSL exception occurred while unwrapping.
+     * @throws GridNioException If failed to pass event to the next filter.
+     */
+    private Status unwrapHandshake() throws SSLException, IgniteCheckedException {
+        // Flip input buffer so we can read the collected data.
+        readFromNet();
+
+        inNetBuf.flip();
+
+        SSLEngineResult res = unwrap0();
+        handshakeStatus = res.getHandshakeStatus();
+
+        checkStatus(res);
+
+        // If handshake finished, no data was produced, and the status is still ok,
+        // try to unwrap more
+        if (handshakeStatus == FINISHED && res.getStatus() == OK && inNetBuf.hasRemaining()) {
+            res = unwrap0();
+
+            handshakeStatus = res.getHandshakeStatus();
+
+            // prepare to be written again
+            inNetBuf.compact();
+
+            renegotiateIfNeeded(res);
+        }
+        else
+            // prepare to be written again
+            inNetBuf.compact();
+
+        return res.getStatus();
+    }
+
+    /**
+     * Performs raw unwrap from network read buffer.
+     *
+     * @return Result.
+     * @throws SSLException If SSL exception occurs.
+     */
+    private SSLEngineResult unwrap0() throws SSLException {
+        SSLEngineResult res;
+
+        do {
+            res = sslEngine.unwrap(inNetBuf, appBuf);
+
+            if (log.isDebugEnabled())
+                log.debug("Unwrapped raw data [status=" + res.getStatus() + ", handshakeStatus=" +
+                    res.getHandshakeStatus() + ']');
+
+            if (res.getStatus() == Status.BUFFER_OVERFLOW)
+                appBuf = expandBuffer(appBuf, appBuf.capacity() * 2);
+        }
+        while ((res.getStatus() == OK || res.getStatus() == Status.BUFFER_OVERFLOW) &&
+            (handshakeFinished && res.getHandshakeStatus() == NOT_HANDSHAKING
+                || res.getHandshakeStatus() == NEED_UNWRAP));
+
+        return res;
+    }
+
+    /**
+     * @param res SSL engine result.
+     * @throws SSLException If status is not acceptable.
+     */
+    private void checkStatus(SSLEngineResult res)
+        throws SSLException {
+
+        Status status = res.getStatus();
+
+        if (status != OK && status != CLOSED && status != BUFFER_UNDERFLOW)
+            throw new SSLException("Failed to unwrap incoming data (SSL engine error). Status: " + status);
+    }
+
+    /**
+     * Check status and retry the negotiation process if needed.
+     *
+     * @param res Result.
+     * @throws GridNioException If exception occurred during handshake.
+     * @throws SSLException If failed to process SSL data
+     */
+    private void renegotiateIfNeeded(SSLEngineResult res) throws IgniteCheckedException, SSLException {
+        if (res.getStatus() != CLOSED && res.getStatus() != BUFFER_UNDERFLOW
+            && res.getHandshakeStatus() != NOT_HANDSHAKING) {
+            // Renegotiation required.
+            handshakeStatus = res.getHandshakeStatus();
+
+            if (log.isDebugEnabled())
+                log.debug("Renegotiation requested [status=" + res.getStatus() + ", handshakeStatus = " +
+                    handshakeStatus + ']');
+
+            handshakeFinished = false;
+
+            handshake();
+        }
+    }
+
+    /**
+     * Allocate application buffer.
+     */
+    private ByteBuffer allocateAppBuff() {
+        int netBufSize = sslEngine.getSession().getPacketBufferSize() + 50;
+
+        int appBufSize = Math.max(sslEngine.getSession().getApplicationBufferSize() + 50, netBufSize * 2);
+
+        return ByteBuffer.allocate(appBufSize);
+    }
+
+    /**
+     * Read data from net buffer.
+     */
+    private void readFromNet() {
+        try {
+            inNetBuf.clear();
+
+            ch.read(inNetBuf);
+        }
+        catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    /**
+     * Copies data from out net buffer and passes it to the underlying chain.
+     *
+     * @return Nothing.
+     * @throws GridNioException If send failed.
+     */
+    private void writeNetBuffer() throws IgniteCheckedException {
+        try {
+            ch.write(outNetBuf);
+        }
+        catch (IOException e) {
+            throw new IgniteCheckedException("Failed to write byte to socket.", e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
index be8a4e8..a05135f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
@@ -393,6 +393,8 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
     public static ByteBuffer expandBuffer(ByteBuffer original, int cap) {
         ByteBuffer res = ByteBuffer.allocate(cap);
 
+        res.order(ByteOrder.nativeOrder());
+
         original.flip();
 
         res.put(original);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
index ac22d74..dc3d870 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
@@ -29,6 +29,7 @@ import java.util.concurrent.locks.*;
 import static javax.net.ssl.SSLEngineResult.*;
 import static javax.net.ssl.SSLEngineResult.HandshakeStatus.*;
 import static javax.net.ssl.SSLEngineResult.Status.*;
+import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.*;
 import static org.apache.ignite.internal.util.nio.ssl.GridNioSslFilter.*;
 
 /**
@@ -96,7 +97,14 @@ class GridNioSslHandler extends ReentrantLock {
 
         sslEngine = engine;
 
-        sslEngine.beginHandshake();
+        if (ses.meta(SSL_ENGINE.ordinal()) == null)
+            sslEngine.beginHandshake();
+        else {
+            sslEngine = ses.meta(SSL_ENGINE.ordinal());
+
+            handshakeFinished = true;
+            initHandshakeComplete = true;
+        }
 
         handshakeStatus = sslEngine.getHandshakeStatus();
 
@@ -114,6 +122,8 @@ class GridNioSslHandler extends ReentrantLock {
 
         appBuf = ByteBuffer.allocate(appBufSize);
 
+        appBuf.order(ByteOrder.nativeOrder());
+
         if (log.isDebugEnabled())
             log.debug("Started SSL session [netBufSize=" + netBufSize + ", appBufSize=" + appBufSize + ']');
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
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 4ca2995..b72299e 100644
--- 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
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.util.ipc.*;
 import org.apache.ignite.internal.util.ipc.shmem.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.nio.*;
+import org.apache.ignite.internal.util.nio.ssl.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.worker.*;
@@ -41,6 +42,7 @@ import org.apache.ignite.thread.*;
 import org.jetbrains.annotations.*;
 import org.jsr166.*;
 
+import javax.net.ssl.*;
 import java.io.*;
 import java.net.*;
 import java.nio.*;
@@ -747,6 +749,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     };
 
     /**
+     * @return {@code True} if ssl enabled.
+     */
+    private boolean isSslEnabled() {
+        return ignite.configuration().getSslContextFactory() != null;
+    }
+
+    /**
      * Sets address resolver.
      *
      * @param addrRslvr Address resolver.
@@ -1465,6 +1474,26 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     } :
                     null;
 
+                GridNioFilter[] filters;
+
+                if (isSslEnabled()) {
+                    GridNioSslFilter sslFilter =
+                        new GridNioSslFilter(ignite.configuration().getSslContextFactory().create(), log);
+
+                    sslFilter.directMode(true);
+
+                    filters = new GridNioFilter[] {
+                        new GridNioCodecFilter(parser, log, true),
+                        new GridConnectionBytesVerifyFilter(log),
+                        sslFilter
+                    };
+                }
+                else
+                    filters = new GridNioFilter[] {
+                        new GridNioCodecFilter(parser, log, true),
+                        new GridConnectionBytesVerifyFilter(log)
+                    };
+
                 GridNioServer<Message> srvr =
                     GridNioServer.<Message>builder()
                         .address(locHost)
@@ -1482,8 +1511,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         .directMode(true)
                         .metricsListener(metricsLsnr)
                         .writeTimeout(sockWriteTimeout)
-                        .filters(new GridNioCodecFilter(parser, log, true),
-                            new GridConnectionBytesVerifyFilter(log))
+                        .filters(filters)
                         .messageFormatter(msgFormatter)
                         .skipRecoveryPredicate(skipRecoveryPred)
                         .messageQueueSizeListener(queueSizeMonitor)
@@ -1501,6 +1529,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 return srvr;
             }
             catch (IgniteCheckedException e) {
+                if (X.hasCause(e, SSLException.class))
+                    throw new IgniteSpiException("Failed to create SSL context. SSL factory: "
+                        + ignite.configuration().getSslContextFactory() + '.', e);
+
                 lastEx = e;
 
                 if (log.isDebugEnabled())
@@ -1872,7 +1904,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             }
 
             try {
-                safeHandshake(client, null, node.id(), connTimeout0);
+                safeHandshake(client, null, node.id(), connTimeout0, null);
             }
             catch (HandshakeTimeoutException e) {
                 if (log.isDebugEnabled())
@@ -2019,10 +2051,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                     long rcvCnt = -1;
 
+                    GridTuple<SSLEngine> ssl = new GridTuple<>();
                     try {
                         ch.socket().connect(addr, (int)connTimeout);
 
-                        rcvCnt = safeHandshake(ch, recoveryDesc, node.id(), connTimeout0);
+                        rcvCnt = safeHandshake(ch, recoveryDesc, node.id(), connTimeout0, ssl);
 
                         if (rcvCnt == -1)
                             return null;
@@ -2037,6 +2070,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         meta.put(NODE_ID_META, node.id());
 
+                        if (isSslEnabled()) {
+                            assert ssl != null;
+                            assert ssl.get() != null;
+
+                            meta.put(GridNioSessionMetaKey.SSL_ENGINE.ordinal(), ssl.get());
+                        }
                         if (recoveryDesc != null) {
                             recoveryDesc.onHandshake(rcvCnt);
 
@@ -2161,6 +2200,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * @param recovery Recovery descriptor if use recovery handshake, otherwise {@code null}.
      * @param rmtNodeId Remote node.
      * @param timeout Timeout for handshake.
+     * @param ssl SSL engine if used cryptography, otherwise {@code null}.
      * @throws IgniteCheckedException If handshake failed or wasn't completed withing timeout.
      * @return Handshake response.
      */
@@ -2169,7 +2209,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         T client,
         @Nullable GridNioRecoveryDescriptor recovery,
         UUID rmtNodeId,
-        long timeout
+        long timeout,
+        @Nullable GridTuple<SSLEngine> ssl
     ) throws IgniteCheckedException {
         HandshakeTimeoutObject<T> obj = new HandshakeTimeoutObject<>(client, U.currentTimeMillis() + timeout);
 
@@ -2186,15 +2227,53 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 boolean success = false;
 
                 try {
-                    ByteBuffer buf = ByteBuffer.allocate(17);
+                    BlockingSslHandler sslHnd = null;
+
+                    ByteBuffer buf;
+
+                    if (isSslEnabled()) {
+                        GridFutureAdapter<ByteBuffer> handFut = new GridFutureAdapter<>();
 
-                    for (int i = 0; i < 17; ) {
-                        int read = ch.read(buf);
+                        SSLEngine sslEngine = ignite.configuration().getSslContextFactory()
+                            .create().createSSLEngine();
 
-                        if (read == -1)
-                            throw new IgniteCheckedException("Failed to read remote node ID (connection closed).");
+                        sslEngine.setUseClientMode(true);
 
-                        i += read;
+                        sslHnd = new BlockingSslHandler(sslEngine, ch, handFut, log);
+
+                        if (!sslHnd.handshake())
+                            throw new IgniteCheckedException("SSL handshake isn't completed.");
+
+                        ssl.set(sslEngine);
+
+                        ByteBuffer handBuff = handFut.get();
+
+                        if (handBuff.limit() < 17) {
+                            buf = ByteBuffer.allocate(1000);
+
+                            int read = ch.read(buf);
+
+                            if (read == -1)
+                                throw new IgniteCheckedException("Failed to read remote node ID (connection closed).");
+
+                            buf.flip();
+
+                            buf = sslHnd.decode(buf);
+                        }
+                        else
+                            buf = handBuff;
+                    }
+                    else {
+                        buf = ByteBuffer.allocate(17);
+
+                        for (int i = 0; i < 17; ) {
+                            int read = ch.read(buf);
+
+                            if (read == -1)
+                                throw new IgniteCheckedException("Failed to read remote node ID (connection closed).");
+
+                            i += read;
+                        }
                     }
 
                     UUID rmtNodeId0 = U.bytesToUuid(buf.array(), 1);
@@ -2205,7 +2284,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     else if (log.isDebugEnabled())
                         log.debug("Received remote node ID: " + rmtNodeId0);
 
-                    ch.write(ByteBuffer.wrap(U.IGNITE_HEADER));
+                    if (isSslEnabled() ) {
+                        assert sslHnd != null;
+
+                        ch.write(sslHnd.encrypt(ByteBuffer.wrap(U.IGNITE_HEADER)));
+                    }
+                    else
+                        ch.write(ByteBuffer.wrap(U.IGNITE_HEADER));
 
                     if (recovery != null) {
                         HandshakeMessage msg = new HandshakeMessage(getLocalNodeId(),
@@ -2225,30 +2310,61 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         buf.flip();
 
-                        ch.write(buf);
+                        if (isSslEnabled()) {
+                            assert sslHnd != null;
+
+                            ch.write(sslHnd.encrypt(buf));
+                        }
+                        else
+                            ch.write(buf);
+                    }
+                    else {
+                        if (isSslEnabled()) {
+                            assert sslHnd != null;
+
+                            ch.write(sslHnd.encrypt(ByteBuffer.wrap(nodeIdMsg.nodeIdBytesWithType)));
+                        }
+                        else
+                            ch.write(ByteBuffer.wrap(nodeIdMsg.nodeIdBytesWithType));
                     }
-                    else
-                        ch.write(ByteBuffer.wrap(nodeIdMsg.nodeIdBytesWithType));
 
                     if (recovery != null) {
                         if (log.isDebugEnabled())
                             log.debug("Waiting for handshake [rmtNode=" + rmtNodeId + ']');
 
-                        buf = ByteBuffer.allocate(9);
+                        if (isSslEnabled()) {
+                            assert sslHnd != null;
 
-                        buf.order(ByteOrder.nativeOrder());
+                            buf = ByteBuffer.allocate(1000);
+
+                            buf.order(ByteOrder.nativeOrder());
 
-                        for (int i = 0; i < 9; ) {
                             int read = ch.read(buf);
 
                             if (read == -1)
                                 throw new IgniteCheckedException("Failed to read remote node recovery handshake " +
                                     "(connection closed).");
 
-                            i += read;
-                        }
+                            buf.flip();
 
-                        rcvCnt = buf.getLong(1);
+                            rcvCnt = sslHnd.decode(buf).getLong(1);
+                        } else {
+                            buf = ByteBuffer.allocate(9);
+
+                            buf.order(ByteOrder.nativeOrder());
+
+                            for (int i = 0; i < 9; ) {
+                                int read = ch.read(buf);
+
+                                if (read == -1)
+                                    throw new IgniteCheckedException("Failed to read remote node recovery handshake " +
+                                        "(connection closed).");
+
+                                i += read;
+                            }
+
+                            rcvCnt = buf.getLong(1);
+                        }
 
                         if (log.isDebugEnabled())
                             log.debug("Received handshake message [rmtNode=" + rmtNodeId + ", rcvCnt=" + rcvCnt + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/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 a5ae5a9..97e910a 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
@@ -3975,7 +3975,10 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             for (port = spi.locPort; port < spi.locPort + spi.locPortRange; port++) {
                 try {
-                    srvrSock = new ServerSocket(port, 0, spi.locHost);
+                    if (spi.isSslEnabled())
+                        srvrSock = spi.sslSrvSocketFactory.createServerSocket(port, 0, spi.locHost);
+                    else
+                        srvrSock = new ServerSocket(port, 0, spi.locHost);
 
                     break;
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 7663fe6..b627010 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -40,6 +40,7 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.discovery.tcp.messages.*;
 import org.jetbrains.annotations.*;
 
+import javax.net.ssl.*;
 import java.io.*;
 import java.net.*;
 import java.util.*;
@@ -305,6 +306,12 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /** Node authenticator. */
     protected DiscoverySpiNodeAuthenticator nodeAuth;
 
+    /** SSL server socket factory. */
+    protected SSLServerSocketFactory sslSrvSocketFactory;
+
+    /** SSL socket factory. */
+    protected SSLSocketFactory sslSocketFactory;
+
     /** Context initialization latch. */
     @GridToStringExclude
     private final CountDownLatch ctxInitLatch = new CountDownLatch(1);
@@ -1106,7 +1113,12 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
 
         assert addr != null;
 
-        Socket sock = new Socket();
+        Socket sock;
+
+        if (isSslEnabled())
+            sock = sslSocketFactory.createSocket();
+        else
+            sock = new Socket();
 
         sock.bind(new InetSocketAddress(locHost, 0));
 
@@ -1578,6 +1590,19 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
         assertParameter(threadPri > 0, "threadPri > 0");
         assertParameter(statsPrintFreq >= 0, "statsPrintFreq >= 0");
 
+        if (isSslEnabled()) {
+            try {
+                SSLContext sslCtx = ignite().configuration().getSslContextFactory().create();
+
+                sslSocketFactory = sslCtx.getSocketFactory();
+                sslSrvSocketFactory = sslCtx.getServerSocketFactory();
+            }
+            catch (IgniteException e) {
+                throw new IgniteSpiException("Failed to create SSL context. SSL factory: "
+                    + ignite.configuration().getSslContextFactory(), e);
+            }
+        }
+
         try {
             locHost = U.resolveLocalHost(locAddr);
         }
@@ -1695,6 +1720,13 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /**
+     * @return {@code True} if ssl enabled.
+     */
+    boolean isSslEnabled() {
+        return ignite().configuration().getSslContextFactory() != null;
+    }
+
+    /**
      * <strong>FOR TEST ONLY!!!</strong>
      */
     public int clientWorkerCount() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/ssl/SslContextFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/ssl/SslContextFactory.java b/modules/core/src/main/java/org/apache/ignite/ssl/SslContextFactory.java
new file mode 100644
index 0000000..bf09541
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/ssl/SslContextFactory.java
@@ -0,0 +1,455 @@
+/*
+ * 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.ssl;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+
+import javax.cache.configuration.*;
+import javax.net.ssl.*;
+import java.io.*;
+import java.security.*;
+import java.security.cert.*;
+import java.util.*;
+
+/**
+ * This SSL context factory that provides ssl context configuration with specified key
+ * and trust stores.
+ * <p>
+ * In some cases it is useful to disable certificate validation of client side (e.g. when connecting
+ * to a server with self-signed certificate). This can be achieved by setting a disabled trust manager
+ * to this factory, which can be obtained by {@link #getDisabledTrustManager()} method:
+ * <pre>
+ *     SslContextFactory factory = new SslContextFactory();
+ *     factory.setTrustManagers(SslContextFactory.getDisabledTrustManager());
+ *     // Rest of initialization.
+ * </pre>
+ */
+public class SslContextFactory implements Factory<SSLContext> {
+    /** Default key store type. */
+    public static final String DFLT_STORE_TYPE = "JKS";
+
+    /** Default SSL protocol. */
+    public static final String DFLT_SSL_PROTOCOL = "TLS";
+
+    /** Default key manager algorithm. */
+    public static final String DFLT_KEY_ALGORITHM = "SunX509";
+
+    /** SSL protocol. */
+    private String proto = DFLT_SSL_PROTOCOL;
+
+    /** Key manager algorithm. */
+    private String keyAlgorithm = DFLT_KEY_ALGORITHM;
+
+    /** Key store type. */
+    private String keyStoreType = DFLT_STORE_TYPE;
+
+    /** Path to key store file */
+    private String keyStoreFilePath;
+
+    /** Key store password */
+    private char[] keyStorePwd;
+
+    /** Trust store type. */
+    private String trustStoreType = DFLT_STORE_TYPE;
+
+    /** Path to trust store. */
+    private String trustStoreFilePath;
+
+    /** Trust store password */
+    private char[] trustStorePwd;
+
+    /** Trust managers. */
+    private TrustManager[] trustMgrs;
+
+    /**
+     * Gets key store type used for context creation.
+     *
+     * @return Key store type.
+     */
+    public String getKeyStoreType() {
+        return keyStoreType;
+    }
+
+    /**
+     * Sets key store type used in context initialization. If not provided, {@link #DFLT_STORE_TYPE} will
+     * be used.
+     *
+     * @param keyStoreType Key store type.
+     */
+    public void setKeyStoreType(String keyStoreType) {
+        A.notNull(keyStoreType, "keyStoreType");
+
+        this.keyStoreType = keyStoreType;
+    }
+
+    /**
+     * Gets trust store type used for context creation.
+     *
+     * @return trust store type.
+     */
+    public String getTrustStoreType() {
+        return trustStoreType;
+    }
+
+    /**
+     * Sets trust store type used in context initialization. If not provided, {@link #DFLT_STORE_TYPE} will
+     * be used.
+     *
+     * @param trustStoreType Trust store type.
+     */
+    public void setTrustStoreType(String trustStoreType) {
+        A.notNull(trustStoreType, "trustStoreType");
+
+        this.trustStoreType = trustStoreType;
+    }
+
+    /**
+     * Gets protocol for secure transport.
+     *
+     * @return SSL protocol name.
+     */
+    public String getProtocol() {
+        return proto;
+    }
+
+    /**
+     * Sets protocol for secure transport. If not specified, {@link #DFLT_SSL_PROTOCOL} will be used.
+     *
+     * @param proto SSL protocol name.
+     */
+    public void setProtocol(String proto) {
+        A.notNull(proto, "proto");
+
+        this.proto = proto;
+    }
+
+    /**
+     * Gets algorithm that will be used to create a key manager. If not specified, {@link #DFLT_KEY_ALGORITHM}
+     * will be used.
+     *
+     * @return Key manager algorithm.
+     */
+    public String getKeyAlgorithm() {
+        return keyAlgorithm;
+    }
+
+    /**
+     * Sets key manager algorithm that will be used to create a key manager. Notice that in most cased default value
+     * suites well, however, on Android platform this value need to be set to <tt>X509<tt/>.
+     *
+     * @param keyAlgorithm Key algorithm name.
+     */
+    public void setKeyAlgorithm(String keyAlgorithm) {
+        A.notNull(keyAlgorithm, "keyAlgorithm");
+
+        this.keyAlgorithm = keyAlgorithm;
+    }
+
+    /**
+     * Gets path to the key store file.
+     *
+     * @return Path to key store file.
+     */
+    public String getKeyStoreFilePath() {
+        return keyStoreFilePath;
+    }
+
+    /**
+     * Sets path to the key store file. This is a mandatory parameter since
+     * ssl context could not be initialized without key manager.
+     *
+     * @param keyStoreFilePath Path to key store file.
+     */
+    public void setKeyStoreFilePath(String keyStoreFilePath) {
+        A.notNull(keyStoreFilePath, "keyStoreFilePath");
+
+        this.keyStoreFilePath = keyStoreFilePath;
+    }
+
+    /**
+     * Gets key store password.
+     *
+     * @return Key store password.
+     */
+    public char[] getKeyStorePassword() {
+        return keyStorePwd;
+    }
+
+    /**
+     * Sets key store password.
+     *
+     * @param keyStorePwd Key store password.
+     */
+    public void setKeyStorePassword(char[] keyStorePwd) {
+        A.notNull(keyStorePwd, "keyStorePwd");
+
+        this.keyStorePwd = keyStorePwd;
+    }
+
+    /**
+     * Gets path to the trust store file.
+     *
+     * @return Path to the trust store file.
+     */
+    public String getTrustStoreFilePath() {
+        return trustStoreFilePath;
+    }
+
+    /**
+     * Sets path to the trust store file. This is an optional parameter,
+     * however one of the {@code setTrustStoreFilePath(String)}, {@link #setTrustManagers(TrustManager[])}
+     * properties must be set.
+     *
+     * @param trustStoreFilePath Path to the trust store file.
+     */
+    public void setTrustStoreFilePath(String trustStoreFilePath) {
+        this.trustStoreFilePath = trustStoreFilePath;
+    }
+
+    /**
+     * Gets trust store password.
+     *
+     * @return Trust store password.
+     */
+    public char[] getTrustStorePassword() {
+        return trustStorePwd;
+    }
+
+    /**
+     * Sets trust store password.
+     *
+     * @param trustStorePwd Trust store password.
+     */
+    public void setTrustStorePassword(char[] trustStorePwd) {
+        this.trustStorePwd = trustStorePwd;
+    }
+
+    /**
+     * Gets pre-configured trust managers.
+     *
+     * @return Trust managers.
+     */
+    public TrustManager[] getTrustManagers() {
+        return trustMgrs;
+    }
+
+    /**
+     * Sets pre-configured trust managers. This is an optional parameter,
+     * however one of the {@link #setTrustStoreFilePath(String)}, {@code #setTrustManagers(TrustManager[])}
+     *
+     * @param trustMgrs Pre-configured trust managers.
+     */
+    public void setTrustManagers(TrustManager... trustMgrs) {
+        this.trustMgrs = trustMgrs;
+    }
+
+    /**
+     * Returns an instance of trust manager that will always succeed regardless of certificate provided.
+     *
+     * @return Trust manager instance.
+     */
+    public static TrustManager getDisabledTrustManager() {
+        return new DisabledX509TrustManager();
+    }
+
+    /**
+     * Creates SSL context based on factory settings.
+     *
+     * @return Initialized SSL context.
+     * @throws SSLException If SSL context could not be created.
+     */
+    private SSLContext createSslContext() throws SSLException {
+        checkParameters();
+
+        try {
+            KeyManagerFactory keyMgrFactory = KeyManagerFactory.getInstance(keyAlgorithm);
+
+            KeyStore keyStore = loadKeyStore(keyStoreType, keyStoreFilePath, keyStorePwd);
+
+            keyMgrFactory.init(keyStore, keyStorePwd);
+
+            TrustManager[] mgrs = trustMgrs;
+
+            if (mgrs == null) {
+                TrustManagerFactory trustMgrFactory = TrustManagerFactory.getInstance(keyAlgorithm);
+
+                KeyStore trustStore = loadKeyStore(trustStoreType, trustStoreFilePath, trustStorePwd);
+
+                trustMgrFactory.init(trustStore);
+
+                mgrs = trustMgrFactory.getTrustManagers();
+            }
+
+            SSLContext ctx = SSLContext.getInstance(proto);
+
+            ctx.init(keyMgrFactory.getKeyManagers(), mgrs, null);
+
+            return ctx;
+        }
+        catch (GeneralSecurityException e) {
+            throw new SSLException("Failed to initialize SSL context " + parameters(), e);
+        }
+    }
+
+    /**
+     * Builds human-readable string with factory parameters.
+     *
+     * @return Parameters string.
+     */
+    private String parameters() {
+        StringBuilder buf = new StringBuilder("[keyStoreType=").append(keyStoreType);
+
+        buf.append(", proto=").append(proto).append(", keyStoreFile=").append(keyStoreFilePath);
+
+        if (trustMgrs != null)
+            buf.append(", trustMgrs=").append(Arrays.toString(trustMgrs));
+        else
+            buf.append(", trustStoreFile=").append(trustStoreFilePath);
+
+        buf.append(']');
+
+        return buf.toString();
+    }
+
+    /**
+     * Checks that all required parameters are set.
+     *
+     * @throws SSLException If any of required parameters is missing.
+     */
+    private void checkParameters() throws SSLException {
+        assert keyStoreType != null;
+        assert proto != null;
+
+        checkNullParameter(keyStoreFilePath, "keyStoreFilePath");
+        checkNullParameter(keyStorePwd, "keyStorePwd");
+
+        if (trustMgrs == null) {
+            if (trustStoreFilePath == null)
+                throw new SSLException("Failed to initialize SSL context (either trustStoreFilePath or " +
+                    "trustManagers must be provided)");
+            else
+                checkNullParameter(trustStorePwd, "trustStorePwd");
+        }
+    }
+
+    /**
+     * @param param Value.
+     * @param name Name.
+     * @throws SSLException If {@code null}.
+     */
+    private void checkNullParameter(Object param, String name) throws SSLException {
+        if (param == null)
+            throw new SSLException("Failed to initialize SSL context (parameter cannot be null): " + name);
+    }
+
+    /**
+     * By default, this method simply opens a raw file input stream. Subclasses may override this method
+     * if some specific location should be handled (this may be a case for Android users).
+     *
+     * @param filePath Path to the file.
+     * @return Opened input stream.
+     * @throws IOException If stream could not be opened.
+     */
+    protected InputStream openFileInputStream(String filePath) throws IOException {
+        return new FileInputStream(filePath);
+    }
+
+    /**
+     * Loads key store with configured parameters.
+     *
+     * @param keyStoreType Type of key store.
+     * @param storeFilePath Path to key store file.
+     * @param keyStorePwd Store password.
+     * @return Initialized key store.
+     * @throws SSLException If key store could not be initialized.
+     */
+    private KeyStore loadKeyStore(String keyStoreType, String storeFilePath, char[] keyStorePwd) throws SSLException {
+        InputStream input = null;
+
+        try {
+            KeyStore keyStore = KeyStore.getInstance(keyStoreType);
+
+            input = openFileInputStream(storeFilePath);
+
+            keyStore.load(input, keyStorePwd);
+
+            return keyStore;
+        }
+        catch (GeneralSecurityException e) {
+            throw new SSLException("Failed to initialize key store (security exception occurred) [type=" +
+                keyStoreType + ", keyStorePath=" + storeFilePath + ']', e);
+        }
+        catch (FileNotFoundException e) {
+            throw new SSLException("Failed to initialize key store (key store file was not found): [path=" +
+                storeFilePath + ", msg=" + e.getMessage() + ']');
+        }
+        catch (IOException e) {
+            throw new SSLException("Failed to initialize key store (I/O error occurred): " + storeFilePath, e);
+        }
+        finally {
+            if (input != null) {
+                try {
+                    input.close();
+                }
+                catch (IOException ignored) {
+                }
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    public String toString() {
+        return getClass().getSimpleName() + parameters();
+    }
+
+    /**
+     * Disabled trust manager, will skip all certificate checks.
+     */
+    private static class DisabledX509TrustManager implements X509TrustManager {
+        /** Empty certificate array. */
+        private static final X509Certificate[] CERTS = new X509Certificate[0];
+
+        /** {@inheritDoc} */
+        @Override public void checkClientTrusted(X509Certificate[] x509Certificates, String s)
+            throws CertificateException {
+            // No-op, all clients are trusted.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void checkServerTrusted(X509Certificate[] x509Certificates, String s)
+            throws CertificateException {
+            // No-op, all servers are trusted.
+        }
+
+        /** {@inheritDoc} */
+        @Override public X509Certificate[] getAcceptedIssuers() {
+            return CERTS;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public SSLContext create() {
+        try {
+            return createSslContext();
+        }
+        catch (SSLException e) {
+            throw new IgniteException(e);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/main/java/org/apache/ignite/ssl/package-info.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/ssl/package-info.java b/modules/core/src/main/java/org/apache/ignite/ssl/package-info.java
new file mode 100644
index 0000000..7be1910
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/ssl/package-info.java
@@ -0,0 +1,22 @@
+/*
+ * 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 description. -->
+ * Contains classes for working with SSL.
+ */
+package org.apache.ignite.ssl;
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
index bfed977..13ed880 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.communication;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -59,6 +60,9 @@ public abstract class GridAbstractCommunicationSelfTest<T extends CommunicationS
     /** */
     private static final Object mux = new Object();
 
+    /** */
+    protected boolean useSsl = false;
+
     /**
      *
      */
@@ -299,6 +303,15 @@ public abstract class GridAbstractCommunicationSelfTest<T extends CommunicationS
 
             rsrcs.inject(spi);
 
+            if (useSsl) {
+                IgniteMock ignite = GridTestUtils.getFieldValue(spi, IgniteSpiAdapter.class, "ignite");
+
+                IgniteConfiguration cfg = ignite.configuration()
+                    .setSslContextFactory(GridTestUtils.sslFactory());
+
+                ignite.setStaticCfg(cfg);
+            }
+
             spi.setListener(new MessageListener(rsrcs.getNodeId()));
 
             node.setAttributes(spi.getNodeAttributes());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSelfTest.java
new file mode 100644
index 0000000..e5f8bb3
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiSslSelfTest.java
@@ -0,0 +1,38 @@
+/*
+ * 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.communication.tcp;
+
+import org.apache.ignite.testframework.junits.spi.*;
+
+/**
+ *
+ */
+@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
+public class GridTcpCommunicationSpiSslSelfTest extends GridTcpCommunicationSpiAbstractTest {
+    /** */
+    public GridTcpCommunicationSpiSslSelfTest() {
+        super(false);
+
+        this.useSsl = true;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected boolean tcpNoDelay() {
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
index 61bb944..4bdcf0a 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/AbstractDiscoverySelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.discovery;
 
 import mx4j.tools.adaptor.http.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.spi.*;
@@ -58,6 +59,9 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
     private static final String TEST_ATTRIBUTE_NAME = "test.node.prop";
 
     /** */
+    protected boolean useSsl = false;
+
+    /** */
     protected AbstractDiscoverySelfTest() {
         super(false);
     }
@@ -394,6 +398,15 @@ public abstract class AbstractDiscoverySelfTest<T extends IgniteSpi> extends Gri
 
                 GridTestUtils.setFieldValue(spi, IgniteSpiAdapter.class, "spiCtx", ctx);
 
+                if (useSsl) {
+                    IgniteMock ignite = GridTestUtils.getFieldValue(spi, IgniteSpiAdapter.class, "ignite");
+
+                    IgniteConfiguration cfg = ignite.configuration()
+                        .setSslContextFactory(GridTestUtils.sslFactory());
+
+                    ignite.setStaticCfg(cfg);
+                }
+
                 spi.spiStart(getTestGridName() + i);
 
                 spis.add(spi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiSslSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiSslSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiSslSelfTest.java
new file mode 100644
index 0000000..3eb8344
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiSslSelfTest.java
@@ -0,0 +1,28 @@
+/*
+ * 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;
+
+/**
+ * TCP discovery spi test with SSL.
+ */
+public class TcpDiscoverySpiSslSelfTest extends TcpDiscoverySpiSelfTest {
+    /** */
+    public TcpDiscoverySpiSslSelfTest() {
+        useSsl = true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSelfTest.java
new file mode 100644
index 0000000..314ca01
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySslSelfTest.java
@@ -0,0 +1,42 @@
+/*
+ * 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 org.apache.ignite.configuration.*;
+import org.apache.ignite.testframework.*;
+
+/**
+ * Test for {@link TcpDiscoverySpi} with SSL.
+ */
+public class TcpDiscoverySslSelfTest extends TcpDiscoverySelfTest {
+    /**
+     * @throws Exception If fails.
+     */
+    public TcpDiscoverySslSelfTest() throws Exception {
+        super();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setSslContextFactory(GridTestUtils.sslFactory());
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index d03d327..964f680 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -32,10 +32,12 @@ import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.ssl.*;
 import org.apache.ignite.testframework.config.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
+import javax.cache.configuration.*;
 import javax.net.ssl.*;
 import java.io.*;
 import java.lang.annotation.*;
@@ -1338,6 +1340,24 @@ public final class GridTestUtils {
         return factory;
     }
 
+
+    /**
+     * Creates test-purposed SSL context factory from test key store with disabled trust manager.
+     *
+     * @return SSL context factory used in test.
+     */
+    public static Factory<SSLContext> sslFactory() {
+        SslContextFactory factory = new SslContextFactory();
+
+        factory.setKeyStoreFilePath(
+            U.resolveIgnitePath(GridTestProperties.getProperty("ssl.keystore.path")).getAbsolutePath());
+        factory.setKeyStorePassword(GridTestProperties.getProperty("ssl.keystore.password").toCharArray());
+
+        factory.setTrustManagers(GridSslBasicContextFactory.getDisabledTrustManager());
+
+        return factory;
+    }
+
     /**
      * @param o1 Object 1.
      * @param o2 Object 2.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
index 2451f59..1471faa 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/IgniteMock.java
@@ -52,6 +52,9 @@ public class IgniteMock implements Ignite {
     /** */
     private final String home;
 
+    /** */
+    private IgniteConfiguration staticCfg;
+
     /**
      * Mock values
      *
@@ -84,6 +87,9 @@ public class IgniteMock implements Ignite {
 
     /** {@inheritDoc} */
     @Override public IgniteConfiguration configuration() {
+        if (staticCfg != null)
+            return staticCfg;
+
         IgniteConfiguration cfg = new IgniteConfiguration();
 
         cfg.setMarshaller(marshaller);
@@ -298,4 +304,11 @@ public class IgniteMock implements Ignite {
     @Override public <K> Affinity<K> affinity(String cacheName) {
         return null;
     }
+
+    /**
+     * @param staticCfg Configuration.
+     */
+    public void setStaticCfg(IgniteConfiguration staticCfg) {
+        this.staticCfg = staticCfg;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1fd31892/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 6f59f14..15797aa 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
@@ -60,6 +60,9 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(TcpDiscoveryRestartTest.class));
         suite.addTest(new TestSuite(TcpDiscoveryMultiThreadedTest.class));
 
+        // SSL.
+        suite.addTest(new TestSuite(TcpDiscoverySslSelfTest.class));
+
         return suite;
     }
 }