You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by se...@apache.org on 2015/06/01 13:44:53 UTC

[01/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Merge TcpDiscoverySpi and TcpClientDiscoverySpi

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-sprint-5 9a16d1906 -> 3c0046e8e


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
deleted file mode 100644
index ddbea0c..0000000
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
+++ /dev/null
@@ -1,1185 +0,0 @@
-/*
- * 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.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.io.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.marshaller.jdk.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.discovery.*;
-import org.apache.ignite.spi.discovery.tcp.internal.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.messages.*;
-import org.jetbrains.annotations.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.atomic.*;
-
-/**
- * Base class for TCP discovery SPIs.
- */
-public abstract class TcpDiscoverySpiAdapter extends IgniteSpiAdapter implements DiscoverySpi {
-    /** Default port to listen (value is <tt>47500</tt>). */
-    public static final int DFLT_PORT = 47500;
-
-    /** Default timeout for joining topology (value is <tt>0</tt>). */
-    public static final long DFLT_JOIN_TIMEOUT = 0;
-
-    /** Default network timeout in milliseconds (value is <tt>5000ms</tt>). */
-    public static final long DFLT_NETWORK_TIMEOUT = 5000;
-
-    /** Default value for thread priority (value is <tt>10</tt>). */
-    public static final int DFLT_THREAD_PRI = 10;
-
-    /** Default heartbeat messages issuing frequency (value is <tt>100ms</tt>). */
-    public static final long DFLT_HEARTBEAT_FREQ = 100;
-
-    /** Default size of topology snapshots history. */
-    public static final int DFLT_TOP_HISTORY_SIZE = 1000;
-
-    /** Response OK. */
-    protected static final int RES_OK = 1;
-
-    /** Response CONTINUE JOIN. */
-    protected static final int RES_CONTINUE_JOIN = 100;
-
-    /** Response WAIT. */
-    protected static final int RES_WAIT = 200;
-
-    /** Local address. */
-    protected String locAddr;
-
-    /** IP finder. */
-    protected TcpDiscoveryIpFinder ipFinder;
-
-    /** Socket operations timeout. */
-    protected long sockTimeout; // Must be initialized in the constructor of child class.
-
-    /** Message acknowledgement timeout. */
-    protected long ackTimeout; // Must be initialized in the constructor of child class.
-
-    /** Network timeout. */
-    protected long netTimeout = DFLT_NETWORK_TIMEOUT;
-
-    /** Join timeout. */
-    @SuppressWarnings("RedundantFieldInitialization")
-    protected long joinTimeout = DFLT_JOIN_TIMEOUT;
-
-    /** Thread priority for all threads started by SPI. */
-    protected int threadPri = DFLT_THREAD_PRI;
-
-    /** Heartbeat messages issuing frequency. */
-    protected long hbFreq = DFLT_HEARTBEAT_FREQ;
-
-    /** Size of topology snapshots history. */
-    protected int topHistSize = DFLT_TOP_HISTORY_SIZE;
-
-    /** Grid discovery listener. */
-    protected volatile DiscoverySpiListener lsnr;
-
-    /** Data exchange. */
-    protected DiscoverySpiDataExchange exchange;
-
-    /** Metrics provider. */
-    protected DiscoveryMetricsProvider metricsProvider;
-
-    /** Local node attributes. */
-    protected Map<String, Object> locNodeAttrs;
-
-    /** Local node version. */
-    protected IgniteProductVersion locNodeVer;
-
-    /** Local node. */
-    protected TcpDiscoveryNode locNode;
-
-    /** Local host. */
-    protected InetAddress locHost;
-
-    /** Internal and external addresses of local node. */
-    protected Collection<InetSocketAddress> locNodeAddrs;
-
-    /** Socket timeout worker. */
-    protected SocketTimeoutWorker sockTimeoutWorker;
-
-    /** Start time of the very first grid node. */
-    protected volatile long gridStartTime;
-
-    /** Marshaller. */
-    protected final Marshaller marsh = new JdkMarshaller();
-
-    /** Statistics. */
-    protected final TcpDiscoveryStatistics stats = new TcpDiscoveryStatistics();
-
-    /** Logger. */
-    @LoggerResource
-    protected IgniteLogger log;
-
-    /**
-     * Check parameters set to this SPI.
-     */
-    protected void checkParameters() {
-        assertParameter(ipFinder != null, "ipFinder != null");
-        assertParameter(hbFreq > 0, "heartbeatFreq > 0");
-        assertParameter(netTimeout > 0, "networkTimeout > 0");
-        assertParameter(sockTimeout > 0, "sockTimeout > 0");
-        assertParameter(ackTimeout > 0, "ackTimeout > 0");
-    }
-
-    /**
-     * Inject resources
-     *
-     * @param ignite Ignite.
-     */
-    @IgniteInstanceResource
-    @Override protected void injectResources(Ignite ignite) {
-        super.injectResources(ignite);
-
-        // Inject resource.
-        if (ignite != null)
-            setLocalAddress(ignite.configuration().getLocalHost());
-    }
-
-    /**
-     * Sets local host IP address that discovery SPI uses.
-     * <p>
-     * If not provided, by default a first found non-loopback address
-     * will be used. If there is no non-loopback address available,
-     * then {@link InetAddress#getLocalHost()} will be used.
-     *
-     * @param locAddr IP address.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setLocalAddress(String locAddr) {
-        // Injection should not override value already set by Spring or user.
-        if (this.locAddr == null)
-            this.locAddr = locAddr;
-    }
-
-    /**
-     * Gets local address that was set to SPI with {@link #setLocalAddress(String)} method.
-     *
-     * @return local address.
-     */
-    public String getLocalAddress() {
-        return locAddr;
-    }
-
-    /**
-     * Gets IP finder for IP addresses sharing and storing.
-     *
-     * @return IP finder for IP addresses sharing and storing.
-     */
-    public TcpDiscoveryIpFinder getIpFinder() {
-        return ipFinder;
-    }
-
-    /**
-     * Sets IP finder for IP addresses sharing and storing.
-     * <p>
-     * If not provided {@link org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder} will be used by default.
-     *
-     * @param ipFinder IP finder.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setIpFinder(TcpDiscoveryIpFinder ipFinder) {
-        this.ipFinder = ipFinder;
-    }
-
-    /**
-     * Sets socket operations timeout. This timeout is used to limit connection time and
-     * write-to-socket time.
-     * <p>
-     * Note that when running Ignite on Amazon EC2, socket timeout must be set to a value
-     * significantly greater than the default (e.g. to {@code 30000}).
-     * <p>
-     * If not specified, default is {@link TcpDiscoverySpi#DFLT_SOCK_TIMEOUT},
-     * {@link TcpClientDiscoverySpi#DFLT_SOCK_TIMEOUT}.
-     *
-     * @param sockTimeout Socket connection timeout.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setSocketTimeout(long sockTimeout) {
-        this.sockTimeout = sockTimeout;
-    }
-
-    /**
-     * Sets timeout for receiving acknowledgement for sent message.
-     * <p>
-     * If acknowledgement is not received within this timeout, sending is considered as failed
-     * and SPI tries to repeat message sending.
-     * <p>
-     * If not specified, default is {@link TcpDiscoverySpi#DFLT_ACK_TIMEOUT},
-     * {@link TcpClientDiscoverySpi#DFLT_ACK_TIMEOUT}.
-     *
-     * @param ackTimeout Acknowledgement timeout.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setAckTimeout(long ackTimeout) {
-        this.ackTimeout = ackTimeout;
-    }
-
-    /**
-     * Sets maximum network timeout to use for network operations.
-     * <p>
-     * If not specified, default is {@link #DFLT_NETWORK_TIMEOUT}.
-     *
-     * @param netTimeout Network timeout.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setNetworkTimeout(long netTimeout) {
-        this.netTimeout = netTimeout;
-    }
-
-    /**
-     * Join timeout.
-     * <p>
-     * If non-shared IP finder is used and node fails to connect to
-     * any address from IP finder, node keeps trying to join within this
-     * timeout. If all addresses are still unresponsive, exception is thrown
-     * and node startup fails.
-     * @return Join timeout in milliseconds, ({@code 0} means wait forever).
-     */
-    public long getJoinTimeout() {
-        return joinTimeout;
-    }
-
-    /**
-     * Sets join timeout.
-     * <p>
-     * If non-shared IP finder is used and node fails to connect to
-     * any address from IP finder, node keeps trying to join within this
-     * timeout. If all addresses are still unresponsive, exception is thrown
-     * and node startup fails.
-     * <p>
-     * If not specified, default is {@link #DFLT_JOIN_TIMEOUT}.
-     *
-     * @param joinTimeout Join timeout ({@code 0} means wait forever).
-     *
-     * @see TcpDiscoveryIpFinder#isShared()
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setJoinTimeout(long joinTimeout) {
-        this.joinTimeout = joinTimeout;
-    }
-
-    /**
-     * Sets thread priority. All threads within SPI will be started with it.
-     * <p>
-     * If not provided, default value is {@link #DFLT_THREAD_PRI}
-     *
-     * @param threadPri Thread priority.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setThreadPriority(int threadPri) {
-        this.threadPri = threadPri;
-    }
-
-    /**
-     * Sets delay between issuing of heartbeat messages. SPI sends heartbeat messages
-     * in configurable time interval to other nodes to notify them about its state.
-     * <p>
-     * If not provided, default value is {@link #DFLT_HEARTBEAT_FREQ}.
-     *
-     * @param hbFreq Heartbeat frequency in milliseconds.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setHeartbeatFrequency(long hbFreq) {
-        this.hbFreq = hbFreq;
-    }
-
-    /**
-     * @return Size of topology snapshots history.
-     */
-    public long getTopHistorySize() {
-        return topHistSize;
-    }
-
-    /**
-     * Sets size of topology snapshots history. Specified size should be greater than or equal to default size
-     * {@link #DFLT_TOP_HISTORY_SIZE}.
-     *
-     * @param topHistSize Size of topology snapshots history.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setTopHistorySize(int topHistSize) {
-        if (topHistSize < DFLT_TOP_HISTORY_SIZE) {
-            U.warn(log, "Topology history size should be greater than or equal to default size. " +
-                "Specified size will not be set [curSize=" + this.topHistSize + ", specifiedSize=" + topHistSize +
-                ", defaultSize=" + DFLT_TOP_HISTORY_SIZE + ']');
-
-            return;
-        }
-
-        this.topHistSize = topHistSize;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
-        assert locNodeAttrs == null;
-        assert locNodeVer == null;
-
-        if (log.isDebugEnabled()) {
-            log.debug("Node attributes to set: " + attrs);
-            log.debug("Node version to set: " + ver);
-        }
-
-        locNodeAttrs = attrs;
-        locNodeVer = ver;
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<Object> injectables() {
-        return F.<Object>asList(ipFinder);
-    }
-
-    /**
-     * Gets socket timeout.
-     *
-     * @return Socket timeout.
-     */
-    public long getSocketTimeout() {
-        return sockTimeout;
-    }
-
-    /**
-     * Gets message acknowledgement timeout.
-     *
-     * @return Message acknowledgement timeout.
-     */
-    public long getAckTimeout() {
-        return ackTimeout;
-    }
-
-    /**
-     * Gets network timeout.
-     *
-     * @return Network timeout.
-     */
-    public long getNetworkTimeout() {
-        return netTimeout;
-    }
-
-    /**
-     * Gets thread priority. All threads within SPI will be started with it.
-     *
-     * @return Thread priority.
-     */
-    public int getThreadPriority() {
-        return threadPri;
-    }
-
-    /**
-     * Gets delay between heartbeat messages sent by coordinator.
-     *
-     * @return Time period in milliseconds.
-     */
-    public long getHeartbeatFrequency() {
-        return hbFreq;
-    }
-
-    /**
-     * Gets {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder} (string representation).
-     *
-     * @return IPFinder (string representation).
-     */
-    public String getIpFinderFormatted() {
-        return ipFinder.toString();
-    }
-
-    /**
-     * Gets joined nodes count.
-     *
-     * @return Nodes joined count.
-     */
-    public long getNodesJoined() {
-        return stats.joinedNodesCount();
-    }
-
-    /**
-     * Gets left nodes count.
-     *
-     * @return Left nodes count.
-     */
-    public long getNodesLeft() {
-        return stats.leftNodesCount();
-    }
-
-    /**
-     * Gets failed nodes count.
-     *
-     * @return Failed nodes count.
-     */
-    public long getNodesFailed() {
-        return stats.failedNodesCount();
-    }
-
-    /**
-     * Gets pending messages registered count.
-     *
-     * @return Pending messages registered count.
-     */
-    public long getPendingMessagesRegistered() {
-        return stats.pendingMessagesRegistered();
-    }
-
-    /**
-     * Gets pending messages discarded count.
-     *
-     * @return Pending messages registered count.
-     */
-    public long getPendingMessagesDiscarded() {
-        return stats.pendingMessagesDiscarded();
-    }
-
-    /**
-     * Gets avg message processing time.
-     *
-     * @return Avg message processing time.
-     */
-    public long getAvgMessageProcessingTime() {
-        return stats.avgMessageProcessingTime();
-    }
-
-    /**
-     * Gets max message processing time.
-     *
-     * @return Max message processing time.
-     */
-    public long getMaxMessageProcessingTime() {
-        return stats.maxMessageProcessingTime();
-    }
-
-    /**
-     * Gets total received messages count.
-     *
-     * @return Total received messages count.
-     */
-    public int getTotalReceivedMessages() {
-        return stats.totalReceivedMessages();
-    }
-
-    /**
-     * Gets received messages counts (grouped by type).
-     *
-     * @return Map containing message types and respective counts.
-     */
-    public Map<String, Integer> getReceivedMessages() {
-        return stats.receivedMessages();
-    }
-
-    /**
-     * Gets total processed messages count.
-     *
-     * @return Total processed messages count.
-     */
-    public int getTotalProcessedMessages() {
-        return stats.totalProcessedMessages();
-    }
-
-    /**
-     * Gets processed messages counts (grouped by type).
-     *
-     * @return Map containing message types and respective counts.
-     */
-    public Map<String, Integer> getProcessedMessages() {
-        return stats.processedMessages();
-    }
-
-    /**
-     * Gets time local node has been coordinator since.
-     *
-     * @return Time local node is coordinator since.
-     */
-    public long getCoordinatorSinceTimestamp() {
-        return stats.coordinatorSinceTimestamp();
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
-        super.onContextInitialized0(spiCtx);
-
-        ipFinder.onSpiContextInitialized(spiCtx);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        super.onContextDestroyed0();
-
-        if (ipFinder != null)
-            ipFinder.onSpiContextDestroyed();
-    }
-
-    /** {@inheritDoc} */
-    @Override public ClusterNode getLocalNode() {
-        return locNode;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setListener(@Nullable DiscoverySpiListener lsnr) {
-        this.lsnr = lsnr;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setDataExchange(DiscoverySpiDataExchange exchange) {
-        this.exchange = exchange;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setMetricsProvider(DiscoveryMetricsProvider metricsProvider) {
-        this.metricsProvider = metricsProvider;
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getGridStartTime() {
-        assert gridStartTime != 0;
-
-        return gridStartTime;
-    }
-
-    /**
-     * @param sockAddr Remote address.
-     * @return Opened socket.
-     * @throws IOException If failed.
-     */
-    protected Socket openSocket(InetSocketAddress sockAddr) throws IOException {
-        assert sockAddr != null;
-
-        InetSocketAddress resolved = sockAddr.isUnresolved() ?
-            new InetSocketAddress(InetAddress.getByName(sockAddr.getHostName()), sockAddr.getPort()) : sockAddr;
-
-        InetAddress addr = resolved.getAddress();
-
-        assert addr != null;
-
-        Socket sock = new Socket();
-
-        sock.bind(new InetSocketAddress(locHost, 0));
-
-        sock.setTcpNoDelay(true);
-
-        sock.connect(resolved, (int)sockTimeout);
-
-        writeToSocket(sock, U.IGNITE_HEADER);
-
-        return sock;
-    }
-
-    /**
-     * Writes message to the socket.
-     *
-     * @param sock Socket.
-     * @param data Raw data to write.
-     * @throws IOException If IO failed or write timed out.
-     */
-    @SuppressWarnings("ThrowFromFinallyBlock")
-    protected void writeToSocket(Socket sock, byte[] data) throws IOException {
-        assert sock != null;
-        assert data != null;
-
-        SocketTimeoutObject obj = new SocketTimeoutObject(sock, U.currentTimeMillis() + sockTimeout);
-
-        sockTimeoutWorker.addTimeoutObject(obj);
-
-        IOException err = null;
-
-        try {
-            OutputStream out = sock.getOutputStream();
-
-            out.write(data);
-
-            out.flush();
-        }
-        catch (IOException e) {
-            err = e;
-        }
-        finally {
-            boolean cancelled = obj.cancel();
-
-            if (cancelled)
-                sockTimeoutWorker.removeTimeoutObject(obj);
-
-            // Throw original exception.
-            if (err != null)
-                throw err;
-
-            if (!cancelled)
-                throw new SocketTimeoutException("Write timed out (socket was concurrently closed).");
-        }
-    }
-
-    /**
-     * Writes message to the socket.
-     *
-     * @param sock Socket.
-     * @param msg Message.
-     * @throws IOException If IO failed or write timed out.
-     * @throws IgniteCheckedException If marshalling failed.
-     */
-    protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg) throws IOException, IgniteCheckedException {
-        writeToSocket(sock, msg, new GridByteArrayOutputStream(8 * 1024)); // 8K.
-    }
-
-    /**
-     * Writes message to the socket.
-     *
-     * @param sock Socket.
-     * @param msg Message.
-     * @param bout Byte array output stream.
-     * @throws IOException If IO failed or write timed out.
-     * @throws IgniteCheckedException If marshalling failed.
-     */
-    @SuppressWarnings("ThrowFromFinallyBlock")
-    protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, GridByteArrayOutputStream bout)
-        throws IOException, IgniteCheckedException {
-        assert sock != null;
-        assert msg != null;
-        assert bout != null;
-
-        // Marshall message first to perform only write after.
-        marsh.marshal(msg, bout);
-
-        SocketTimeoutObject obj = new SocketTimeoutObject(sock, U.currentTimeMillis() + sockTimeout);
-
-        sockTimeoutWorker.addTimeoutObject(obj);
-
-        IOException err = null;
-
-        try {
-            OutputStream out = sock.getOutputStream();
-
-            bout.writeTo(out);
-
-            out.flush();
-        }
-        catch (IOException e) {
-            err = e;
-        }
-        finally {
-            boolean cancelled = obj.cancel();
-
-            if (cancelled)
-                sockTimeoutWorker.removeTimeoutObject(obj);
-
-            // Throw original exception.
-            if (err != null)
-                throw err;
-
-            if (!cancelled)
-                throw new SocketTimeoutException("Write timed out (socket was concurrently closed).");
-        }
-    }
-
-    /**
-     * Writes response to the socket.
-     *
-     * @param sock Socket.
-     * @param res Integer response.
-     * @throws IOException If IO failed or write timed out.
-     */
-    @SuppressWarnings("ThrowFromFinallyBlock")
-    protected void writeToSocket(Socket sock, int res) throws IOException {
-        assert sock != null;
-
-        SocketTimeoutObject obj = new SocketTimeoutObject(sock, U.currentTimeMillis() + sockTimeout);
-
-        sockTimeoutWorker.addTimeoutObject(obj);
-
-        OutputStream out = sock.getOutputStream();
-
-        IOException err = null;
-
-        try {
-            out.write(res);
-
-            out.flush();
-        }
-        catch (IOException e) {
-            err = e;
-        }
-        finally {
-            boolean cancelled = obj.cancel();
-
-            if (cancelled)
-                sockTimeoutWorker.removeTimeoutObject(obj);
-
-            // Throw original exception.
-            if (err != null)
-                throw err;
-
-            if (!cancelled)
-                throw new SocketTimeoutException("Write timed out (socket was concurrently closed).");
-        }
-    }
-
-    /**
-     * Reads message from the socket limiting read time.
-     *
-     * @param sock Socket.
-     * @param in Input stream (in case socket stream was wrapped).
-     * @param timeout Socket timeout for this operation.
-     * @return Message.
-     * @throws IOException If IO failed or read timed out.
-     * @throws IgniteCheckedException If unmarshalling failed.
-     */
-    protected <T> T readMessage(Socket sock, @Nullable InputStream in, long timeout) throws IOException, IgniteCheckedException {
-        assert sock != null;
-
-        int oldTimeout = sock.getSoTimeout();
-
-        try {
-            sock.setSoTimeout((int)timeout);
-
-            return marsh.unmarshal(in == null ? sock.getInputStream() : in, U.gridClassLoader());
-        }
-        catch (IOException | IgniteCheckedException e) {
-            if (X.hasCause(e, SocketTimeoutException.class))
-                LT.warn(log, null, "Timed out waiting for message to be read (most probably, the reason is " +
-                    "in long GC pauses on remote node. Current timeout: " + timeout + '.');
-
-            throw e;
-        }
-        finally {
-            // Quietly restore timeout.
-            try {
-                sock.setSoTimeout(oldTimeout);
-            }
-            catch (SocketException ignored) {
-                // No-op.
-            }
-        }
-    }
-
-    /**
-     * Reads message delivery receipt from the socket.
-     *
-     * @param sock Socket.
-     * @param timeout Socket timeout for this operation.
-     * @return Receipt.
-     * @throws IOException If IO failed or read timed out.
-     */
-    protected int readReceipt(Socket sock, long timeout) throws IOException {
-        assert sock != null;
-
-        int oldTimeout = sock.getSoTimeout();
-
-        try {
-            sock.setSoTimeout((int)timeout);
-
-            int res = sock.getInputStream().read();
-
-            if (res == -1)
-                throw new EOFException();
-
-            return res;
-        }
-        catch (SocketTimeoutException e) {
-            LT.warn(log, null, "Timed out waiting for message delivery receipt (most probably, the reason is " +
-                "in long GC pauses on remote node; consider tuning GC and increasing 'ackTimeout' " +
-                "configuration property). Will retry to send message with increased timeout. " +
-                "Current timeout: " + timeout + '.');
-
-            stats.onAckTimeout();
-
-            throw e;
-        }
-        finally {
-            // Quietly restore timeout.
-            try {
-                sock.setSoTimeout(oldTimeout);
-            }
-            catch (SocketException ignored) {
-                // No-op.
-            }
-        }
-    }
-
-    /**
-     * Resolves addresses registered in the IP finder, removes duplicates and local host
-     * address and returns the collection of.
-     *
-     * @return Resolved addresses without duplicates and local address (potentially
-     *      empty but never null).
-     * @throws org.apache.ignite.spi.IgniteSpiException If an error occurs.
-     */
-    protected Collection<InetSocketAddress> resolvedAddresses() throws IgniteSpiException {
-        List<InetSocketAddress> res = new ArrayList<>();
-
-        Collection<InetSocketAddress> addrs;
-
-        // Get consistent addresses collection.
-        while (true) {
-            try {
-                addrs = registeredAddresses();
-
-                break;
-            }
-            catch (IgniteSpiException e) {
-                LT.error(log, e, "Failed to get registered addresses from IP finder on start " +
-                    "(retrying every 2000 ms).");
-            }
-
-            try {
-                U.sleep(2000);
-            }
-            catch (IgniteInterruptedCheckedException e) {
-                throw new IgniteSpiException("Thread has been interrupted.", e);
-            }
-        }
-
-        for (InetSocketAddress addr : addrs) {
-            assert addr != null;
-
-            try {
-                InetSocketAddress resolved = addr.isUnresolved() ?
-                    new InetSocketAddress(InetAddress.getByName(addr.getHostName()), addr.getPort()) : addr;
-
-                if (locNodeAddrs == null || !locNodeAddrs.contains(resolved))
-                    res.add(resolved);
-            }
-            catch (UnknownHostException ignored) {
-                LT.warn(log, null, "Failed to resolve address from IP finder (host is unknown): " + addr);
-
-                // Add address in any case.
-                res.add(addr);
-            }
-        }
-
-        if (!res.isEmpty())
-            Collections.shuffle(res);
-
-        return res;
-    }
-
-    /**
-     * Gets addresses registered in the IP finder, initializes addresses having no
-     * port (or 0 port) with {@link #DFLT_PORT}.
-     *
-     * @return Registered addresses.
-     * @throws org.apache.ignite.spi.IgniteSpiException If an error occurs.
-     */
-    protected Collection<InetSocketAddress> registeredAddresses() throws IgniteSpiException {
-        Collection<InetSocketAddress> res = new ArrayList<>();
-
-        for (InetSocketAddress addr : ipFinder.getRegisteredAddresses()) {
-            if (addr.getPort() == 0) {
-                // TcpDiscoveryNode.discoveryPort() returns an correct port for a server node and 0 for client node.
-                int port = locNode.discoveryPort() != 0 ? locNode.discoveryPort() : DFLT_PORT;
-
-                addr = addr.isUnresolved() ? new InetSocketAddress(addr.getHostName(), port) :
-                    new InetSocketAddress(addr.getAddress(), port);
-            }
-
-            res.add(addr);
-        }
-
-        return res;
-    }
-
-    /**
-     * @param msg Message.
-     * @return Error.
-     */
-    protected IgniteSpiException duplicateIdError(TcpDiscoveryDuplicateIdMessage msg) {
-        assert msg != null;
-
-        return new IgniteSpiException("Local node has the same ID as existing node in topology " +
-            "(fix configuration and restart local node) [localNode=" + locNode +
-            ", existingNode=" + msg.node() + ']');
-    }
-
-    /**
-     * @param msg Message.
-     * @return Error.
-     */
-    protected IgniteSpiException authenticationFailedError(TcpDiscoveryAuthFailedMessage msg) {
-        assert msg != null;
-
-        return new IgniteSpiException(new IgniteAuthenticationException("Authentication failed [nodeId=" +
-            msg.creatorNodeId() + ", addr=" + msg.address().getHostAddress() + ']'));
-    }
-
-    /**
-     * @param msg Message.
-     * @return Error.
-     */
-    protected IgniteSpiException checkFailedError(TcpDiscoveryCheckFailedMessage msg) {
-        assert msg != null;
-
-        return versionCheckFailed(msg) ? new IgniteSpiVersionCheckException(msg.error()) :
-            new IgniteSpiException(msg.error());
-    }
-
-    /**
-     * @param msg Message.
-     * @return Whether delivery of the message is ensured.
-     */
-    protected boolean ensured(TcpDiscoveryAbstractMessage msg) {
-        return U.getAnnotation(msg.getClass(), TcpDiscoveryEnsureDelivery.class) != null;
-    }
-
-    /**
-     * @param msg Failed message.
-     * @return {@code True} if specified failed message relates to version incompatibility, {@code false} otherwise.
-     * @deprecated Parsing of error message was used for preserving backward compatibility. We should remove it
-     *      and create separate message for failed version check with next major release.
-     */
-    @Deprecated
-    private static boolean versionCheckFailed(TcpDiscoveryCheckFailedMessage msg) {
-        return msg.error().contains("versions are not compatible");
-    }
-
-    /**
-     * @param nodeId Node ID.
-     * @return Marshalled exchange data.
-     */
-    protected Map<Integer, byte[]> collectExchangeData(UUID nodeId) {
-        Map<Integer, Serializable> data = exchange.collect(nodeId);
-
-        if (data == null)
-            return null;
-
-        Map<Integer, byte[]> data0 = U.newHashMap(data.size());
-
-        for (Map.Entry<Integer, Serializable> entry : data.entrySet()) {
-            try {
-                byte[] bytes = marsh.marshal(entry.getValue());
-
-                data0.put(entry.getKey(), bytes);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to marshal discovery data " +
-                    "[comp=" + entry.getKey() + ", data=" + entry.getValue() + ']', e);
-            }
-        }
-
-        return data0;
-    }
-
-    /**
-     * @param joiningNodeID Joining node ID.
-     * @param nodeId Remote node ID for which data is provided.
-     * @param data Collection of marshalled discovery data objects from different components.
-     * @param clsLdr Class loader for discovery data unmarshalling.
-     */
-    protected void onExchange(UUID joiningNodeID,
-        UUID nodeId,
-        Map<Integer, byte[]> data,
-        ClassLoader clsLdr)
-    {
-        Map<Integer, Serializable> data0 = U.newHashMap(data.size());
-
-        for (Map.Entry<Integer, byte[]> entry : data.entrySet()) {
-            try {
-                Serializable compData = marsh.unmarshal(entry.getValue(), clsLdr);
-
-                data0.put(entry.getKey(), compData);
-            }
-            catch (IgniteCheckedException e) {
-                U.error(log, "Failed to unmarshal discovery data for component: "  + entry.getKey(), e);
-            }
-        }
-
-        exchange.onExchange(joiningNodeID, nodeId, data0);
-    }
-
-    /**
-     * Handles sockets timeouts.
-     */
-    protected class SocketTimeoutWorker extends IgniteSpiThread {
-        /** Time-based sorted set for timeout objects. */
-        private final GridConcurrentSkipListSet<SocketTimeoutObject> timeoutObjs =
-            new GridConcurrentSkipListSet<>(new Comparator<SocketTimeoutObject>() {
-                @Override public int compare(SocketTimeoutObject o1, SocketTimeoutObject o2) {
-                    int res = Long.compare(o1.endTime(), o2.endTime());
-
-                    if (res != 0)
-                        return res;
-
-                    return Long.compare(o1.id(), o2.id());
-                }
-            });
-
-        /** Mutex. */
-        private final Object mux0 = new Object();
-
-        /**
-         *
-         */
-        SocketTimeoutWorker() {
-            super(gridName, "tcp-disco-sock-timeout-worker", log);
-
-            setPriority(threadPri);
-        }
-
-        /**
-         * @param timeoutObj Timeout object to add.
-         */
-        @SuppressWarnings({"NakedNotify"})
-        public void addTimeoutObject(SocketTimeoutObject timeoutObj) {
-            assert timeoutObj != null && timeoutObj.endTime() > 0 && timeoutObj.endTime() != Long.MAX_VALUE;
-
-            timeoutObjs.add(timeoutObj);
-
-            if (timeoutObjs.firstx() == timeoutObj) {
-                synchronized (mux0) {
-                    mux0.notifyAll();
-                }
-            }
-        }
-
-        /**
-         * @param timeoutObj Timeout object to remove.
-         */
-        public void removeTimeoutObject(SocketTimeoutObject timeoutObj) {
-            assert timeoutObj != null;
-
-            timeoutObjs.remove(timeoutObj);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            if (log.isDebugEnabled())
-                log.debug("Socket timeout worker has been started.");
-
-            while (!isInterrupted()) {
-                long now = U.currentTimeMillis();
-
-                for (Iterator<SocketTimeoutObject> iter = timeoutObjs.iterator(); iter.hasNext(); ) {
-                    SocketTimeoutObject timeoutObj = iter.next();
-
-                    if (timeoutObj.endTime() <= now) {
-                        iter.remove();
-
-                        if (timeoutObj.onTimeout()) {
-                            LT.warn(log, null, "Socket write has timed out (consider increasing " +
-                                "'sockTimeout' configuration property) [sockTimeout=" + sockTimeout + ']');
-
-                            stats.onSocketTimeout();
-                        }
-                    }
-                    else
-                        break;
-                }
-
-                synchronized (mux0) {
-                    while (true) {
-                        // Access of the first element must be inside of
-                        // synchronization block, so we don't miss out
-                        // on thread notification events sent from
-                        // 'addTimeoutObject(..)' method.
-                        SocketTimeoutObject first = timeoutObjs.firstx();
-
-                        if (first != null) {
-                            long waitTime = first.endTime() - U.currentTimeMillis();
-
-                            if (waitTime > 0)
-                                mux0.wait(waitTime);
-                            else
-                                break;
-                        }
-                        else
-                            mux0.wait(5000);
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Socket timeout object.
-     */
-    private static class SocketTimeoutObject {
-        /** */
-        private static final AtomicLong idGen = new AtomicLong();
-
-        /** */
-        private final long id = idGen.incrementAndGet();
-
-        /** */
-        private final Socket sock;
-
-        /** */
-        private final long endTime;
-
-        /** */
-        private final AtomicBoolean done = new AtomicBoolean();
-
-        /**
-         * @param sock Socket.
-         * @param endTime End time.
-         */
-        SocketTimeoutObject(Socket sock, long endTime) {
-            assert sock != null;
-            assert endTime > 0;
-
-            this.sock = sock;
-            this.endTime = endTime;
-        }
-
-        /**
-         * @return {@code True} if object has not yet been processed.
-         */
-        boolean cancel() {
-            return done.compareAndSet(false, true);
-        }
-
-        /**
-         * @return {@code True} if object has not yet been canceled.
-         */
-        boolean onTimeout() {
-            if (done.compareAndSet(false, true)) {
-                // Close socket - timeout occurred.
-                U.closeQuiet(sock);
-
-                return true;
-            }
-
-            return false;
-        }
-
-        /**
-         * @return End time.
-         */
-        long endTime() {
-            return endTime;
-        }
-
-        /**
-         * @return ID.
-         */
-        long id() {
-            return id;
-        }
-
-        /** {@inheritDoc} */
-        @Override public String toString() {
-            return S.toString(SocketTimeoutObject.class, this);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
index df9d0f4..95281c3 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
@@ -272,4 +272,12 @@ public interface TcpDiscoverySpiMBean extends IgniteSpiManagementMBean {
      */
     @MXBeanDescription("Dump debug info.")
     public void dumpDebugInfo();
+
+    /**
+     * Whether or not discovery is in client mode.
+     *
+     * @return {@code true} if node is in client mode.
+     */
+    @MXBeanDescription("Client mode.")
+    public boolean isClientMode();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
index b2374fb..603f66c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
@@ -45,25 +45,33 @@ public class GridReleaseTypeSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpiAdapter discoSpi;
+        TcpDiscoverySpi discoSpi;
 
         if (clientMode) {
-            discoSpi = new TcpClientDiscoverySpi() {
-                @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+            discoSpi = new TcpDiscoverySpi() {
+                @Override public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs,
+                    IgniteProductVersion ver) {
                     super.setNodeAttributes(attrs, ver);
 
                     attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
+
+                    return this;
                 }
             };
 
+            discoSpi.setClientMode(true);
+
             cfg.setClientMode(true);
         }
         else {
             discoSpi = new TcpDiscoverySpi() {
-                @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+                @Override public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs,
+                    IgniteProductVersion ver) {
                     super.setNodeAttributes(attrs, ver);
 
                     attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
+
+                    return this;
                 }
             };
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index 03fc01b..01c8377 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -89,20 +89,14 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
         cCfg.setRebalanceMode(SYNC);
         cCfg.setWriteSynchronizationMode(FULL_SYNC);
 
-        TcpDiscoverySpiAdapter disc;
+        TcpDiscoverySpi disc = new TcpDiscoverySpi();
 
         if (clientMode && ((gridName.charAt(gridName.length() - 1) - '0') & 1) != 0) {
-            disc = new TcpClientDiscoverySpi();
-
+            disc.setClientMode(true);
             cfg.setClientMode(true);
         }
-        else {
-            TcpDiscoverySpi srvDisc = new TcpDiscoverySpi();
-
-            srvDisc.setMaxMissedClientHeartbeats(50);
-
-            disc = srvDisc;
-        }
+        else
+            disc.setMaxMissedClientHeartbeats(50);
 
         disc.setHeartbeatFrequency(500);
         disc.setIpFinder(IP_FINDER);
@@ -186,8 +180,7 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
     @SuppressWarnings("BusyWait")
     private void awaitDiscovery(long nodesCnt) throws InterruptedException {
         for (Ignite g : alive) {
-            if (g.configuration().getDiscoverySpi() instanceof TcpClientDiscoverySpi)
-                ((TcpClientDiscoverySpi)g.configuration().getDiscoverySpi()).waitForMessagePrecessed();
+            ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed();
 
             while (g.cluster().nodes().size() != nodesCnt)
                 Thread.sleep(10);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
index fc953d9..dbe7d72 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
@@ -53,7 +53,7 @@ public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonA
         cfg.setDeploymentMode(mode);
         cfg.setPeerClassLoadingEnabled(p2pEnabled);
 
-        TcpDiscoverySpiAdapter discoverySpi = createDiscovery(cfg);
+        TcpDiscoverySpi discoverySpi = new TcpDiscoverySpi();
 
         discoverySpi.setIpFinder(IP_FINDER);
 
@@ -75,14 +75,6 @@ public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonA
     }
 
     /**
-     * @return Discovery SPI.
-     * @param cfg DiscoverySpi
-     */
-    protected TcpDiscoverySpiAdapter createDiscovery(IgniteConfiguration cfg) {
-        return new TcpDiscoverySpi();
-    }
-
-    /**
      * @throws Exception If failed.
      */
     public void testPreferIpV4StackTrue() throws Exception {
@@ -185,13 +177,14 @@ public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonA
      *
      */
     public static class ClientDiscovery extends GridDiscoveryManagerAttributesSelfTest {
-        /** {@inheritDoc}
-         * @param cfg*/
-        @Override protected TcpDiscoverySpiAdapter createDiscovery(IgniteConfiguration cfg) {
+        /** {@inheritDoc} */
+        @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+            IgniteConfiguration cfg = super.getConfiguration(gridName);
+
             if (Boolean.TRUE.equals(cfg.isClientMode()))
-                return new TcpClientDiscoverySpi();
+                ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientMode(true);
 
-            return super.createDiscovery(cfg);
+            return cfg;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
index 5f5bddb..7a4baa8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
@@ -70,7 +70,7 @@ public abstract class GridDiscoveryManagerSelfTest extends GridCommonAbstractTes
             cfg.setCacheConfiguration(ccfg1, ccfg2);
         }
 
-        TcpDiscoverySpiAdapter discoverySpi = createDiscovery(cfg);
+        TcpDiscoverySpi discoverySpi = new TcpDiscoverySpi();
 
         discoverySpi.setIpFinder(IP_FINDER);
 
@@ -80,14 +80,6 @@ public abstract class GridDiscoveryManagerSelfTest extends GridCommonAbstractTes
     }
 
     /**
-     * @return Discovery SPI.
-     * @param cfg DiscoverySpi
-     */
-    protected TcpDiscoverySpiAdapter createDiscovery(IgniteConfiguration cfg) {
-        return new TcpDiscoverySpi();
-    }
-
-    /**
      * @throws Exception If failed.
      */
     public void testHasNearCache() throws Exception {
@@ -207,13 +199,14 @@ public abstract class GridDiscoveryManagerSelfTest extends GridCommonAbstractTes
      *
      */
     public static class ClientDiscovery extends GridDiscoveryManagerSelfTest {
-        /** {@inheritDoc}
-         * @param cfg*/
-        @Override protected TcpDiscoverySpiAdapter createDiscovery(IgniteConfiguration cfg) {
+        /** {@inheritDoc} */
+        @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+            IgniteConfiguration cfg = super.getConfiguration(gridName);
+
             if (Boolean.TRUE.equals(cfg.isClientMode()))
-                return new TcpClientDiscoverySpi();
+                ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientMode(true);
 
-            return super.createDiscovery(cfg);
+            return cfg;
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
index 72fa72e..e22bd57 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
@@ -32,7 +32,7 @@ import java.util.concurrent.atomic.*;
 import static org.apache.ignite.cache.CacheMode.*;
 
 /**
- * Tests {@link TcpClientDiscoverySpi} with multiple client nodes that interact with a cache concurrently.
+ * Tests {@link TcpDiscoverySpi} in client mode with multiple client nodes that interact with a cache concurrently.
  */
 public class GridCacheTcpClientDiscoveryMultiThreadedTest extends GridCacheAbstractSelfTest {
     /** Server nodes count. */
@@ -78,10 +78,10 @@ public class GridCacheTcpClientDiscoveryMultiThreadedTest extends GridCacheAbstr
 
             clientFinder.setAddresses(addrs);
 
-            TcpClientDiscoverySpi discoverySpi = new TcpClientDiscoverySpi();
-            discoverySpi.setIpFinder(clientFinder);
+            cfg.setDiscoverySpi(new TcpDiscoverySpi()
+                .setClientMode(true)
+                .setIpFinder(clientFinder));
 
-            cfg.setDiscoverySpi(discoverySpi);
             cfg.setClientMode(true);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
index 21233cc..6aecdad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -47,10 +47,10 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
             cfg.setClientMode(true);
 
             if (clientDiscovery())
-                cfg.setDiscoverySpi(new TcpClientDiscoverySpi());
+                ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientMode(true);
         }
 
-        ((TcpDiscoverySpiAdapter)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
 
         return cfg;
     }
@@ -265,10 +265,7 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
 
         assertTrue(ignite.configuration().isClientMode());
 
-        if (clientDiscovery())
-            assertTrue(ignite.configuration().getDiscoverySpi() instanceof TcpClientDiscoverySpi);
-        else
-            assertTrue(ignite.configuration().getDiscoverySpi() instanceof TcpDiscoverySpi);
+        assertEquals(clientDiscovery(), ignite.configuration().getDiscoverySpi().isClientMode());
 
         return ignite;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
index 62dbb18..bd20ddc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
@@ -24,7 +24,7 @@ import org.apache.ignite.spi.discovery.tcp.*;
 import static org.apache.ignite.cache.CacheMode.*;
 
 /**
- * Tests {@link TcpClientDiscoverySpi}.
+ * Tests {@link org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi} in client mode.
  */
 @SuppressWarnings("RedundantMethodOverride")
 public abstract class GridCacheClientModesTcpClientDiscoveryAbstractTest extends GridCacheClientModesAbstractSelfTest {
@@ -38,11 +38,9 @@ public abstract class GridCacheClientModesTcpClientDiscoveryAbstractTest extends
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         if (cfg.isClientMode() != null && cfg.isClientMode()) {
-            TcpClientDiscoverySpi discoverySpi = new TcpClientDiscoverySpi();
-
-            discoverySpi.setIpFinder(ipFinder);
-
-            cfg.setDiscoverySpi(discoverySpi);
+            cfg.setDiscoverySpi(new TcpDiscoverySpi()
+                .setClientMode(true)
+                .setIpFinder(ipFinder));
         }
 
         return cfg;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
index fcc15e0..e7daec3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
@@ -55,7 +55,8 @@ public abstract class GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest
 
         // Override node attributes in discovery spi.
         TcpDiscoverySpi spi = new TcpDiscoverySpi() {
-            @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+            @Override public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs,
+                IgniteProductVersion ver) {
                 super.setNodeAttributes(attrs, ver);
 
                 // Set unique mac addresses for every group of three nodes.
@@ -64,6 +65,8 @@ public abstract class GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest
                 attrs.put(IgniteNodeAttributes.ATTR_MACS, macAddrs);
 
                 gridInstanceNum++;
+
+                return this;
             }
         };
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
index 9e6fc37..1a6146e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheSyncReplicatedPreloadSelfTest.java
@@ -58,7 +58,6 @@ public class GridCacheSyncReplicatedPreloadSelfTest extends GridCommonAbstractTe
         TcpDiscoverySpi disco = new TcpDiscoverySpi();
 
         disco.setIpFinder(ipFinder);
-        disco.setDebugMode(DISCO_DEBUG_MODE);
 
         cfg.setDiscoverySpi(disco);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
index a2dee89..e9620ea 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
@@ -37,15 +37,12 @@ public class TcpClientDiscoveryMarshallerCheckSelfTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg =  super.getConfiguration(gridName);
 
-        TcpDiscoverySpiAdapter discoSpi;
-
-        if (gridName.endsWith("0")) {
-            discoSpi = new TcpDiscoverySpi();
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
+        if (gridName.endsWith("0"))
             cfg.setMarshaller(new JdkMarshaller());
-        }
         else {
-            discoSpi = new TcpClientDiscoverySpi();
+            discoSpi.setClientMode(true);
 
             cfg.setClientMode(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
deleted file mode 100644
index 3e8b231..0000000
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiConfigSelfTest.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * 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.testframework.junits.spi.*;
-
-/**
- *
- */
-@GridSpiTest(spi = TcpClientDiscoverySpi.class, group = "Discovery SPI")
-public class TcpClientDiscoverySpiConfigSelfTest extends GridSpiAbstractConfigTest<TcpDiscoverySpi> {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testNegativeConfig() throws Exception {
-        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "ipFinder", null);
-        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "networkTimeout", 0);
-        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "socketTimeout", 0);
-        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "ackTimeout", 0);
-        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "heartbeatFrequency", 0);
-        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "threadPriority", -1);
-        checkNegativeSpiProperty(new TcpClientDiscoverySpi(), "joinTimeout", -1);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index 8157d59..1655c33 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -96,10 +96,10 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     private TcpDiscoveryVmIpFinder clientIpFinder;
 
     /** */
-    private long joinTimeout = TcpClientDiscoverySpi.DFLT_JOIN_TIMEOUT;
+    private long joinTimeout = TcpDiscoverySpi.DFLT_JOIN_TIMEOUT;
 
     /** */
-    private long netTimeout = TcpDiscoverySpiAdapter.DFLT_NETWORK_TIMEOUT;
+    private long netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT;
 
     /** */
     private boolean longSockTimeouts;
@@ -108,7 +108,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpiAdapter disco;
+        TcpDiscoverySpi disco;
 
         if (gridName.startsWith("server")) {
             disco = new TcpDiscoverySpi();
@@ -116,7 +116,9 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
             disco.setIpFinder(IP_FINDER);
         }
         else if (gridName.startsWith("client")) {
-            disco = new TestTcpClientDiscovery();
+            disco = new TestTcpDiscoverySpi();
+
+            disco.setClientMode(true);
 
             cfg.setClientMode(true);
 
@@ -186,8 +188,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
         nodeId = null;
         clientIpFinder = null;
-        joinTimeout = TcpClientDiscoverySpi.DFLT_JOIN_TIMEOUT;
-        netTimeout = TcpClientDiscoverySpi.DFLT_NETWORK_TIMEOUT;
+        joinTimeout = TcpDiscoverySpi.DFLT_JOIN_TIMEOUT;
+        netTimeout = TcpDiscoverySpi.DFLT_NETWORK_TIMEOUT;
         longSockTimeouts = false;
 
         assert G.allGrids().isEmpty();
@@ -408,14 +410,14 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         Ignite srv1 = G.ignite("server-1");
         Ignite client = G.ignite("client-0");
 
-        ((TcpDiscoverySpiAdapter)srv0.configuration().getDiscoverySpi()).setAckTimeout(1000);
+        ((TcpDiscoverySpi)srv0.configuration().getDiscoverySpi()).setAckTimeout(1000);
 
-        ((TestTcpClientDiscovery)client.configuration().getDiscoverySpi()).pauseSocketWrite();
+        ((TestTcpDiscoverySpi)client.configuration().getDiscoverySpi()).pauseSocketWrite();
 
         assert !((IgniteEx)srv1).context().discovery().pingNode(client.cluster().localNode().id());
         assert !((IgniteEx)srv0).context().discovery().pingNode(client.cluster().localNode().id());
 
-        ((TestTcpClientDiscovery)client.configuration().getDiscoverySpi()).resumeAll();
+        ((TestTcpDiscoverySpi)client.configuration().getDiscoverySpi()).resumeAll();
 
         assert ((IgniteEx)srv1).context().discovery().pingNode(client.cluster().localNode().id());
         assert ((IgniteEx)srv0).context().discovery().pingNode(client.cluster().localNode().id());
@@ -465,7 +467,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
         attachListeners(2, 3);
 
-        ((TcpClientDiscoverySpi)G.ignite("client-2").configuration().getDiscoverySpi()).brakeConnection();
+        ((TcpDiscoverySpi)G.ignite("client-2").configuration().getDiscoverySpi()).brakeConnection();
 
         G.ignite("client-2").message().remoteListen(null, new MessageListener()); // Send some discovery message.
 
@@ -488,7 +490,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
         attachListeners(2, 2);
 
-        ((TestTcpClientDiscovery)G.ignite("client-1").configuration().getDiscoverySpi()).pauseAll();
+        ((TestTcpDiscoverySpi)G.ignite("client-1").configuration().getDiscoverySpi()).pauseAll();
 
         stopGrid("server-2");
 
@@ -502,7 +504,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
         clientLeftLatch = new CountDownLatch(1);
 
-        ((TestTcpClientDiscovery)G.ignite("client-1").configuration().getDiscoverySpi()).resumeAll();
+        ((TestTcpDiscoverySpi)G.ignite("client-1").configuration().getDiscoverySpi()).resumeAll();
 
         await(clientLeftLatch);
 
@@ -535,7 +537,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         };
         G.addListener(lsnr);
 
-        final TcpClientDiscoverySpi client2Disco = (TcpClientDiscoverySpi)G.ignite("client-2").configuration().getDiscoverySpi();
+        final TcpDiscoverySpi disco = (TcpDiscoverySpi)G.ignite("client-2").configuration().getDiscoverySpi();
 
         try {
             failServer(2);
@@ -551,7 +553,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
             G.removeListener(lsnr);
         }
 
-        assert client2Disco.getRemoteNodes().isEmpty();
+        assert disco.getRemoteNodes().isEmpty();
     }
 
     /**
@@ -875,8 +877,8 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
      * @throws Exception In case of error.
      */
     private void setClientRouter(int clientIdx, int srvIdx) throws Exception {
-        TcpClientDiscoverySpi disco =
-            (TcpClientDiscoverySpi)G.ignite("client-" + clientIdx).configuration().getDiscoverySpi();
+        TcpDiscoverySpi disco =
+            (TcpDiscoverySpi)G.ignite("client-" + clientIdx).configuration().getDiscoverySpi();
 
         TcpDiscoveryVmIpFinder ipFinder = (TcpDiscoveryVmIpFinder)disco.getIpFinder();
 
@@ -923,7 +925,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
      * @param idx Index.
      */
     private void failClient(int idx) {
-        ((TcpClientDiscoverySpi)G.ignite("client-" + idx).configuration().getDiscoverySpi()).simulateNodeFailure();
+        ((TcpDiscoverySpi)G.ignite("client-" + idx).configuration().getDiscoverySpi()).simulateNodeFailure();
     }
 
     /**
@@ -1034,7 +1036,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         for (int i = 0; i < clientCnt; i++) {
             Ignite g = G.ignite("client-" + i);
 
-            ((TcpClientDiscoverySpi)g.configuration().getDiscoverySpi()).waitForMessagePrecessed();
+            ((TcpDiscoverySpi)g.configuration().getDiscoverySpi()).waitForClientMessagePrecessed();
 
             assertTrue(clientNodeIds.contains(g.cluster().localNode().id()));
 
@@ -1094,7 +1096,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     /**
      *
      */
-    private static class TestTcpClientDiscovery extends TcpClientDiscoverySpi {
+    private static class TestTcpDiscoverySpi extends TcpDiscoverySpi {
         /** */
         private final Object mux = new Object();
 
@@ -1162,7 +1164,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         public void pauseAll() {
             pauseResumeOperation(true, openSockLock, writeLock);
 
-            msgWorker.suspend();
+            impl.workerThread().suspend();
         }
 
         /**
@@ -1171,7 +1173,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
         public void resumeAll() {
             pauseResumeOperation(false, openSockLock, writeLock);
 
-            msgWorker.resume();
+            impl.workerThread().resume();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
index d8512e3..c64b5a8 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
@@ -44,7 +44,9 @@ public class TcpDiscoveryConcurrentStartTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg =  super.getConfiguration(gridName);
 
-        TcpDiscoverySpiAdapter discoSpi = client ? new TcpClientDiscoverySpi() : new TcpDiscoverySpi();
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
+
+        discoSpi.setClientMode(client);
 
         discoSpi.setIpFinder(ipFinder);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
index 6509a6d..8c287b1 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
@@ -71,13 +71,11 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
         if (client()) {
-            TcpClientDiscoverySpi spi = new TcpClientDiscoverySpi();
-
-            spi.setIpFinder(ipFinder);
-
             cfg.setClientMode(true);
 
-            cfg.setDiscoverySpi(spi);
+            cfg.setDiscoverySpi(new TcpDiscoverySpi()
+                .setClientMode(true)
+                .setIpFinder(ipFinder));
         }
         else {
             TcpDiscoverySpi spi = new TcpDiscoverySpi();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
index 5648c31..ad12753 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySelfTest.java
@@ -175,7 +175,7 @@ public class TcpDiscoverySelfTest extends GridCommonAbstractTest {
                 }
             }, 4, "grid-starter");
 
-            Collection<TcpDiscoveryNode> nodes = discoMap.get(g1.name()).ring().allNodes();
+            Collection<TcpDiscoveryNode> nodes = ((ServerImpl)discoMap.get(g1.name()).impl).ring().allNodes();
 
             ByteArrayOutputStream bos = new ByteArrayOutputStream();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java
index 3e895be..da9eed9 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java
@@ -41,5 +41,13 @@ public class TcpDiscoverySpiConfigSelfTest extends GridSpiAbstractConfigTest<Tcp
         checkNegativeSpiProperty(new TcpDiscoverySpi(), "threadPriority", -1);
         checkNegativeSpiProperty(new TcpDiscoverySpi(), "maxMissedHeartbeats", 0);
         checkNegativeSpiProperty(new TcpDiscoverySpi(), "statisticsPrintFrequency", 0);
+
+        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "ipFinder", null);
+        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "networkTimeout", 0);
+        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "socketTimeout", 0);
+        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "ackTimeout", 0);
+        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "heartbeatFrequency", 0);
+        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "threadPriority", -1);
+        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "joinTimeout", -1);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 2cae73a..9c42920 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -676,11 +676,11 @@ public abstract class GridAbstractTest extends TestCase {
     protected IgniteConfiguration optimize(IgniteConfiguration cfg) throws IgniteCheckedException {
         // TODO: IGNITE-605: propose another way to avoid network overhead in tests.
         if (cfg.getLocalHost() == null) {
-            if (cfg.getDiscoverySpi() instanceof TcpDiscoverySpiAdapter) {
+            if (cfg.getDiscoverySpi() instanceof TcpDiscoverySpi) {
                 cfg.setLocalHost("127.0.0.1");
 
-                if (((TcpDiscoverySpiAdapter)cfg.getDiscoverySpi()).getJoinTimeout() == 0)
-                    ((TcpDiscoverySpiAdapter)cfg.getDiscoverySpi()).setJoinTimeout(8000);
+                if (((TcpDiscoverySpi)cfg.getDiscoverySpi()).getJoinTimeout() == 0)
+                    ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setJoinTimeout(8000);
             }
             else
                 cfg.setLocalHost(getTestResources().getLocalHost());
@@ -740,7 +740,7 @@ public abstract class GridAbstractTest extends TestCase {
         Collection<Ignite> srvs = new ArrayList<>();
 
         for (Ignite g : G.allGrids()) {
-            if (g.configuration().getDiscoverySpi() instanceof TcpClientDiscoverySpi)
+            if (g.configuration().getDiscoverySpi().isClientMode())
                 clients.add(g);
             else
                 srvs.add(g);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/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 8bf8dbc..dc35b24 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
@@ -52,7 +52,6 @@ public class IgniteSpiDiscoverySelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridTcpSpiForwardingSelfTest.class));
 
         suite.addTest(new TestSuite(TcpClientDiscoverySpiSelfTest.class));
-        suite.addTest(new TestSuite(TcpClientDiscoverySpiConfigSelfTest.class));
         suite.addTest(new TestSuite(TcpClientDiscoveryMarshallerCheckSelfTest.class));
 
         return suite;


[09/53] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-709_2

Posted by se...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-709_2


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

Branch: refs/heads/ignite-sprint-5
Commit: 9d3ab162f3a691fdb9e758a60dbd04dafe06b6d0
Parents: 9188f4e 4be517c
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 27 18:06:52 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 27 18:06:52 2015 +0300

----------------------------------------------------------------------
 dev-tools/slurp.sh                              |  14 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      | 265 +++++++++++++------
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../processors/cache/IgniteCacheProxy.java      |   3 +-
 .../GridDistributedTxRemoteAdapter.java         |  10 +-
 .../processors/query/GridQueryIndexing.java     |  16 ++
 .../processors/query/GridQueryProcessor.java    |  52 ++--
 .../core/src/main/resources/ignite.properties   |   2 +-
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 .../query/h2/GridH2IndexingGeoSelfTest.java     |  20 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |  17 ++
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  18 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |   4 +-
 .../query/h2/sql/GridSqlOperationType.java      |   2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   2 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |   2 +-
 .../local/IgniteCacheLocalQuerySelfTest.java    |   6 +
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 modules/visor-console/pom.xml                   |   2 +-
 .../visor/commands/node/VisorNodeCommand.scala  |   2 +-
 .../commands/tasks/VisorTasksCommand.scala      |   2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |  63 ++++-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |  13 +-
 49 files changed, 390 insertions(+), 183 deletions(-)
----------------------------------------------------------------------



[20/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Remove TcpDiscoverySpi.clientMode, create TcpDiscoverySpi.forceServerMode

Posted by se...@apache.org.
# IGNITE-943 Remove TcpDiscoverySpi.clientMode, create TcpDiscoverySpi.forceServerMode


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

Branch: refs/heads/ignite-sprint-5
Commit: e2e90c4bd57ec74c6816f1c595ff8ff867b5a757
Parents: 77d1dcb
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 14:45:15 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 14:46:52 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/IgnitionEx.java  | 13 +------
 .../discovery/GridDiscoveryManager.java         |  2 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |  5 ++-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 39 +++++++++++++-------
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |  5 ++-
 .../internal/GridReleaseTypeSelfTest.java       |  2 -
 .../GridDiscoveryManagerAliveCacheSelfTest.java |  4 +-
 .../GridDiscoveryManagerAttributesSelfTest.java | 17 ++++-----
 .../discovery/GridDiscoveryManagerSelfTest.java | 17 ++++-----
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  4 +-
 .../GridCacheClientModesAbstractSelfTest.java   |  3 ++
 ...ientModesTcpClientDiscoveryAbstractTest.java |  6 +--
 ...pClientDiscoveryMarshallerCheckSelfTest.java |  8 +---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 35 ++----------------
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |  8 +---
 .../tcp/TcpDiscoveryMultiThreadedTest.java      | 16 +-------
 .../tcp/TcpDiscoverySpiConfigSelfTest.java      |  8 ----
 18 files changed, 65 insertions(+), 131 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 7eb6262..6e4efb5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1743,13 +1743,6 @@ public class IgnitionEx {
                 myCfg.setFileSystemConfiguration(clone);
             }
 
-            if (myCfg.isClientMode() == null || !myCfg.isClientMode()) {
-                if (myCfg.getDiscoverySpi() != null && myCfg.getDiscoverySpi().isClientMode()) {
-                    throw new IgniteCheckedException("DiscoverySpi is in client mode, but node is not in client mode" +
-                        "(consider changing 'IgniteConfiguration.clientMode' to 'true').");
-                }
-            }
-
             initializeDefaultSpi(myCfg);
 
             initializeDefaultCacheConfiguration(myCfg);
@@ -1814,13 +1807,9 @@ public class IgnitionEx {
          * @param cfg Ignite configuration.
          */
         private void initializeDefaultSpi(IgniteConfiguration cfg) {
-            if (cfg.getDiscoverySpi() == null) {
+            if (cfg.getDiscoverySpi() == null)
                 cfg.setDiscoverySpi(new TcpDiscoverySpi());
 
-                if (Boolean.TRUE.equals(cfg.isClientMode()))
-                    ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientMode(true);
-            }
-
             if (cfg.getDiscoverySpi() instanceof TcpDiscoverySpi) {
                 TcpDiscoverySpi tcpDisco = (TcpDiscoverySpi)cfg.getDiscoverySpi();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 0f79084..a910950 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -285,7 +285,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** {@inheritDoc} */
     @Override protected void onKernalStart0() throws IgniteCheckedException {
         if (Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode())
-            ctx.performance().add("Enable client mode for TcpDiscoverySpi (set TcpDiscoverySpi.clientMode to true)");
+            ctx.performance().add("Enable client mode for TcpDiscoverySpi (set TcpDiscoverySpi.forceServerMode to true)");
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index f9cec34..e7fbadc 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -154,9 +154,10 @@ public interface DiscoverySpi extends IgniteSpi {
     public void failNode(UUID nodeId);
 
     /**
-     * Whether or not discovery is in client mode.
+     * Whether or not discovery is started in client mode.
      *
      * @return {@code true} if node is in client mode.
+     * @throws IllegalStateException If discovery SPI has not started.
      */
-    public boolean isClientMode();
+    public boolean isClientMode() throws IllegalStateException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/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 aa8fb76..5a2ddd6 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
@@ -53,9 +53,11 @@ import java.util.concurrent.atomic.*;
  * Nodes are organized in ring. So almost all network exchange (except few cases) is
  * done across it.
  * <p>
- * Node may be started in client mode, in this case node does not insert to the ring,
+ * If node is configured as client node (see {@link IgniteConfiguration#clientMode})
+ * TcpDiscoverySpi starts in client mode too. In this case node does not insert to the ring,
  * it connects to any node in the ring router and communicated with that node only.
- * Thereby slowing or shutdown of client node will not affect whole cluster.
+ * Thereby slowing or shutdown of client node will not affect whole cluster. If you want to start TcpDiscoverySpi in
+ * server mode regardless {@link IgniteConfiguration#clientMode} you can set {@link #forceSrvMode} to true.
  * <p>
  * At startup SPI tries to send messages to random IP taken from
  * {@link TcpDiscoveryIpFinder} about self start (stops when send succeeds)
@@ -97,6 +99,7 @@ import java.util.concurrent.atomic.*;
  * <li>Thread priority for threads started by SPI (see {@link #setThreadPriority(int)})</li>
  * <li>IP finder clean frequency (see {@link #setIpFinderCleanFrequency(long)})</li>
  * <li>Statistics print frequency (see {@link #setStatisticsPrintFrequency(long)}</li>
+ * <li>Force server mode (see {@link #setForceServerMode(boolean)}</li>
  * </ul>
  * <h2 class="header">Java Example</h2>
  * <pre name="code" class="java">
@@ -309,7 +312,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     protected TcpDiscoveryImpl impl;
 
     /** */
-    private boolean clientMode;
+    private boolean forceSrvMode;
 
     /** {@inheritDoc} */
     @Override public String getSpiState() {
@@ -368,20 +371,30 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
 
     /** {@inheritDoc} */
     @Override public boolean isClientMode() {
-        return clientMode;
+        if (impl == null)
+            throw new IllegalStateException("TcpDiscoverySpi has not started");
+
+        return impl instanceof ClientImpl;
     }
 
     /**
-     * @param clientMode New client mode.
+     * If {@code true} TcpDiscoverySpi will started in server mode regardless
+     * of {@link IgniteConfiguration#isClientMode()}
+     *
+     * @return forceServerMode flag.
      */
-    @IgniteSpiConfiguration(optional = true)
-    public TcpDiscoverySpi setClientMode(boolean clientMode) {
-        if (impl != null)
-            throw new IllegalStateException("You cannot change mode, TcpDiscoverySpi already started.");
-
-        this.clientMode = clientMode;
+    public boolean isForceServerMode() {
+        return forceSrvMode;
+    }
 
-        return this;
+    /**
+     * If {@code true} TcpDiscoverySpi will started in server mode regardless
+     * of {@link IgniteConfiguration#isClientMode()}
+     *
+     * @param forceSrvMode forceServerMode flag.
+     */
+    public void setForceServerMode(boolean forceSrvMode) {
+        this.forceSrvMode = forceSrvMode;
     }
 
     /**
@@ -1500,7 +1513,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
 
     /** {@inheritDoc} */
     @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        if (clientMode) {
+        if (!forceSrvMode && (Boolean.TRUE.equals(ignite.configuration().isClientMode()))) {
             if (ackTimeout == 0)
                 ackTimeout = DFLT_ACK_TIMEOUT_CLIENT;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
index 95281c3..6f2ea6f 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
@@ -274,10 +274,11 @@ public interface TcpDiscoverySpiMBean extends IgniteSpiManagementMBean {
     public void dumpDebugInfo();
 
     /**
-     * Whether or not discovery is in client mode.
+     * Whether or not discovery is started in client mode.
      *
      * @return {@code true} if node is in client mode.
+     * @throws IllegalStateException If discovery SPI has not started.
      */
     @MXBeanDescription("Client mode.")
-    public boolean isClientMode();
+    public boolean isClientMode() throws IllegalStateException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
index e6a69cc..79e8905 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
@@ -57,8 +57,6 @@ public class GridReleaseTypeSelfTest extends GridCommonAbstractTest {
                 }
             };
 
-            discoSpi.setClientMode(true);
-
             cfg.setClientMode(true);
         }
         else {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index f928f6c..af4dd0f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -91,10 +91,8 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
 
         TcpDiscoverySpi disc = new TcpDiscoverySpi();
 
-        if (clientMode && ((gridName.charAt(gridName.length() - 1) - '0') & 1) != 0) {
-            disc.setClientMode(true);
+        if (clientMode && ((gridName.charAt(gridName.length() - 1) - '0') & 1) != 0)
             cfg.setClientMode(true);
-        }
         else
             disc.setMaxMissedClientHeartbeats(50);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
index dbe7d72..e76c615 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAttributesSelfTest.java
@@ -170,21 +170,20 @@ public abstract class GridDiscoveryManagerAttributesSelfTest extends GridCommonA
      *
      */
     public static class RegularDiscovery extends GridDiscoveryManagerAttributesSelfTest {
-        // No-op.
-    }
-
-    /**
-     *
-     */
-    public static class ClientDiscovery extends GridDiscoveryManagerAttributesSelfTest {
         /** {@inheritDoc} */
         @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
             IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-            if (Boolean.TRUE.equals(cfg.isClientMode()))
-                ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientMode(true);
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
 
             return cfg;
         }
     }
+
+    /**
+     *
+     */
+    public static class ClientDiscovery extends GridDiscoveryManagerAttributesSelfTest {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
index 7a4baa8..65aec49 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
@@ -192,21 +192,20 @@ public abstract class GridDiscoveryManagerSelfTest extends GridCommonAbstractTes
      *
      */
     public static class RegularDiscovery extends GridDiscoveryManagerSelfTest {
-        // No-op.
-    }
-
-    /**
-     *
-     */
-    public static class ClientDiscovery extends GridDiscoveryManagerSelfTest {
         /** {@inheritDoc} */
         @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
             IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-            if (Boolean.TRUE.equals(cfg.isClientMode()))
-                ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientMode(true);
+            ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
 
             return cfg;
         }
     }
+
+    /**
+     *
+     */
+    public static class ClientDiscovery extends GridDiscoveryManagerSelfTest {
+        // No-op.
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
index e22bd57..b817f4c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTcpClientDiscoveryMultiThreadedTest.java
@@ -78,9 +78,7 @@ public class GridCacheTcpClientDiscoveryMultiThreadedTest extends GridCacheAbstr
 
             clientFinder.setAddresses(addrs);
 
-            cfg.setDiscoverySpi(new TcpDiscoverySpi()
-                .setClientMode(true)
-                .setIpFinder(clientFinder));
+            cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(clientFinder));
 
             cfg.setClientMode(true);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
index 6aecdad..5a6be8e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/IgniteClientDataStructuresAbstractTest.java
@@ -46,8 +46,8 @@ public abstract class IgniteClientDataStructuresAbstractTest extends GridCommonA
         if (gridName.equals(getTestGridName(NODE_CNT - 1))) {
             cfg.setClientMode(true);
 
-            if (clientDiscovery())
-                ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientMode(true);
+            if (!clientDiscovery())
+                ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
         }
 
         ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
index a665601..f996568 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesAbstractSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 
 import java.io.*;
 import java.util.concurrent.atomic.*;
@@ -66,6 +67,8 @@ public abstract class GridCacheClientModesAbstractSelfTest extends GridCacheAbst
             nearOnlyGridName = gridName;
         }
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
index bd20ddc..bbc9144 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheClientModesTcpClientDiscoveryAbstractTest.java
@@ -37,11 +37,7 @@ public abstract class GridCacheClientModesTcpClientDiscoveryAbstractTest extends
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        if (cfg.isClientMode() != null && cfg.isClientMode()) {
-            cfg.setDiscoverySpi(new TcpDiscoverySpi()
-                .setClientMode(true)
-                .setIpFinder(ipFinder));
-        }
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(false);
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
index e9620ea..016854a 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoveryMarshallerCheckSelfTest.java
@@ -37,21 +37,15 @@ public class TcpClientDiscoveryMarshallerCheckSelfTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg =  super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
         if (gridName.endsWith("0"))
             cfg.setMarshaller(new JdkMarshaller());
         else {
-            discoSpi.setClientMode(true);
-
             cfg.setClientMode(true);
 
             cfg.setMarshaller(new OptimizedMarshaller());
         }
 
-        discoSpi.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder));
 
         return cfg;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index 0d08a0d..be3474d 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -103,26 +103,16 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     /** */
     private boolean longSockTimeouts;
 
-    /** */
-    private Boolean cfgClientMode;
-
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi disco;
-
-        if (gridName.startsWith("server")) {
-            disco = new TcpDiscoverySpi();
+        TcpDiscoverySpi disco = new TestTcpDiscoverySpi();
 
+        if (gridName.startsWith("server"))
             disco.setIpFinder(IP_FINDER);
-        }
         else if (gridName.startsWith("client")) {
-            disco = new TestTcpDiscoverySpi();
-
-            disco.setClientMode(true);
-
-            cfg.setClientMode(cfgClientMode == null ? true : cfgClientMode);
+            cfg.setClientMode(true);
 
             TcpDiscoveryVmIpFinder ipFinder;
 
@@ -875,25 +865,6 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     }
 
     /**
-     *
-     */
-    public void testBrokenConfiguration() throws Exception {
-        startServerNodes(1);
-
-        cfgClientMode = false;
-
-        try {
-            startClientNodes(1);
-
-            fail("Configuration is boken, node cannot be started with DiscoverySpi.clientMode = true and " +
-                "IgniteConfiguration.clientMode = false");
-        }
-        catch (Exception e) {
-            assertTrue(e.getMessage(), e.getMessage().contains("DiscoverySpi is in client mode"));
-        }
-    }
-
-    /**
      * @param clientIdx Index.
      * @throws Exception In case of error.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
index c64b5a8..6438268 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryConcurrentStartTest.java
@@ -44,13 +44,7 @@ public class TcpDiscoveryConcurrentStartTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg =  super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setClientMode(client);
-
-        discoSpi.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder));
 
         cfg.setCacheConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
index 8c287b1..cfefff4 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryMultiThreadedTest.java
@@ -70,20 +70,10 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        if (client()) {
+        if (client())
             cfg.setClientMode(true);
 
-            cfg.setDiscoverySpi(new TcpDiscoverySpi()
-                .setClientMode(true)
-                .setIpFinder(ipFinder));
-        }
-        else {
-            TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-            spi.setIpFinder(ipFinder);
-
-            cfg.setDiscoverySpi(spi);
-        }
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder));
 
         cfg.setCacheConfiguration();
 
@@ -91,8 +81,6 @@ public class TcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
 
         cfg.setIncludeProperties();
 
-        cfg.setLocalHost("127.0.0.1");
-
         return cfg;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e2e90c4b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java
index da9eed9..3e895be 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiConfigSelfTest.java
@@ -41,13 +41,5 @@ public class TcpDiscoverySpiConfigSelfTest extends GridSpiAbstractConfigTest<Tcp
         checkNegativeSpiProperty(new TcpDiscoverySpi(), "threadPriority", -1);
         checkNegativeSpiProperty(new TcpDiscoverySpi(), "maxMissedHeartbeats", 0);
         checkNegativeSpiProperty(new TcpDiscoverySpi(), "statisticsPrintFrequency", 0);
-
-        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "ipFinder", null);
-        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "networkTimeout", 0);
-        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "socketTimeout", 0);
-        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "ackTimeout", 0);
-        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "heartbeatFrequency", 0);
-        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "threadPriority", -1);
-        checkNegativeSpiProperty(new TcpDiscoverySpi().setClientMode(true), "joinTimeout", -1);
     }
 }


[18/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Added test for broken configuration.

Posted by se...@apache.org.
# IGNITE-943 Added test for broken configuration.


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

Branch: refs/heads/ignite-sprint-5
Commit: b3ac88f8f6613c1cef89657cd2118134f4ad4a06
Parents: b0d9ef8
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 12:45:19 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 14:46:08 2015 +0300

----------------------------------------------------------------------
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 24 +++++++++++++++++++-
 1 file changed, 23 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3ac88f8/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index f27ef33..0d08a0d 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -103,6 +103,9 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     /** */
     private boolean longSockTimeouts;
 
+    /** */
+    private Boolean cfgClientMode;
+
     /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
@@ -119,7 +122,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
             disco.setClientMode(true);
 
-            cfg.setClientMode(true);
+            cfg.setClientMode(cfgClientMode == null ? true : cfgClientMode);
 
             TcpDiscoveryVmIpFinder ipFinder;
 
@@ -872,6 +875,25 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
     }
 
     /**
+     *
+     */
+    public void testBrokenConfiguration() throws Exception {
+        startServerNodes(1);
+
+        cfgClientMode = false;
+
+        try {
+            startClientNodes(1);
+
+            fail("Configuration is boken, node cannot be started with DiscoverySpi.clientMode = true and " +
+                "IgniteConfiguration.clientMode = false");
+        }
+        catch (Exception e) {
+            assertTrue(e.getMessage(), e.getMessage().contains("DiscoverySpi is in client mode"));
+        }
+    }
+
+    /**
      * @param clientIdx Index.
      * @throws Exception In case of error.
      */


[10/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix tests.

Posted by se...@apache.org.
# IGNITE-943 Fix tests.


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

Branch: refs/heads/ignite-sprint-5
Commit: bb35c8814e092b420157a0119dd31104df6093db
Parents: 4117f68
Author: sevdokimov <se...@gridgain.com>
Authored: Wed May 27 18:07:00 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Wed May 27 18:07:00 2015 +0300

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/ignite/internal/IgnitionEx.java  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bb35c881/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 13b015b..42d9cf9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1744,7 +1744,7 @@ public class IgnitionEx {
             }
 
             if (myCfg.isClientMode() == null || !myCfg.isClientMode()) {
-                if (myCfg.getDiscoverySpi().isClientMode()) {
+                if (myCfg.getDiscoverySpi() != null && myCfg.getDiscoverySpi().isClientMode()) {
                     throw new IgniteCheckedException("DiscoverySpi is in client mode, but node is not in client mode" +
                         "(consider changing 'IgniteConfiguration.clientMode' to 'true').");
                 }


[36/53] [abbrv] incubator-ignite git commit: # IGNITE-709 Fix IgfsOneClientNodeTest

Posted by se...@apache.org.
# IGNITE-709 Fix IgfsOneClientNodeTest


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

Branch: refs/heads/ignite-sprint-5
Commit: 2bc07956a828ac836641f4fbe6ae0db7b3cc1793
Parents: 2799c3a
Author: sevdokimov <se...@jetbrains.com>
Authored: Thu May 28 21:47:48 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Thu May 28 21:47:48 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 112 ++++++++-----------
 .../processors/igfs/IgfsOneClientNodeTest.java  |   8 +-
 2 files changed, 47 insertions(+), 73 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2bc07956/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 59e25fc..b5c9519 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
@@ -810,84 +810,61 @@ class ServerImpl extends TcpDiscoveryImpl {
             boolean retry = false;
             Collection<Exception> errs = new ArrayList<>();
 
-            for (int j = 2; --j >= 0;) {
-                for (InetSocketAddress addr : addrs) {
-                    Socket sock = null;
-                    Exception ex = null;
+            for (InetSocketAddress addr : addrs) {
+                try {
+                    Integer res = sendMessageDirectly(joinReq, addr);
 
-                    try {
-                        sock = spi.openSocket(addr);
-                    }
-                    catch (Exception e) {
-                        if (j > 0)
-                            continue;
+                    assert res != null;
 
-                        ex = e;
-                    }
+                    noResAddrs.remove(addr);
 
-                    if (ex == null) {
-                        try {
-                            Integer res = sendMessageDirectly(joinReq, addr, sock);
+                    // Address is responsive, reset period start.
+                    noResStart = 0;
 
-                            assert res != null;
+                    switch (res) {
+                        case RES_WAIT:
+                            // Concurrent startup, try sending join request again or wait if no success.
+                            retry = true;
 
-                            noResAddrs.remove(addr);
+                            break;
+                        case RES_OK:
+                            if (log.isDebugEnabled())
+                                log.debug("Join request message has been sent to address [addr=" + addr +
+                                    ", req=" + joinReq + ']');
 
-                            // Address is responsive, reset period start.
-                            noResStart = 0;
+                            // Join request sending succeeded, wait for response from topology.
+                            return true;
 
-                            switch (res) {
-                                case RES_WAIT:
-                                    // Concurrent startup, try sending join request again or wait if no success.
+                        default:
+                            // Concurrent startup, try next node.
+                            if (res == RES_CONTINUE_JOIN) {
+                                if (!fromAddrs.contains(addr))
                                     retry = true;
+                            }
+                            else {
+                                if (log.isDebugEnabled())
+                                    log.debug("Unexpected response to join request: " + res);
 
-                                    break;
-                                case RES_OK:
-                                    if (log.isDebugEnabled())
-                                        log.debug("Join request message has been sent to address [addr=" + addr +
-                                            ", req=" + joinReq + ']');
-
-                                    // Join request sending succeeded, wait for response from topology.
-                                    return true;
-
-                                default:
-                                    // Concurrent startup, try next node.
-                                    if (res == RES_CONTINUE_JOIN) {
-                                        if (!fromAddrs.contains(addr))
-                                            retry = true;
-                                    }
-                                    else {
-                                        if (log.isDebugEnabled())
-                                            log.debug("Unexpected response to join request: " + res);
-
-                                        retry = true;
-                                    }
-
-                                    break;
+                                retry = true;
                             }
-                        }
-                        catch (IgniteSpiException e) {
-                            e.printStackTrace();
 
-                            ex = e;
-                        }
+                            break;
                     }
+                }
+                catch (IgniteSpiException e) {
+                    errs.add(e);
 
-                    if (ex != null) {
-                        errs.add(ex);
-
-                        if (log.isDebugEnabled()) {
-                            IOException ioe = X.cause(ex, IOException.class);
-
-                            log.debug("Failed to send join request message [addr=" + addr +
-                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']');
+                    if (log.isDebugEnabled()) {
+                        IOException ioe = X.cause(e, IOException.class);
 
-                            onException("Failed to send join request message [addr=" + addr +
-                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']', ioe);
-                        }
+                        log.debug("Failed to send join request message [addr=" + addr +
+                            ", msg=" + (ioe != null ? ioe.getMessage() : e.getMessage()) + ']');
 
-                        noResAddrs.add(addr);
+                        onException("Failed to send join request message [addr=" + addr +
+                            ", msg=" + (ioe != null ? ioe.getMessage() : e.getMessage()) + ']', ioe);
                     }
+
+                    noResAddrs.add(addr);
                 }
             }
 
@@ -950,7 +927,7 @@ class ServerImpl extends TcpDiscoveryImpl {
      * @return Response read from the recipient or {@code null} if no response is supposed.
      * @throws IgniteSpiException If an error occurs.
      */
-    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr, Socket sock)
+    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr)
         throws IgniteSpiException {
         assert msg != null;
         assert addr != null;
@@ -972,11 +949,12 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             boolean openSock = false;
 
+            Socket sock = null;
+
             try {
                 long tstamp = U.currentTimeMillis();
 
-                if (sock == null)
-                    sock = spi.openSocket(addr);
+                sock = spi.openSocket(addr);
 
                 openSock = true;
 
@@ -1060,8 +1038,6 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
             finally {
                 U.closeQuiet(sock);
-
-                sock = null;
             }
         }
 
@@ -2718,7 +2694,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             for (InetSocketAddress addr : spi.getNodeAddresses(node, U.sameMacs(locNode, node))) {
                 try {
-                    sendMessageDirectly(msg, addr, null);
+                    sendMessageDirectly(msg, addr);
 
                     ex = null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2bc07956/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
index 49ddb03..3498cd9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/igfs/IgfsOneClientNodeTest.java
@@ -52,11 +52,9 @@ public class IgfsOneClientNodeTest extends GridCommonAbstractTest {
 
         cfg.setClientMode(true);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(new TcpDiscoveryVmIpFinder(true));
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi()
+            .setForceServerMode(true)
+            .setIpFinder(new TcpDiscoveryVmIpFinder(true)));
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 


[04/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Merge TcpDiscoverySpi and TcpClientDiscoverySpi

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/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
new file mode 100644
index 0000000..7e1f592
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -0,0 +1,4792 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.events.*;
+import org.apache.ignite.internal.processors.security.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.io.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.security.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.internal.*;
+import org.apache.ignite.spi.discovery.tcp.messages.*;
+import org.jetbrains.annotations.*;
+import org.jsr166.*;
+
+import java.io.*;
+import java.net.*;
+import java.text.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.IgniteNodeAttributes.*;
+import static org.apache.ignite.spi.IgnitePortProtocol.*;
+import static org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState.*;
+import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage.*;
+
+/**
+ *
+ */
+class ServerImpl extends TcpDiscoveryImpl {
+    /** */
+    private final Executor utilityPool = new ThreadPoolExecutor(0, 1, 2000, TimeUnit.MILLISECONDS,
+        new LinkedBlockingQueue<Runnable>());
+
+    /** Nodes ring. */
+    @GridToStringExclude
+    private final TcpDiscoveryNodesRing ring = new TcpDiscoveryNodesRing();
+
+    /** Topology snapshots history. */
+    private final SortedMap<Long, Collection<ClusterNode>> topHist = new TreeMap<>();
+
+    /** Socket readers. */
+    private final Collection<SocketReader> readers = new LinkedList<>();
+
+    /** TCP server for discovery SPI. */
+    private TcpServer tcpSrvr;
+
+    /** Message worker. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private RingMessageWorker msgWorker;
+
+    /** Client message workers. */
+    private ConcurrentMap<UUID, ClientMessageWorker> clientMsgWorkers = new ConcurrentHashMap8<>();
+
+    /** Metrics sender. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private HeartbeatsSender hbsSnd;
+
+    /** Status checker. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private CheckStatusSender chkStatusSnd;
+
+    /** IP finder cleaner. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private IpFinderCleaner ipFinderCleaner;
+
+    /** Statistics printer thread. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private StatisticsPrinter statsPrinter;
+
+    /** Failed nodes (but still in topology). */
+    private Collection<TcpDiscoveryNode> failedNodes = new HashSet<>();
+
+    /** Leaving nodes (but still in topology). */
+    private Collection<TcpDiscoveryNode> leavingNodes = new HashSet<>();
+
+    /** If non-shared IP finder is used this flag shows whether IP finder contains local address. */
+    private boolean ipFinderHasLocAddr;
+
+    /** Addresses that do not respond during join requests send (for resolving concurrent start). */
+    private final Collection<SocketAddress> noResAddrs = new GridConcurrentHashSet<>();
+
+    /** Addresses that incoming join requests send were send from (for resolving concurrent start). */
+    private final Collection<SocketAddress> fromAddrs = new GridConcurrentHashSet<>();
+
+    /** Response on join request from coordinator (in case of duplicate ID or auth failure). */
+    private final GridTuple<TcpDiscoveryAbstractMessage> joinRes = F.t1();
+
+    /** Node authenticator. */
+    private DiscoverySpiNodeAuthenticator nodeAuth;
+
+    /** Mutex. */
+    private final Object mux = new Object();
+
+    /** Discovery state. */
+    protected TcpDiscoverySpiState spiState = DISCONNECTED;
+
+    /** Map with proceeding ping requests. */
+    private final ConcurrentMap<InetSocketAddress, IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>>> pingMap =
+        new ConcurrentHashMap8<>();
+
+    /** Debug mode. */
+    private boolean debugMode;
+
+    /** Debug messages history. */
+    private int debugMsgHist = 512;
+
+    /** Received messages. */
+    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+    private ConcurrentLinkedDeque<String> debugLog;
+
+    /**
+     * @param adapter Adapter.
+     */
+    ServerImpl(TcpDiscoverySpi adapter) {
+        super(adapter);
+    }
+
+    /**
+     * This method is intended for troubleshooting purposes only.
+     *
+     * @param debugMode {code True} to start SPI in debug mode.
+     */
+    public void setDebugMode(boolean debugMode) {
+        this.debugMode = debugMode;
+    }
+
+    /**
+     * This method is intended for troubleshooting purposes only.
+     *
+     * @param debugMsgHist Message history log size.
+     */
+    public void setDebugMessageHistory(int debugMsgHist) {
+        this.debugMsgHist = debugMsgHist;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSpiState() {
+        synchronized (mux) {
+            return spiState.name();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMessageWorkerQueueSize() {
+        return msgWorker.queueSize();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public UUID getCoordinator() {
+        TcpDiscoveryNode crd = resolveCoordinator();
+
+        return crd != null ? crd.id() : null;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
+        assert nodeId != null;
+
+        UUID locNodeId0 = getLocalNodeId();
+
+        if (locNodeId0 != null && locNodeId0.equals(nodeId))
+            // Return local node directly.
+            return locNode;
+
+        TcpDiscoveryNode node = ring.node(nodeId);
+
+        if (node != null && !node.visible())
+            return null;
+
+        return node;
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> getRemoteNodes() {
+        return F.upcast(ring.visibleRemoteNodes());
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(String gridName) throws IgniteSpiException {
+        synchronized (mux) {
+            spiState = DISCONNECTED;
+        }
+
+        if (debugMode) {
+            if (!log.isInfoEnabled())
+                throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " +
+                    "in debug mode.");
+
+            debugLog = new ConcurrentLinkedDeque<>();
+
+            U.quietAndWarn(log, "TCP discovery SPI is configured in debug mode.");
+        }
+
+        // Clear addresses collections.
+        fromAddrs.clear();
+        noResAddrs.clear();
+
+        msgWorker = new RingMessageWorker();
+        msgWorker.start();
+
+        tcpSrvr = new TcpServer();
+
+        adapter.initLocalNode(tcpSrvr.port, true);
+
+        locNode = adapter.locNode;
+
+        // Start TCP server thread after local node is initialized.
+        tcpSrvr.start();
+
+        ring.localNode(locNode);
+
+        if (adapter.ipFinder.isShared())
+            registerLocalNodeAddress();
+        else {
+            if (F.isEmpty(adapter.ipFinder.getRegisteredAddresses()))
+                throw new IgniteSpiException("Non-shared IP finder must have IP addresses specified in " +
+                    "GridTcpDiscoveryIpFinder.getRegisteredAddresses() configuration property " +
+                    "(specify list of IP addresses in configuration).");
+
+            ipFinderHasLocAddr = adapter.ipFinderHasLocalAddress();
+        }
+
+        if (adapter.getStatisticsPrintFrequency() > 0 && log.isInfoEnabled()) {
+            statsPrinter = new StatisticsPrinter();
+            statsPrinter.start();
+        }
+
+        adapter.stats.onJoinStarted();
+
+        joinTopology();
+
+        adapter.stats.onJoinFinished();
+
+        hbsSnd = new HeartbeatsSender();
+        hbsSnd.start();
+
+        chkStatusSnd = new CheckStatusSender();
+        chkStatusSnd.start();
+
+        if (adapter.ipFinder.isShared()) {
+            ipFinderCleaner = new IpFinderCleaner();
+            ipFinderCleaner.start();
+        }
+
+        adapter.printStartInfo();
+    }
+
+    /**
+     * @throws IgniteSpiException If failed.
+     */
+    @SuppressWarnings("BusyWait")
+    private void registerLocalNodeAddress() throws IgniteSpiException {
+        // Make sure address registration succeeded.
+        while (true) {
+            try {
+                adapter.ipFinder.initializeLocalAddresses(locNode.socketAddresses());
+
+                // Success.
+                break;
+            }
+            catch (IllegalStateException e) {
+                throw new IgniteSpiException("Failed to register local node address with IP finder: " +
+                    locNode.socketAddresses(), e);
+            }
+            catch (IgniteSpiException e) {
+                LT.error(log, e, "Failed to register local node address in IP finder on start " +
+                    "(retrying every 2000 ms).");
+            }
+
+            try {
+                U.sleep(2000);
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                throw new IgniteSpiException("Thread has been interrupted.", e);
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        spiCtx.registerPort(tcpSrvr.port, TCP);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        spiStop0(false);
+    }
+
+    /**
+     * Stops SPI finally or stops SPI for restart.
+     *
+     * @param disconnect {@code True} if SPI is being disconnected.
+     * @throws IgniteSpiException If failed.
+     */
+    private void spiStop0(boolean disconnect) throws IgniteSpiException {
+        if (log.isDebugEnabled()) {
+            if (disconnect)
+                log.debug("Disconnecting SPI.");
+            else
+                log.debug("Preparing to start local node stop procedure.");
+        }
+
+        if (disconnect) {
+            synchronized (mux) {
+                spiState = DISCONNECTING;
+            }
+        }
+
+        if (msgWorker != null && msgWorker.isAlive() && !disconnect) {
+            // Send node left message only if it is final stop.
+            msgWorker.addMessage(new TcpDiscoveryNodeLeftMessage(locNode.id()));
+
+            synchronized (mux) {
+                long threshold = U.currentTimeMillis() + adapter.netTimeout;
+
+                long timeout = adapter.netTimeout;
+
+                while (spiState != LEFT && timeout > 0) {
+                    try {
+                        mux.wait(timeout);
+
+                        timeout = threshold - U.currentTimeMillis();
+                    }
+                    catch (InterruptedException ignored) {
+                        Thread.currentThread().interrupt();
+
+                        break;
+                    }
+                }
+
+                if (spiState == LEFT) {
+                    if (log.isDebugEnabled())
+                        log.debug("Verification for local node leave has been received from coordinator" +
+                            " (continuing stop procedure).");
+                }
+                else if (log.isInfoEnabled()) {
+                    log.info("No verification for local node leave has been received from coordinator" +
+                        " (will stop node anyway).");
+                }
+            }
+        }
+
+        U.interrupt(tcpSrvr);
+        U.join(tcpSrvr, log);
+
+        Collection<SocketReader> tmp;
+
+        synchronized (mux) {
+            tmp = U.arrayList(readers);
+        }
+
+        U.interrupt(tmp);
+        U.joinThreads(tmp, log);
+
+        U.interrupt(hbsSnd);
+        U.join(hbsSnd, log);
+
+        U.interrupt(chkStatusSnd);
+        U.join(chkStatusSnd, log);
+
+        U.interrupt(ipFinderCleaner);
+        U.join(ipFinderCleaner, log);
+
+        U.interrupt(msgWorker);
+        U.join(msgWorker, log);
+
+        U.interrupt(statsPrinter);
+        U.join(statsPrinter, log);
+
+        Collection<TcpDiscoveryNode> rmts = null;
+
+        if (!disconnect)
+            adapter.printStopInfo();
+        else {
+            adapter.getSpiContext().deregisterPorts();
+
+            rmts = ring.visibleRemoteNodes();
+        }
+
+        long topVer = ring.topologyVersion();
+
+        ring.clear();
+
+        if (rmts != null && !rmts.isEmpty()) {
+            // This is restart/disconnection and remote nodes are not empty.
+            // We need to fire FAIL event for each.
+            DiscoverySpiListener lsnr = adapter.lsnr;
+
+            if (lsnr != null) {
+                Set<ClusterNode> processed = new HashSet<>();
+
+                for (TcpDiscoveryNode n : rmts) {
+                    assert n.visible();
+
+                    processed.add(n);
+
+                    List<ClusterNode> top = U.arrayList(rmts, F.notIn(processed));
+
+                    topVer++;
+
+                    Map<Long, Collection<ClusterNode>> hist = updateTopologyHistory(topVer,
+                        Collections.unmodifiableList(top));
+
+                    lsnr.onDiscovery(EVT_NODE_FAILED, topVer, n, top, hist, null);
+                }
+            }
+        }
+
+        printStatistics();
+
+        adapter.stats.clear();
+
+        synchronized (mux) {
+            // Clear stored data.
+            leavingNodes.clear();
+            failedNodes.clear();
+
+            spiState = DISCONNECTED;
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean pingNode(UUID nodeId) {
+        assert nodeId != null;
+
+        if (log.isDebugEnabled())
+            log.debug("Pinging node: " + nodeId + "].");
+
+        if (nodeId == getLocalNodeId())
+            return true;
+
+        TcpDiscoveryNode node = ring.node(nodeId);
+
+        if (node == null || !node.visible())
+            return false;
+
+        boolean res = pingNode(node);
+
+        if (!res && !node.isClient()) {
+            LT.warn(log, null, "Failed to ping node (status check will be initiated): " + nodeId);
+
+            msgWorker.addMessage(new TcpDiscoveryStatusCheckMessage(locNode, node.id()));
+        }
+
+        return res;
+    }
+
+    /**
+     * Pings the remote node to see if it's alive.
+     *
+     * @param node Node.
+     * @return {@code True} if ping succeeds.
+     */
+    private boolean pingNode(TcpDiscoveryNode node) {
+        assert node != null;
+
+        if (node.id().equals(getLocalNodeId()))
+            return true;
+
+        UUID clientNodeId = null;
+
+        if (node.isClient()) {
+            clientNodeId = node.id();
+
+            node = ring.node(node.clientRouterNodeId());
+
+            if (node == null || !node.visible())
+                return false;
+        }
+
+        for (InetSocketAddress addr : adapter.getNodeAddresses(node, U.sameMacs(locNode, node))) {
+            try {
+                // ID returned by the node should be the same as ID of the parameter for ping to succeed.
+                IgniteBiTuple<UUID, Boolean> t = pingNode(addr, clientNodeId);
+
+                return node.id().equals(t.get1()) && (clientNodeId == null || t.get2());
+            }
+            catch (IgniteCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to ping node [node=" + node + ", err=" + e.getMessage() + ']');
+
+                onException("Failed to ping node [node=" + node + ", err=" + e.getMessage() + ']', e);
+                // continue;
+            }
+        }
+
+        return false;
+    }
+
+    /**
+     * Pings the node by its address to see if it's alive.
+     *
+     * @param addr Address of the node.
+     * @return ID of the remote node and "client exists" flag if node alive.
+     * @throws IgniteSpiException If an error occurs.
+     */
+    private IgniteBiTuple<UUID, Boolean> pingNode(InetSocketAddress addr, @Nullable UUID clientNodeId)
+        throws IgniteCheckedException {
+        assert addr != null;
+
+        UUID locNodeId = getLocalNodeId();
+
+        if (F.contains(adapter.locNodeAddrs, addr)) {
+            if (clientNodeId == null)
+                return F.t(getLocalNodeId(), false);
+
+            ClientMessageWorker clientWorker = clientMsgWorkers.get(clientNodeId);
+
+            if (clientWorker == null)
+                return F.t(getLocalNodeId(), false);
+
+            boolean clientPingRes;
+
+            try {
+                clientPingRes = clientWorker.ping();
+            }
+            catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+
+                throw new IgniteInterruptedCheckedException(e);
+            }
+
+            return F.t(getLocalNodeId(), clientPingRes);
+        }
+
+        GridFutureAdapter<IgniteBiTuple<UUID, Boolean>> fut = new GridFutureAdapter<>();
+
+        IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>> oldFut = pingMap.putIfAbsent(addr, fut);
+
+        if (oldFut != null)
+            return oldFut.get();
+        else {
+            Collection<Throwable> errs = null;
+
+            try {
+                Socket sock = null;
+
+                for (int i = 0; i < adapter.reconCnt; i++) {
+                    try {
+                        if (addr.isUnresolved())
+                            addr = new InetSocketAddress(InetAddress.getByName(addr.getHostName()), addr.getPort());
+
+                        long tstamp = U.currentTimeMillis();
+
+                        sock = adapter.openSocket(addr);
+
+                        adapter.writeToSocket(sock, new TcpDiscoveryPingRequest(locNodeId, clientNodeId));
+
+                        TcpDiscoveryPingResponse res = adapter.readMessage(sock, null, adapter.netTimeout);
+
+                        if (locNodeId.equals(res.creatorNodeId())) {
+                            if (log.isDebugEnabled())
+                                log.debug("Ping response from local node: " + res);
+
+                            break;
+                        }
+
+                        adapter.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+
+                        IgniteBiTuple<UUID, Boolean> t = F.t(res.creatorNodeId(), res.clientExists());
+
+                        fut.onDone(t);
+
+                        return t;
+                    }
+                    catch (IOException | IgniteCheckedException e) {
+                        if (errs == null)
+                            errs = new ArrayList<>();
+
+                        errs.add(e);
+                    }
+                    finally {
+                        U.closeQuiet(sock);
+                    }
+                }
+            }
+            catch (Throwable t) {
+                fut.onDone(t);
+
+                if (t instanceof Error)
+                    throw t;
+
+                throw U.cast(t);
+            }
+            finally {
+                if (!fut.isDone())
+                    fut.onDone(U.exceptionWithSuppressed("Failed to ping node by address: " + addr, errs));
+
+                boolean b = pingMap.remove(addr, fut);
+
+                assert b;
+            }
+
+            return fut.get();
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disconnect() throws IgniteSpiException {
+        spiStop0(true);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator nodeAuth) {
+        this.nodeAuth = nodeAuth;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
+        try {
+            msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, adapter.marsh.marshal(evt)));
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal custom event: " + evt, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void failNode(UUID nodeId) {
+        ClusterNode node = ring.node(nodeId);
+
+        if (node != null) {
+            TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
+                node.id(), node.order());
+
+            msgWorker.addMessage(msg);
+        }
+    }
+
+    /**
+     * Tries to join this node to topology.
+     *
+     * @throws IgniteSpiException If any error occurs.
+     */
+    private void joinTopology() throws IgniteSpiException {
+        synchronized (mux) {
+            assert spiState == CONNECTING || spiState == DISCONNECTED;
+
+            spiState = CONNECTING;
+        }
+
+        SecurityCredentials locCred = (SecurityCredentials)locNode.getAttributes()
+            .get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
+
+        // Marshal credentials for backward compatibility and security.
+        marshalCredentials(locNode);
+
+        while (true) {
+            if (!sendJoinRequestMessage()) {
+                if (log.isDebugEnabled())
+                    log.debug("Join request message has not been sent (local node is the first in the topology).");
+
+                if (nodeAuth != null) {
+                    // Authenticate local node.
+                    try {
+                        SecurityContext subj = nodeAuth.authenticateNode(locNode, locCred);
+
+                        if (subj == null)
+                            throw new IgniteSpiException("Authentication failed for local node: " + locNode.id());
+
+                        Map<String, Object> attrs = new HashMap<>(locNode.attributes());
+
+                        attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT,
+                            adapter.ignite().configuration().getMarshaller().marshal(subj));
+                        attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
+
+                        locNode.setAttributes(attrs);
+                    }
+                    catch (IgniteException | IgniteCheckedException e) {
+                        throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e);
+                    }
+                }
+
+                locNode.order(1);
+                locNode.internalOrder(1);
+
+                adapter.gridStartTime = U.currentTimeMillis();
+
+                locNode.visible(true);
+
+                ring.clear();
+
+                ring.topologyVersion(1);
+
+                synchronized (mux) {
+                    topHist.clear();
+
+                    spiState = CONNECTED;
+
+                    mux.notifyAll();
+                }
+
+                notifyDiscovery(EVT_NODE_JOINED, 1, locNode);
+
+                break;
+            }
+
+            if (log.isDebugEnabled())
+                log.debug("Join request message has been sent (waiting for coordinator response).");
+
+            synchronized (mux) {
+                long threshold = U.currentTimeMillis() + adapter.netTimeout;
+
+                long timeout = adapter.netTimeout;
+
+                while (spiState == CONNECTING && timeout > 0) {
+                    try {
+                        mux.wait(timeout);
+
+                        timeout = threshold - U.currentTimeMillis();
+                    }
+                    catch (InterruptedException ignored) {
+                        Thread.currentThread().interrupt();
+
+                        throw new IgniteSpiException("Thread has been interrupted.");
+                    }
+                }
+
+                if (spiState == CONNECTED)
+                    break;
+                else if (spiState == DUPLICATE_ID)
+                    throw adapter.duplicateIdError((TcpDiscoveryDuplicateIdMessage)joinRes.get());
+                else if (spiState == AUTH_FAILED)
+                    throw adapter.authenticationFailedError((TcpDiscoveryAuthFailedMessage)joinRes.get());
+                else if (spiState == CHECK_FAILED)
+                    throw adapter.checkFailedError((TcpDiscoveryCheckFailedMessage)joinRes.get());
+                else if (spiState == LOOPBACK_PROBLEM) {
+                    TcpDiscoveryLoopbackProblemMessage msg = (TcpDiscoveryLoopbackProblemMessage)joinRes.get();
+
+                    boolean locHostLoopback = adapter.locHost.isLoopbackAddress();
+
+                    String firstNode = locHostLoopback ? "local" : "remote";
+
+                    String secondNode = locHostLoopback ? "remote" : "local";
+
+                    throw new IgniteSpiException("Failed to add node to topology because " + firstNode +
+                        " node is configured to use loopback address, but " + secondNode + " node is not " +
+                        "(consider changing 'localAddress' configuration parameter) " +
+                        "[locNodeAddrs=" + U.addressesAsString(locNode) + ", rmtNodeAddrs=" +
+                        U.addressesAsString(msg.addresses(), msg.hostNames()) + ']');
+                }
+                else
+                    LT.warn(log, null, "Node has not been connected to topology and will repeat join process. " +
+                        "Check remote nodes logs for possible error messages. " +
+                        "Note that large topology may require significant time to start. " +
+                        "Increase 'TcpDiscoverySpi.networkTimeout' configuration property " +
+                        "if getting this message on the starting nodes [networkTimeout=" + adapter.netTimeout + ']');
+            }
+        }
+
+        assert locNode.order() != 0;
+        assert locNode.internalOrder() != 0;
+
+        if (log.isDebugEnabled())
+            log.debug("Discovery SPI has been connected to topology with order: " + locNode.internalOrder());
+    }
+
+    /**
+     * Tries to send join request message to a random node presenting in topology.
+     * Address is provided by {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder} and message is
+     * sent to first node connection succeeded to.
+     *
+     * @return {@code true} if send succeeded.
+     * @throws IgniteSpiException If any error occurs.
+     */
+    @SuppressWarnings({"BusyWait"})
+    private boolean sendJoinRequestMessage() throws IgniteSpiException {
+        TcpDiscoveryAbstractMessage joinReq = new TcpDiscoveryJoinRequestMessage(locNode,
+            adapter.collectExchangeData(getLocalNodeId()));
+
+        // Time when it has been detected, that addresses from IP finder do not respond.
+        long noResStart = 0;
+
+        while (true) {
+            Collection<InetSocketAddress> addrs = adapter.resolvedAddresses();
+
+            if (F.isEmpty(addrs))
+                return false;
+
+            boolean retry = false;
+            Collection<Exception> errs = new ArrayList<>();
+
+            try (SocketMultiConnector multiConnector = new SocketMultiConnector(adapter, addrs, 2)) {
+                GridTuple3<InetSocketAddress, Socket, Exception> tuple;
+
+                while ((tuple = multiConnector.next()) != null) {
+                    InetSocketAddress addr = tuple.get1();
+                    Socket sock = tuple.get2();
+                    Exception ex = tuple.get3();
+
+                    if (ex == null) {
+                        assert sock != null;
+
+                        try {
+                            Integer res = sendMessageDirectly(joinReq, addr, sock);
+
+                            assert res != null;
+
+                            noResAddrs.remove(addr);
+
+                            // Address is responsive, reset period start.
+                            noResStart = 0;
+
+                            switch (res) {
+                                case RES_WAIT:
+                                    // Concurrent startup, try sending join request again or wait if no success.
+                                    retry = true;
+
+                                    break;
+                                case RES_OK:
+                                    if (log.isDebugEnabled())
+                                        log.debug("Join request message has been sent to address [addr=" + addr +
+                                            ", req=" + joinReq + ']');
+
+                                    // Join request sending succeeded, wait for response from topology.
+                                    return true;
+
+                                default:
+                                    // Concurrent startup, try next node.
+                                    if (res == RES_CONTINUE_JOIN) {
+                                        if (!fromAddrs.contains(addr))
+                                            retry = true;
+                                    }
+                                    else {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Unexpected response to join request: " + res);
+
+                                        retry = true;
+                                    }
+
+                                    break;
+                            }
+                        }
+                        catch (IgniteSpiException e) {
+                            e.printStackTrace();
+
+                            ex = e;
+                        }
+                    }
+
+                    if (ex != null) {
+                        errs.add(ex);
+
+                        if (log.isDebugEnabled()) {
+                            IOException ioe = X.cause(ex, IOException.class);
+
+                            log.debug("Failed to send join request message [addr=" + addr +
+                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']');
+
+                            onException("Failed to send join request message [addr=" + addr +
+                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']', ioe);
+                        }
+
+                        noResAddrs.add(addr);
+                    }
+                }
+            }
+
+            if (retry) {
+                if (log.isDebugEnabled())
+                    log.debug("Concurrent discovery SPI start has been detected (local node should wait).");
+
+                try {
+                    U.sleep(2000);
+                }
+                catch (IgniteInterruptedCheckedException e) {
+                    throw new IgniteSpiException("Thread has been interrupted.", e);
+                }
+            }
+            else if (!adapter.ipFinder.isShared() && !ipFinderHasLocAddr) {
+                IgniteCheckedException e = null;
+
+                if (!errs.isEmpty()) {
+                    e = new IgniteCheckedException("Multiple connection attempts failed.");
+
+                    for (Exception err : errs)
+                        e.addSuppressed(err);
+                }
+
+                if (e != null && X.hasCause(e, ConnectException.class))
+                    LT.warn(log, null, "Failed to connect to any address from IP finder " +
+                        "(make sure IP finder addresses are correct and firewalls are disabled on all host machines): " +
+                        addrs);
+
+                if (adapter.joinTimeout > 0) {
+                    if (noResStart == 0)
+                        noResStart = U.currentTimeMillis();
+                    else if (U.currentTimeMillis() - noResStart > adapter.joinTimeout)
+                        throw new IgniteSpiException(
+                            "Failed to connect to any address from IP finder within join timeout " +
+                                "(make sure IP finder addresses are correct, and operating system firewalls are disabled " +
+                                "on all host machines, or consider increasing 'joinTimeout' configuration property): " +
+                                addrs, e);
+                }
+
+                try {
+                    U.sleep(2000);
+                }
+                catch (IgniteInterruptedCheckedException ex) {
+                    throw new IgniteSpiException("Thread has been interrupted.", ex);
+                }
+            }
+            else
+                break;
+        }
+
+        return false;
+    }
+
+    /**
+     * Establishes connection to an address, sends message and returns the response (if any).
+     *
+     * @param msg Message to send.
+     * @param addr Address to send message to.
+     * @return Response read from the recipient or {@code null} if no response is supposed.
+     * @throws IgniteSpiException If an error occurs.
+     */
+    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr, Socket sock)
+        throws IgniteSpiException {
+        assert msg != null;
+        assert addr != null;
+
+        Collection<Throwable> errs = null;
+
+        long ackTimeout0 = adapter.ackTimeout;
+
+        int connectAttempts = 1;
+
+        boolean joinReqSent = false;
+
+        UUID locNodeId = getLocalNodeId();
+
+        for (int i = 0; i < adapter.reconCnt; i++) {
+            // Need to set to false on each new iteration,
+            // since remote node may leave in the middle of the first iteration.
+            joinReqSent = false;
+
+            boolean openSock = false;
+
+            try {
+                long tstamp = U.currentTimeMillis();
+
+                if (sock == null)
+                    sock = adapter.openSocket(addr);
+
+                openSock = true;
+
+                // Handshake.
+                adapter.writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId));
+
+                TcpDiscoveryHandshakeResponse res = adapter.readMessage(sock, null, ackTimeout0);
+
+                if (locNodeId.equals(res.creatorNodeId())) {
+                    if (log.isDebugEnabled())
+                        log.debug("Handshake response from local node: " + res);
+
+                    break;
+                }
+
+                adapter.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+
+                // Send message.
+                tstamp = U.currentTimeMillis();
+
+                adapter.writeToSocket(sock, msg);
+
+                adapter.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
+
+                if (debugMode)
+                    debugLog("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
+                        ", rmtNodeId=" + res.creatorNodeId() + ']');
+
+                if (log.isDebugEnabled())
+                    log.debug("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
+                        ", rmtNodeId=" + res.creatorNodeId() + ']');
+
+                // Connection has been established, but
+                // join request may not be unmarshalled on remote host.
+                // E.g. due to class not found issue.
+                joinReqSent = msg instanceof TcpDiscoveryJoinRequestMessage;
+
+                return adapter.readReceipt(sock, ackTimeout0);
+            }
+            catch (ClassCastException e) {
+                // This issue is rarely reproducible on AmazonEC2, but never
+                // on dedicated machines.
+                if (log.isDebugEnabled())
+                    U.error(log, "Class cast exception on direct send: " + addr, e);
+
+                onException("Class cast exception on direct send: " + addr, e);
+
+                if (errs == null)
+                    errs = new ArrayList<>();
+
+                errs.add(e);
+            }
+            catch (IOException | IgniteCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.error("Exception on direct send: " + e.getMessage(), e);
+
+                onException("Exception on direct send: " + e.getMessage(), e);
+
+                if (errs == null)
+                    errs = new ArrayList<>();
+
+                errs.add(e);
+
+                if (!openSock) {
+                    // Reconnect for the second time, if connection is not established.
+                    if (connectAttempts < 2) {
+                        connectAttempts++;
+
+                        continue;
+                    }
+
+                    break; // Don't retry if we can not establish connection.
+                }
+
+                if (e instanceof SocketTimeoutException || X.hasCause(e, SocketTimeoutException.class)) {
+                    ackTimeout0 *= 2;
+
+                    if (!checkAckTimeout(ackTimeout0))
+                        break;
+                }
+            }
+            finally {
+                U.closeQuiet(sock);
+
+                sock = null;
+            }
+        }
+
+        if (joinReqSent) {
+            if (log.isDebugEnabled())
+                log.debug("Join request has been sent, but receipt has not been read (returning RES_WAIT).");
+
+            // Topology will not include this node,
+            // however, warning on timed out join will be output.
+            return RES_OK;
+        }
+
+        throw new IgniteSpiException(
+            "Failed to send message to address [addr=" + addr + ", msg=" + msg + ']',
+            U.exceptionWithSuppressed("Failed to send message to address " +
+                "[addr=" + addr + ", msg=" + msg + ']', errs));
+    }
+
+    /**
+     * Marshalls credentials with discovery SPI marshaller (will replace attribute value).
+     *
+     * @param node Node to marshall credentials for.
+     * @throws IgniteSpiException If marshalling failed.
+     */
+    private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
+        try {
+            // Use security-unsafe getter.
+            Map<String, Object> attrs = new HashMap<>(node.getAttributes());
+
+            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS,
+                adapter.marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS)));
+
+            node.setAttributes(attrs);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal node security credentials: " + node.id(), e);
+        }
+    }
+
+    /**
+     * Unmarshalls credentials with discovery SPI marshaller (will not replace attribute value).
+     *
+     * @param node Node to unmarshall credentials for.
+     * @return Security credentials.
+     * @throws IgniteSpiException If unmarshal fails.
+     */
+    private SecurityCredentials unmarshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
+        try {
+            byte[] credBytes = (byte[])node.getAttributes().get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
+
+            if (credBytes == null)
+                return null;
+
+            return adapter.marsh.unmarshal(credBytes, null);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to unmarshal node security credentials: " + node.id(), e);
+        }
+    }
+
+    /**
+     * @param ackTimeout Acknowledgement timeout.
+     * @return {@code True} if acknowledgement timeout is less or equal to
+     * maximum acknowledgement timeout, {@code false} otherwise.
+     */
+    private boolean checkAckTimeout(long ackTimeout) {
+        if (ackTimeout > adapter.maxAckTimeout) {
+            LT.warn(log, null, "Acknowledgement timeout is greater than maximum acknowledgement timeout " +
+                "(consider increasing 'maxAckTimeout' configuration property) " +
+                "[ackTimeout=" + ackTimeout + ", maxAckTimeout=" + adapter.maxAckTimeout + ']');
+
+            return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Notify external listener on discovery event.
+     *
+     * @param type Discovery event type. See {@link org.apache.ignite.events.DiscoveryEvent} for more details.
+     * @param topVer Topology version.
+     * @param node Remote node this event is connected with.
+     */
+    private void notifyDiscovery(int type, long topVer, TcpDiscoveryNode node) {
+        assert type > 0;
+        assert node != null;
+
+        DiscoverySpiListener lsnr = adapter.lsnr;
+
+        TcpDiscoverySpiState spiState = spiStateCopy();
+
+        if (lsnr != null && node.visible() && (spiState == CONNECTED || spiState == DISCONNECTING)) {
+            if (log.isDebugEnabled())
+                log.debug("Discovery notification [node=" + node + ", spiState=" + spiState +
+                    ", type=" + U.gridEventName(type) + ", topVer=" + topVer + ']');
+
+            Collection<ClusterNode> top = F.upcast(ring.visibleNodes());
+
+            Map<Long, Collection<ClusterNode>> hist = updateTopologyHistory(topVer, top);
+
+            lsnr.onDiscovery(type, topVer, node, top, hist, null);
+        }
+        else if (log.isDebugEnabled())
+            log.debug("Skipped discovery notification [node=" + node + ", spiState=" + spiState +
+                ", type=" + U.gridEventName(type) + ", topVer=" + topVer + ']');
+    }
+
+    /**
+     * Update topology history with new topology snapshots.
+     *
+     * @param topVer Topology version.
+     * @param top Topology snapshot.
+     * @return Copy of updated topology history.
+     */
+    @Nullable private Map<Long, Collection<ClusterNode>> updateTopologyHistory(long topVer, Collection<ClusterNode> top) {
+        synchronized (mux) {
+            if (topHist.containsKey(topVer))
+                return null;
+
+            topHist.put(topVer, top);
+
+            while (topHist.size() > adapter.topHistSize)
+                topHist.remove(topHist.firstKey());
+
+            if (log.isDebugEnabled())
+                log.debug("Added topology snapshot to history, topVer=" + topVer + ", historySize=" + topHist.size());
+
+            return new TreeMap<>(topHist);
+        }
+    }
+
+    /**
+     * Checks whether local node is coordinator. Nodes that are leaving or failed
+     * (but are still in topology) are removed from search.
+     *
+     * @return {@code true} if local node is coordinator.
+     */
+    private boolean isLocalNodeCoordinator() {
+        synchronized (mux) {
+            boolean crd = spiState == CONNECTED && locNode.equals(resolveCoordinator());
+
+            if (crd)
+                adapter.stats.onBecomingCoordinator();
+
+            return crd;
+        }
+    }
+
+    /**
+     * @return Spi state copy.
+     */
+    private TcpDiscoverySpiState spiStateCopy() {
+        TcpDiscoverySpiState state;
+
+        synchronized (mux) {
+            state = spiState;
+        }
+
+        return state;
+    }
+
+    /**
+     * Resolves coordinator. Nodes that are leaving or failed (but are still in
+     * topology) are removed from search.
+     *
+     * @return Coordinator node or {@code null} if there are no coordinator
+     * (i.e. local node is the last one and is currently stopping).
+     */
+    @Nullable private TcpDiscoveryNode resolveCoordinator() {
+        return resolveCoordinator(null);
+    }
+
+    /**
+     * Resolves coordinator. Nodes that are leaving or failed (but are still in
+     * topology) are removed from search as well as provided filter.
+     *
+     * @param filter Nodes to exclude when resolving coordinator (optional).
+     * @return Coordinator node or {@code null} if there are no coordinator
+     * (i.e. local node is the last one and is currently stopping).
+     */
+    @Nullable private TcpDiscoveryNode resolveCoordinator(
+        @Nullable Collection<TcpDiscoveryNode> filter) {
+        synchronized (mux) {
+            Collection<TcpDiscoveryNode> excluded = F.concat(false, failedNodes, leavingNodes);
+
+            if (!F.isEmpty(filter))
+                excluded = F.concat(false, excluded, filter);
+
+            return ring.coordinator(excluded);
+        }
+    }
+
+    /**
+     * Prints SPI statistics.
+     */
+    private void printStatistics() {
+        if (log.isInfoEnabled() && adapter.statsPrintFreq > 0) {
+            int failedNodesSize;
+            int leavingNodesSize;
+
+            synchronized (mux) {
+                failedNodesSize = failedNodes.size();
+                leavingNodesSize = leavingNodes.size();
+            }
+
+            Runtime runtime = Runtime.getRuntime();
+
+            TcpDiscoveryNode coord = resolveCoordinator();
+
+            log.info("Discovery SPI statistics [statistics=" + adapter.stats + ", spiState=" + spiStateCopy() +
+                ", coord=" + coord +
+                ", topSize=" + ring.allNodes().size() +
+                ", leavingNodesSize=" + leavingNodesSize + ", failedNodesSize=" + failedNodesSize +
+                ", msgWorker.queue.size=" + (msgWorker != null ? msgWorker.queueSize() : "N/A") +
+                ", lastUpdate=" + (locNode != null ? U.format(locNode.lastUpdateTime()) : "N/A") +
+                ", heapFree=" + runtime.freeMemory() / (1024 * 1024) +
+                "M, heapTotal=" + runtime.maxMemory() / (1024 * 1024) + "M]");
+        }
+    }
+
+    /**
+     * @param msg Message to prepare.
+     * @param destNodeId Destination node ID.
+     * @param msgs Messages to include.
+     * @param discardMsgId Discarded message ID.
+     */
+    private void prepareNodeAddedMessage(TcpDiscoveryAbstractMessage msg, UUID destNodeId,
+        @Nullable Collection<TcpDiscoveryAbstractMessage> msgs, @Nullable IgniteUuid discardMsgId) {
+        assert destNodeId != null;
+
+        if (msg instanceof TcpDiscoveryNodeAddedMessage) {
+            TcpDiscoveryNodeAddedMessage nodeAddedMsg = (TcpDiscoveryNodeAddedMessage)msg;
+
+            TcpDiscoveryNode node = nodeAddedMsg.node();
+
+            if (node.id().equals(destNodeId)) {
+                Collection<TcpDiscoveryNode> allNodes = ring.allNodes();
+                Collection<TcpDiscoveryNode> topToSend = new ArrayList<>(allNodes.size());
+
+                for (TcpDiscoveryNode n0 : allNodes) {
+                    assert n0.internalOrder() != 0 : n0;
+
+                    // Skip next node and nodes added after next
+                    // in case this message is resent due to failures/leaves.
+                    // There will be separate messages for nodes with greater
+                    // internal order.
+                    if (n0.internalOrder() < nodeAddedMsg.node().internalOrder())
+                        topToSend.add(n0);
+                }
+
+                nodeAddedMsg.topology(topToSend);
+                nodeAddedMsg.messages(msgs, discardMsgId);
+
+                Map<Long, Collection<ClusterNode>> hist;
+
+                synchronized (mux) {
+                    hist = new TreeMap<>(topHist);
+                }
+
+                nodeAddedMsg.topologyHistory(hist);
+            }
+        }
+    }
+
+    /**
+     * @param msg Message to clear.
+     */
+    private void clearNodeAddedMessage(TcpDiscoveryAbstractMessage msg) {
+        if (msg instanceof TcpDiscoveryNodeAddedMessage) {
+            // Nullify topology before registration.
+            TcpDiscoveryNodeAddedMessage nodeAddedMsg = (TcpDiscoveryNodeAddedMessage)msg;
+
+            nodeAddedMsg.topology(null);
+            nodeAddedMsg.topologyHistory(null);
+            nodeAddedMsg.messages(null, null);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override void simulateNodeFailure() {
+        U.warn(log, "Simulating node failure: " + getLocalNodeId());
+
+        U.interrupt(tcpSrvr);
+        U.join(tcpSrvr, log);
+
+        U.interrupt(hbsSnd);
+        U.join(hbsSnd, log);
+
+        U.interrupt(chkStatusSnd);
+        U.join(chkStatusSnd, log);
+
+        U.interrupt(ipFinderCleaner);
+        U.join(ipFinderCleaner, log);
+
+        Collection<SocketReader> tmp;
+
+        synchronized (mux) {
+            tmp = U.arrayList(readers);
+        }
+
+        U.interrupt(tmp);
+        U.joinThreads(tmp, log);
+
+        U.interrupt(msgWorker);
+        U.join(msgWorker, log);
+
+        U.interrupt(statsPrinter);
+        U.join(statsPrinter, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void brakeConnection() {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteSpiThread workerThread() {
+        return msgWorker;
+    }
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     * <p>
+     * Simulates situation when next node is still alive but is bypassed
+     * since it has been excluded from the ring, possibly, due to short time
+     * network problems.
+     * <p>
+     * This method is intended for test purposes only.
+     */
+    void forceNextNodeFailure() {
+        U.warn(log, "Next node will be forcibly failed (if any).");
+
+        TcpDiscoveryNode next;
+
+        synchronized (mux) {
+            next = ring.nextNode(failedNodes);
+        }
+
+        if (next != null)
+            msgWorker.addMessage(new TcpDiscoveryNodeFailedMessage(getLocalNodeId(), next.id(),
+                next.internalOrder()));
+    }
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     * <p>
+     * This method is intended for test purposes only.
+     *
+     * @return Nodes ring.
+     */
+    TcpDiscoveryNodesRing ring() {
+        return ring;
+    }
+
+    /** {@inheritDoc} */
+    public void dumpDebugInfo(IgniteLogger log) {
+        if (!debugMode) {
+            U.quietAndWarn(log, "Failed to dump debug info (discovery SPI was not configured " +
+                "in debug mode, consider setting 'debugMode' configuration property to 'true').");
+
+            return;
+        }
+
+        assert log.isInfoEnabled();
+
+        synchronized (mux) {
+            StringBuilder b = new StringBuilder(U.nl());
+
+            b.append(">>>").append(U.nl());
+            b.append(">>>").append("Dumping discovery SPI debug info.").append(U.nl());
+            b.append(">>>").append(U.nl());
+
+            b.append("Local node ID: ").append(getLocalNodeId()).append(U.nl()).append(U.nl());
+            b.append("Local node: ").append(locNode).append(U.nl()).append(U.nl());
+            b.append("SPI state: ").append(spiState).append(U.nl()).append(U.nl());
+
+            b.append("Internal threads: ").append(U.nl());
+
+            b.append("    Message worker: ").append(threadStatus(msgWorker)).append(U.nl());
+            b.append("    Check status sender: ").append(threadStatus(chkStatusSnd)).append(U.nl());
+            b.append("    HB sender: ").append(threadStatus(hbsSnd)).append(U.nl());
+            b.append("    Socket timeout worker: ").append(threadStatus(adapter.sockTimeoutWorker)).append(U.nl());
+            b.append("    IP finder cleaner: ").append(threadStatus(ipFinderCleaner)).append(U.nl());
+            b.append("    Stats printer: ").append(threadStatus(statsPrinter)).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("Socket readers: ").append(U.nl());
+
+            for (SocketReader rdr : readers)
+                b.append("    ").append(rdr).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("In-memory log messages: ").append(U.nl());
+
+            for (String msg : debugLog)
+                b.append("    ").append(msg).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("Leaving nodes: ").append(U.nl());
+
+            for (TcpDiscoveryNode node : leavingNodes)
+                b.append("    ").append(node.id()).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("Failed nodes: ").append(U.nl());
+
+            for (TcpDiscoveryNode node : failedNodes)
+                b.append("    ").append(node.id()).append(U.nl());
+
+            b.append(U.nl());
+
+            b.append("Stats: ").append(adapter.stats).append(U.nl());
+
+            U.quietAndInfo(log, b.toString());
+        }
+    }
+
+    /**
+     * @param msg Message.
+     */
+    private void debugLog(String msg) {
+        assert debugMode;
+
+        String msg0 = new SimpleDateFormat("[HH:mm:ss,SSS]").format(new Date(System.currentTimeMillis())) +
+            '[' + Thread.currentThread().getName() + "][" + getLocalNodeId() +
+            "-" + locNode.internalOrder() + "] " +
+            msg;
+
+        debugLog.add(msg0);
+
+        int delta = debugLog.size() - debugMsgHist;
+
+        for (int i = 0; i < delta && debugLog.size() > debugMsgHist; i++)
+            debugLog.poll();
+    }
+
+    /**
+     * @param msg Message.
+     * @return {@code True} if recordable in debug mode.
+     */
+    private boolean recordable(TcpDiscoveryAbstractMessage msg) {
+        return !(msg instanceof TcpDiscoveryHeartbeatMessage) &&
+            !(msg instanceof TcpDiscoveryStatusCheckMessage) &&
+            !(msg instanceof TcpDiscoveryDiscardMessage);
+    }
+
+    /**
+     * Checks if two given {@link SecurityPermissionSet} objects contain the same permissions.
+     * Each permission belongs to one of three groups : cache, task or system.
+     *
+     * @param locPerms The first set of permissions.
+     * @param rmtPerms The second set of permissions.
+     * @return {@code True} if given parameters contain the same permissions, {@code False} otherwise.
+     */
+    private boolean permissionsEqual(SecurityPermissionSet locPerms, SecurityPermissionSet rmtPerms) {
+        boolean dfltAllowMatch = !(locPerms.defaultAllowAll() ^ rmtPerms.defaultAllowAll());
+
+        boolean bothHaveSamePerms = F.eqNotOrdered(rmtPerms.systemPermissions(), locPerms.systemPermissions()) &&
+            F.eqNotOrdered(rmtPerms.cachePermissions(), locPerms.cachePermissions()) &&
+            F.eqNotOrdered(rmtPerms.taskPermissions(), locPerms.taskPermissions());
+
+        return dfltAllowMatch && bothHaveSamePerms;
+    }
+
+    /**
+     * @param msg Message.
+     * @param nodeId Node ID.
+     */
+    private static void removeMetrics(TcpDiscoveryHeartbeatMessage msg, UUID nodeId) {
+        msg.removeMetrics(nodeId);
+        msg.removeCacheMetrics(nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(ServerImpl.class, this);
+    }
+
+    /**
+     * Thread that sends heartbeats.
+     */
+    private class HeartbeatsSender extends IgniteSpiThread {
+        /**
+         * Constructor.
+         */
+        private HeartbeatsSender() {
+            super(adapter.ignite().name(), "tcp-disco-hb-sender", log);
+
+            setPriority(adapter.threadPri);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("BusyWait")
+        @Override protected void body() throws InterruptedException {
+            while (!isLocalNodeCoordinator())
+                Thread.sleep(1000);
+
+            if (log.isDebugEnabled())
+                log.debug("Heartbeats sender has been started.");
+
+            while (!isInterrupted()) {
+                if (spiStateCopy() != CONNECTED) {
+                    if (log.isDebugEnabled())
+                        log.debug("Stopping heartbeats sender (SPI is not connected to topology).");
+
+                    return;
+                }
+
+                TcpDiscoveryHeartbeatMessage msg = new TcpDiscoveryHeartbeatMessage(getLocalNodeId());
+
+                msg.verify(getLocalNodeId());
+
+                msgWorker.addMessage(msg);
+
+                Thread.sleep(adapter.hbFreq);
+            }
+        }
+    }
+
+    /**
+     * Thread that sends status check messages to next node if local node has not
+     * been receiving heartbeats ({@link TcpDiscoveryHeartbeatMessage})
+     * for {@link TcpDiscoverySpi#getMaxMissedHeartbeats()} *
+     * {@link TcpDiscoverySpi#getHeartbeatFrequency()}.
+     */
+    private class CheckStatusSender extends IgniteSpiThread {
+        /**
+         * Constructor.
+         */
+        private CheckStatusSender() {
+            super(adapter.ignite().name(), "tcp-disco-status-check-sender", log);
+
+            setPriority(adapter.threadPri);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("BusyWait")
+        @Override protected void body() throws InterruptedException {
+            if (log.isDebugEnabled())
+                log.debug("Status check sender has been started.");
+
+            // Only 1 heartbeat missing is acceptable. Add 50 ms to avoid false alarm.
+            long checkTimeout = (long)adapter.maxMissedHbs * adapter.hbFreq + 50;
+
+            long lastSent = 0;
+
+            while (!isInterrupted()) {
+                // 1. Determine timeout.
+                if (lastSent < locNode.lastUpdateTime())
+                    lastSent = locNode.lastUpdateTime();
+
+                long timeout = (lastSent + checkTimeout) - U.currentTimeMillis();
+
+                if (timeout > 0)
+                    Thread.sleep(timeout);
+
+                // 2. Check if SPI is still connected.
+                if (spiStateCopy() != CONNECTED) {
+                    if (log.isDebugEnabled())
+                        log.debug("Stopping status check sender (SPI is not connected to topology).");
+
+                    return;
+                }
+
+                // 3. Was there an update?
+                if (locNode.lastUpdateTime() > lastSent || !ring.hasRemoteNodes()) {
+                    if (log.isDebugEnabled())
+                        log.debug("Skipping status check send " +
+                            "[locNodeLastUpdate=" + U.format(locNode.lastUpdateTime()) +
+                            ", hasRmts=" + ring.hasRemoteNodes() + ']');
+
+                    continue;
+                }
+
+                // 4. Send status check message.
+                lastSent = U.currentTimeMillis();
+
+                msgWorker.addMessage(new TcpDiscoveryStatusCheckMessage(locNode, null));
+            }
+        }
+    }
+
+    /**
+     * Thread that cleans IP finder and keeps it in the correct state, unregistering
+     * addresses of the nodes that has left the topology.
+     * <p>
+     * This thread should run only on coordinator node and will clean IP finder
+     * if and only if {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder#isShared()} is {@code true}.
+     */
+    private class IpFinderCleaner extends IgniteSpiThread {
+        /**
+         * Constructor.
+         */
+        private IpFinderCleaner() {
+            super(adapter.ignite().name(), "tcp-disco-ip-finder-cleaner", log);
+
+            setPriority(adapter.threadPri);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("BusyWait")
+        @Override protected void body() throws InterruptedException {
+            if (log.isDebugEnabled())
+                log.debug("IP finder cleaner has been started.");
+
+            while (!isInterrupted()) {
+                Thread.sleep(adapter.ipFinderCleanFreq);
+
+                if (!isLocalNodeCoordinator())
+                    continue;
+
+                if (spiStateCopy() != CONNECTED) {
+                    if (log.isDebugEnabled())
+                        log.debug("Stopping IP finder cleaner (SPI is not connected to topology).");
+
+                    return;
+                }
+
+                if (adapter.ipFinder.isShared())
+                    cleanIpFinder();
+            }
+        }
+
+        /**
+         * Cleans IP finder.
+         */
+        private void cleanIpFinder() {
+            assert adapter.ipFinder.isShared();
+
+            try {
+                // Addresses that belongs to nodes in topology.
+                Collection<InetSocketAddress> currAddrs = F.flatCollections(
+                    F.viewReadOnly(
+                        ring.allNodes(),
+                        new C1<TcpDiscoveryNode, Collection<InetSocketAddress>>() {
+                            @Override public Collection<InetSocketAddress> apply(TcpDiscoveryNode node) {
+                                return !node.isClient() ? adapter.getNodeAddresses(node) :
+                                    Collections.<InetSocketAddress>emptyList();
+                            }
+                        }
+                    )
+                );
+
+                // Addresses registered in IP finder.
+                Collection<InetSocketAddress> regAddrs = adapter.registeredAddresses();
+
+                // Remove all addresses that belong to alive nodes, leave dead-node addresses.
+                Collection<InetSocketAddress> rmvAddrs = F.view(
+                    regAddrs,
+                    F.notContains(currAddrs),
+                    new P1<InetSocketAddress>() {
+                        private final Map<InetSocketAddress, Boolean> pingResMap =
+                            new HashMap<>();
+
+                        @Override public boolean apply(InetSocketAddress addr) {
+                            Boolean res = pingResMap.get(addr);
+
+                            if (res == null) {
+                                try {
+                                    res = pingNode(addr, null).get1() != null;
+                                }
+                                catch (IgniteCheckedException e) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Failed to ping node [addr=" + addr +
+                                            ", err=" + e.getMessage() + ']');
+
+                                    res = false;
+                                }
+                                finally {
+                                    pingResMap.put(addr, res);
+                                }
+                            }
+
+                            return !res;
+                        }
+                    }
+                );
+
+                // Unregister dead-nodes addresses.
+                if (!rmvAddrs.isEmpty()) {
+                    adapter.ipFinder.unregisterAddresses(rmvAddrs);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Unregistered addresses from IP finder: " + rmvAddrs);
+                }
+
+                // Addresses that were removed by mistake (e.g. on segmentation).
+                Collection<InetSocketAddress> missingAddrs = F.view(
+                    currAddrs,
+                    F.notContains(regAddrs)
+                );
+
+                // Re-register missing addresses.
+                if (!missingAddrs.isEmpty()) {
+                    adapter.ipFinder.registerAddresses(missingAddrs);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Registered missing addresses in IP finder: " + missingAddrs);
+                }
+            }
+            catch (IgniteSpiException e) {
+                LT.error(log, e, "Failed to clean IP finder up.");
+            }
+        }
+    }
+
+    /**
+     * Pending messages container.
+     */
+    private static class PendingMessages {
+        /** */
+        private static final int MAX = 1024;
+
+        /** Pending messages. */
+        private final Queue<TcpDiscoveryAbstractMessage> msgs = new ArrayDeque<>(MAX * 2);
+
+        /** Discarded message ID. */
+        private IgniteUuid discardId;
+
+        /**
+         * Adds pending message and shrinks queue if it exceeds limit
+         * (messages that were not discarded yet are never removed).
+         *
+         * @param msg Message to add.
+         */
+        void add(TcpDiscoveryAbstractMessage msg) {
+            msgs.add(msg);
+
+            while (msgs.size() > MAX) {
+                TcpDiscoveryAbstractMessage polled = msgs.poll();
+
+                assert polled != null;
+
+                if (polled.id().equals(discardId))
+                    break;
+            }
+        }
+
+        /**
+         * Gets messages starting from provided ID (exclusive). If such
+         * message is not found, {@code null} is returned (this indicates
+         * a failure condition when it was already removed from queue).
+         *
+         * @param lastMsgId Last message ID.
+         * @return Collection of messages.
+         */
+        @Nullable Collection<TcpDiscoveryAbstractMessage> messages(IgniteUuid lastMsgId) {
+            assert lastMsgId != null;
+
+            Collection<TcpDiscoveryAbstractMessage> copy = new ArrayList<>(msgs.size());
+
+            boolean skip = true;
+
+            for (TcpDiscoveryAbstractMessage msg : msgs) {
+                if (skip) {
+                    if (msg.id().equals(lastMsgId))
+                        skip = false;
+                }
+                else
+                    copy.add(msg);
+            }
+
+            return !skip ? copy : null;
+        }
+
+        /**
+         * Resets pending messages.
+         *
+         * @param msgs Message.
+         * @param discardId Discarded message ID.
+         */
+        void reset(@Nullable Collection<TcpDiscoveryAbstractMessage> msgs, @Nullable IgniteUuid discardId) {
+            this.msgs.clear();
+
+            if (msgs != null)
+                this.msgs.addAll(msgs);
+
+            this.discardId = discardId;
+        }
+
+        /**
+         * Clears pending messages.
+         */
+        void clear() {
+            msgs.clear();
+
+            discardId = null;
+        }
+
+        /**
+         * Discards message with provided ID and all before it.
+         *
+         * @param id Discarded message ID.
+         */
+        void discard(IgniteUuid id) {
+            discardId = id;
+        }
+    }
+
+    /**
+     * Message worker thread for messages processing.
+     */
+    private class RingMessageWorker extends MessageWorkerAdapter {
+        /** Next node. */
+        @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
+        private TcpDiscoveryNode next;
+
+        /** Pending messages. */
+        private final PendingMessages pendingMsgs = new PendingMessages();
+
+        /** Last message that updated topology. */
+        private TcpDiscoveryAbstractMessage lastMsg;
+
+        /** Force pending messages send. */
+        private boolean forceSndPending;
+
+        /** Socket. */
+        private Socket sock;
+
+        /**
+         */
+        protected RingMessageWorker() {
+            super("tcp-disco-msg-worker");
+        }
+
+        /**
+         * @param msg Message to process.
+         */
+        @Override protected void processMessage(TcpDiscoveryAbstractMessage msg) {
+            if (log.isDebugEnabled())
+                log.debug("Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
+
+            if (debugMode)
+                debugLog("Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
+
+            adapter.stats.onMessageProcessingStarted(msg);
+
+            if (msg instanceof TcpDiscoveryJoinRequestMessage)
+                processJoinRequestMessage((TcpDiscoveryJoinRequestMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryClientReconnectMessage)
+                processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryNodeAddedMessage)
+                processNodeAddedMessage((TcpDiscoveryNodeAddedMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage)
+                processNodeAddFinishedMessage((TcpDiscoveryNodeAddFinishedMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryNodeLeftMessage)
+                processNodeLeftMessage((TcpDiscoveryNodeLeftMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryNodeFailedMessage)
+                processNodeFailedMessage((TcpDiscoveryNodeFailedMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryClientHeartbeatMessage)
+                processClientHeartbeatMessage((TcpDiscoveryClientHeartbeatMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryHeartbeatMessage)
+                processHeartbeatMessage((TcpDiscoveryHeartbeatMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryStatusCheckMessage)
+                processStatusCheckMessage((TcpDiscoveryStatusCheckMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryDiscardMessage)
+                processDiscardMessage((TcpDiscoveryDiscardMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryCustomEventMessage)
+                processCustomMessage((TcpDiscoveryCustomEventMessage)msg);
+
+            else if (msg instanceof TcpDiscoveryClientPingRequest)
+                processClientPingRequest((TcpDiscoveryClientPingRequest)msg);
+
+            else
+                assert false : "Unknown message type: " + msg.getClass().getSimpleName();
+
+            adapter.stats.onMessageProcessingFinished(msg);
+        }
+
+        /**
+         * Sends message across the ring.
+         *
+         * @param msg Message to send
+         */
+        @SuppressWarnings({"BreakStatementWithLabel", "LabeledStatement", "ContinueStatementWithLabel"})
+        private void sendMessageAcrossRing(TcpDiscoveryAbstractMessage msg) {
+            assert msg != null;
+
+            assert ring.hasRemoteNodes();
+
+            for (IgniteInClosure<TcpDiscoveryAbstractMessage> msgLsnr : adapter.sendMsgLsnrs)
+                msgLsnr.apply(msg);
+
+            if (redirectToClients(msg)) {
+                byte[] marshalledMsg = null;
+
+                for (ClientMessageWorker clientMsgWorker : clientMsgWorkers.values()) {
+                    // Send a clone to client to avoid ConcurrentModificationException
+                    TcpDiscoveryAbstractMessage msgClone;
+
+                    try {
+                        if (marshalledMsg == null)
+                            marshalledMsg = adapter.marsh.marshal(msg);
+
+                        msgClone = adapter.marsh.unmarshal(marshalledMsg, null);
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to marshal message: " + msg, e);
+
+                        msgClone = msg;
+                    }
+
+                    clientMsgWorker.addMessage(msgClone);
+                }
+            }
+
+            Collection<TcpDiscoveryNode> failedNodes;
+
+            TcpDiscoverySpiState state;
+
+            synchronized (mux) {
+                failedNodes = U.arrayList(ServerImpl.this.failedNodes);
+
+                state = spiState;
+            }
+
+            Collection<Throwable> errs = null;
+
+            boolean sent = false;
+
+            boolean searchNext = true;
+
+            UUID locNodeId = getLocalNodeId();
+
+            while (true) {
+                if (searchNext) {
+                    TcpDiscoveryNode newNext = ring.nextNode(failedNodes);
+
+                    if (newNext == null) {
+                        if (log.isDebugEnabled())
+                            log.debug("No next node in topology.");
+
+                        if (debugMode)
+                            debugLog("No next node in topology.");
+
+                        if (ring.hasRemoteNodes()) {
+                            msg.senderNodeId(locNodeId);
+
+                            addMessage(msg);
+                        }
+
+                        break;
+                    }
+
+                    if (!newNext.equals(next)) {
+                        if (log.isDebugEnabled())
+                            log.debug("New next node [newNext=" + newNext + ", formerNext=" + next +
+                                ", ring=" + ring + ", failedNodes=" + failedNodes + ']');
+
+                        if (debugMode)
+                            debugLog("New next node [newNext=" + newNext + ", formerNext=" + next +
+                                ", ring=" + ring + ", failedNodes=" + failedNodes + ']');
+
+                        U.closeQuiet(sock);
+
+                        sock = null;
+
+                        next = newNext;
+                    }
+                    else if (log.isDebugEnabled())
+                        log.debug("Next node remains the same [nextId=" + next.id() +
+                            ", nextOrder=" + next.internalOrder() + ']');
+                }
+
+                // Flag that shows whether next node exists and accepts incoming connections.
+                boolean nextNodeExists = sock != null;
+
+                final boolean sameHost = U.sameMacs(locNode, next);
+
+                List<InetSocketAddress> localNodeAddresses = U.arrayList(locNode.socketAddresses());
+
+                addr: for (InetSocketAddress addr : adapter.getNodeAddresses(next, sameHost)) {
+                    long ackTimeout0 = adapter.ackTimeout;
+
+                    if (localNodeAddresses.contains(addr)){
+                        if (log.isDebugEnabled())
+                            log.debug("Skip to send message to the local node (probably remote node has the same " +
+                                "loopback address that local node): " + addr);
+
+                        continue;
+                    }
+
+                    for (int i = 0; i < adapter.reconCnt; i++) {
+                        if (sock == null) {
+                            nextNodeExists = false;
+
+                            boolean success = false;
+
+                            boolean openSock = false;
+
+                            // Restore ring.
+                            try {
+                                long tstamp = U.currentTimeMillis();
+
+                                sock = adapter.openSocket(addr);
+
+                                openSock = true;
+
+                                // Handshake.
+                                writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId));
+
+                                TcpDiscoveryHandshakeResponse res = adapter.readMessage(sock, null, ackTimeout0);
+
+                                if (locNodeId.equals(res.creatorNodeId())) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Handshake response from local node: " + res);
+
+                                    U.closeQuiet(sock);
+
+                                    sock = null;
+
+                                    break;
+                                }
+
+                                adapter.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+
+                                UUID nextId = res.creatorNodeId();
+
+                                long nextOrder = res.order();
+
+                                if (!next.id().equals(nextId)) {
+                                    // Node with different ID has bounded to the same port.
+                                    if (log.isDebugEnabled())
+                                        log.debug("Failed to restore ring because next node ID received is not as " +
+                                            "expected [expectedId=" + next.id() + ", rcvdId=" + nextId + ']');
+
+                                    if (debugMode)
+                                        debugLog("Failed to restore ring because next node ID received is not as " +
+                                            "expected [expectedId=" + next.id() + ", rcvdId=" + nextId + ']');
+
+                                    break;
+                                }
+                                else {
+                                    // ID is as expected. Check node order.
+                                    if (nextOrder != next.internalOrder()) {
+                                        // Is next currently being added?
+                                        boolean nextNew = (msg instanceof TcpDiscoveryNodeAddedMessage &&
+                                            ((TcpDiscoveryNodeAddedMessage)msg).node().id().equals(nextId));
+
+                                        if (!nextNew) {
+                                            if (log.isDebugEnabled())
+                                                log.debug("Failed to restore ring because next node order received " +
+                                                    "is not as expected [expected=" + next.internalOrder() +
+                                                    ", rcvd=" + nextOrder + ", id=" + next.id() + ']');
+
+                                            if (debugMode)
+                                                debugLog("Failed to restore ring because next node order received " +
+                                                    "is not as expected [expected=" + next.internalOrder() +
+                                                    ", rcvd=" + nextOrder + ", id=" + next.id() + ']');
+
+                                            break;
+                                        }
+                                    }
+
+                                    if (log.isDebugEnabled())
+                                        log.debug("Initialized connection with next node: " + next.id());
+
+                                    if (debugMode)
+                                        debugLog("Initialized connection with next node: " + next.id());
+
+                                    errs = null;
+
+                                    success = true;
+                                }
+                            }
+                            catch (IOException | IgniteCheckedException e) {
+                                if (errs == null)
+                                    errs = new ArrayList<>();
+
+                                errs.add(e);
+
+                                if (log.isDebugEnabled())
+                                    U.error(log, "Failed to connect to next node [msg=" + msg
+                                        + ", err=" + e.getMessage() + ']', e);
+
+                                onException("Failed to connect to next node [msg=" + msg + ", err=" + e + ']', e);
+
+                                if (!openSock)
+                                    break; // Don't retry if we can not establish connection.
+
+                                if (e instanceof SocketTimeoutException ||
+                                    X.hasCause(e, SocketTimeoutException.class)) {
+                                    ackTimeout0 *= 2;
+
+                                    if (!checkAckTimeout(ackTimeout0))
+                                        break;
+                                }
+
+                                continue;
+                            }
+                            finally {
+                                if (!success) {
+                                    U.closeQuiet(sock);
+
+                                    sock = null;
+                                }
+                                else
+                                    // Next node exists and accepts incoming messages.
+                                    nextNodeExists = true;
+                            }
+                        }
+
+                        try {
+                            boolean failure;
+
+                            synchronized (mux) {
+                                failure = ServerImpl.this.failedNodes.size() < failedNodes.size();
+                            }
+
+                            assert !forceSndPending || msg instanceof TcpDiscoveryNodeLeftMessage;
+
+                            if (failure || forceSndPending) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Pending messages will be sent [failure=" + failure +
+                                        ", forceSndPending=" + forceSndPending + ']');
+
+                                if (debugMode)
+                                    debugLog("Pending messages will be sent [failure=" + failure +
+                                        ", forceSndPending=" + forceSndPending + ']');
+
+                                boolean skip = pendingMsgs.discardId != null;
+
+                                for (TcpDiscoveryAbstractMessage pendingMsg : pendingMsgs.msgs) {
+                                    if (skip) {
+                                        if (pendingMsg.id().equals(pendingMsgs.discardId))
+                                            skip = false;
+
+                                        continue;
+                                    }
+
+                                    long tstamp = U.currentTimeMillis();
+
+                                    prepareNodeAddedMessage(pendingMsg, next.id(), pendingMsgs.msgs,
+                                        pendingMsgs.discardId);
+
+                                    try {
+                                        writeToSocket(sock, pendingMsg);
+                                    }
+                                    finally {
+                                        clearNodeAddedMessage(pendingMsg);
+                                    }
+
+                                    adapter.stats.onMessageSent(pendingMsg, U.currentTimeMillis() - tstamp);
+
+                                    int res = adapter.readReceipt(sock, ackTimeout0);
+
+                                    if (log.isDebugEnabled())
+                                        log.debug("Pending message has been sent to next node [msg=" + msg.id() +
+                                            ", pendingMsgId=" + pendingMsg + ", next=" + next.id() +
+                                            ", res=" + res + ']');
+
+                                    if (debugMode)
+                                        debugLog("Pending message has been sent to next node [msg=" + msg.id() +
+                                            ", pendingMsgId=" + pendingMsg + ", next=" + next.id() +
+                                            ", res=" + res + ']');
+                                }
+                            }
+
+                            prepareNodeAddedMessage(msg, next.id(), pendingMsgs.msgs, pendingMsgs.discardId);
+
+                            try {
+                                long tstamp = U.currentTimeMillis();
+
+                                writeToSocket(sock, msg);
+
+                                adapter.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
+
+                                int res = adapter.readReceipt(sock, ackTimeout0);
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Message has been sent to next node [msg=" + msg +
+                                        ", next=" + next.id() +
+                                        ", res=" + res + ']');
+
+                                if (debugMode)
+                                    debugLog("Message has been sent to next node [msg=" + msg +
+                                        ", next=" + next.id() +
+                                        ", res=" + res + ']');
+                            }
+                            finally {
+                                clearNodeAddedMessage(msg);
+                            }
+
+                            registerPendingMessage(msg);
+
+                            sent = true;
+
+                            break addr;
+                        }
+                        catch (IOException | IgniteCheckedException e) {
+                            if (errs == null)
+                                errs = new ArrayList<>();
+
+                            errs.add(e);
+
+                            if (log.isDebugEnabled())
+                                U.error(log, "Failed to send message to next node [next=" + next.id() + ", msg=" + msg +
+                                    ", err=" + e + ']', e);
+
+                            onException("Failed to send message to next node [next=" + next.id() + ", msg=" + msg + ']',
+                                e);
+
+                            if (e instanceof SocketTimeoutException || X.hasCause(e, SocketTimeoutException.class)) {
+                                ackTimeout0 *= 2;
+
+                                if (!checkAckTimeout(ackTimeout0))
+                                    break;
+                            }
+                        }
+                        finally {
+                            forceSndPending = false;
+
+                            if (!sent) {
+                                U.closeQuiet(sock);
+
+                                sock = null;
+
+                                if (log.isDebugEnabled())
+                                    log.debug("Message has not been sent [next=" + next.id() + ", msg=" + msg +
+                                        ", i=" + i + ']');
+                            }
+                        }
+                    } // Try to reconnect.
+                } // Iterating node's addresses.
+
+                if (!sent) {
+                    if (!failedNodes.contains(next)) {
+                        failedNodes.add(next);
+
+                        if (state == CONNECTED) {
+                            Exception err = errs != null ?
+                                U.exceptionWithSuppressed("Failed to send message to next node [msg=" + msg +
+                                    ", next=" + U.toShortString(next) + ']', errs) :
+                                null;
+
+                            // If node existed on connection initialization we should check
+                            // whether it has not gone yet.
+                            if (nextNodeExists && pingNode(next))
+                                U.error(log, "Failed to send message to next node [msg=" + msg +
+                                    ", next=" + next + ']', err);
+                            else if (log.isDebugEnabled())
+                                log.debug("Failed to send message to next node [msg=" + msg + ", next=" + next +
+                                    ", errMsg=" + (err != null ? err.getMessage() : "N/A") + ']');
+                        }
+                    }
+
+                    if (msg instanceof TcpDiscoveryStatusCheckMessage) {
+                        TcpDiscoveryStatusCheckMessage msg0 = (TcpDiscoveryStatusCheckMessage)msg;
+
+                        if (next.id().equals(msg0.failedNodeId())) {
+                            next = null;
+
+                            if (log.isDebugEnabled())
+                                log.debug("Discarding status check since next node has indeed failed [next=" + next +
+                                    ", msg=" + msg + ']');
+
+                            // Discard status check message by exiting loop and handle failure.
+                            break;
+                        }
+                    }
+
+                    next = null;
+
+                    searchNext = true;
+
+                    errs = null;
+                }
+                else
+                    break;
+            }
+
+            synchronized (mux) {
+                failedNodes.removeAll(ServerImpl.this.failedNodes);
+            }
+
+            if (!failedNodes.isEmpty()) {
+                if (state == CONNECTED) {
+                    if (!sent && log.isDebugEnabled())
+                        // Message has not been sent due to some problems.
+                        log.debug("Message has not been sent: " + m

<TRUNCATED>


[46/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix GridSegmentationSelfTest

Posted by se...@apache.org.
# IGNITE-943 Fix GridSegmentationSelfTest


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

Branch: refs/heads/ignite-sprint-5
Commit: ccc0d4038a5d41a8d83e99508279dd17d90d06df
Parents: 99234d0
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 16:39:59 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 16:40:37 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java  | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ccc0d403/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 1b8424c..af85050 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
@@ -1688,6 +1688,13 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /**
      * <strong>FOR TEST ONLY!!!</strong>
      */
+    void forceNextNodeFailure() {
+        ((ServerImpl)impl).forceNextNodeFailure();
+    }
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     */
     public void addSendMessageListener(IgniteInClosure<TcpDiscoveryAbstractMessage> lsnr) {
         sendMsgLsnrs.add(lsnr);
     }


[47/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Increase test timeout.

Posted by se...@apache.org.
# IGNITE-943 Increase test timeout.


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

Branch: refs/heads/ignite-sprint-5
Commit: b3d8e1e13bfc0049de5832ca1291ed5511e3f191
Parents: ccc0d40
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 17:11:44 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 17:11:44 2015 +0300

----------------------------------------------------------------------
 .../ignite/internal/processors/cache/CacheRemoveAllSelfTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b3d8e1e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
index f5de96f..1d4d2f4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRemoveAllSelfTest.java
@@ -32,7 +32,7 @@ import java.util.concurrent.atomic.*;
 public class CacheRemoveAllSelfTest extends GridCacheAbstractSelfTest {
     /** {@inheritDoc} */
     @Override protected long getTestTimeout() {
-        return 60000;
+        return 2 * 60 * 1000;
     }
 
     /** {@inheritDoc} */


[41/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix TcpClientDiscoverySpiSelfTest.testMetrics

Posted by se...@apache.org.
# IGNITE-943 Fix TcpClientDiscoverySpiSelfTest.testMetrics


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

Branch: refs/heads/ignite-sprint-5
Commit: 2477c0ebd2e8003b58e3a83b4389d1118b30b5fc
Parents: 1b0a051
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 13:44:29 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 13:44:29 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java  | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2477c0eb/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index be3474d..f7be340 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -681,7 +681,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
             Ignite g = G.ignite("server-" + i);
 
             for (ClusterNode n : g.cluster().nodes()) {
-                if (n.metrics().getTotalExecutedJobs() != execJobsCnt)
+                if (n.metrics().getTotalExecutedJobs() != (n.isClient() ? 0 : execJobsCnt))
                     return false;
             }
         }
@@ -690,7 +690,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
             Ignite g = G.ignite("client-" + i);
 
             for (ClusterNode n : g.cluster().nodes()) {
-                if (n.metrics().getTotalExecutedJobs() != execJobsCnt)
+                if (n.metrics().getTotalExecutedJobs() != (n.isClient() ? 0 : execJobsCnt))
                     return false;
             }
         }


[25/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix tests

Posted by se...@apache.org.
# IGNITE-943 Fix tests


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

Branch: refs/heads/ignite-sprint-5
Commit: 154bd9e318b1ed68c7894169800695df47649d75
Parents: cc1612d
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 16:07:25 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 16:07:25 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAtomicMessageCountSelfTest.java    | 1 +
 .../distributed/IgniteCacheClientNodeChangingTopologyTest.java   | 2 +-
 .../near/GridCacheRendezvousAffinityClientSelfTest.java          | 4 ++++
 3 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/154bd9e3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
index da2b81c..db4061a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAtomicMessageCountSelfTest.java
@@ -60,6 +60,7 @@ public class GridCacheAtomicMessageCountSelfTest extends GridCommonAbstractTest
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
+        discoSpi.setForceServerMode(true);
         discoSpi.setIpFinder(ipFinder);
 
         cfg.setDiscoverySpi(discoSpi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/154bd9e3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index 47c1d7e..ff99d3b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -79,7 +79,7 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
 
         cfg.setClientMode(client);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/154bd9e3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
index 80651bf..3bab050 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheRendezvousAffinityClientSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.cache.affinity.rendezvous.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import java.util.*;
@@ -35,9 +36,12 @@ public class GridCacheRendezvousAffinityClientSelfTest extends GridCommonAbstrac
     /** Client node. */
     private boolean client;
 
+    /** {@inheritDoc} */
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         CacheConfiguration ccfg = defaultCacheConfiguration();
 
         ccfg.setCacheMode(CacheMode.PARTITIONED);


[28/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix tests

Posted by se...@apache.org.
# IGNITE-943 Fix  tests


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

Branch: refs/heads/ignite-sprint-5
Commit: a02cb41ef75945f1f57cf960494705bde5439dc9
Parents: 9759b19
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 17:10:09 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 17:10:09 2015 +0300

----------------------------------------------------------------------
 .../cache/distributed/GridCacheMixedModeSelfTest.java          | 3 +++
 .../distributed/near/GridCacheNearOnlyTopologySelfTest.java    | 1 +
 .../lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java    | 6 +-----
 3 files changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a02cb41e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
index e19442f..a3c977f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMixedModeSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 /**
@@ -31,6 +32,8 @@ public class GridCacheMixedModeSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         cfg.setCacheConfiguration(cacheConfiguration(gridName));
 
         if (F.eq(gridName, getTestGridName(0)))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a02cb41e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
index 2893678..b6bc56e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearOnlyTopologySelfTest.java
@@ -70,6 +70,7 @@ public class GridCacheNearOnlyTopologySelfTest extends GridCommonAbstractTest {
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
+        discoSpi.setForceServerMode(true);
         discoSpi.setIpFinder(IP_FINDER);
 
         cfg.setDiscoverySpi(discoSpi);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a02cb41e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
index c401180..8bcb68a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/lru/GridCacheNearOnlyLruNearEvictionPolicySelfTest.java
@@ -79,11 +79,7 @@ public class GridCacheNearOnlyLruNearEvictionPolicySelfTest extends GridCommonAb
             c.setCacheConfiguration(cc);
         }
 
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
+        c.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder).setForceServerMode(true));
 
         cnt++;
 


[51/53] [abbrv] incubator-ignite git commit: # ignite-943

Posted by se...@apache.org.
# ignite-943


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

Branch: refs/heads/ignite-sprint-5
Commit: 6c837c0783f04874157ac414f7161273c8c4d9f8
Parents: f499cfe
Author: sboikov <se...@inria.fr>
Authored: Fri May 29 23:43:25 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Fri May 29 23:43:25 2015 +0300

----------------------------------------------------------------------
 .../processors/service/GridServiceProcessor.java         | 11 ++++++-----
 1 file changed, 6 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6c837c07/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index aedac75..8eff6bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -1101,11 +1101,12 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             try {
                 depExe.submit(new BusyRunnable() {
                     @Override public void run0() {
-                        long topVer = ((DiscoveryEvent)evt).topologyVersion();
+                        AffinityTopologyVersion topVer =
+                            new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion());
 
-                        ClusterNode oldest = U.oldest(ctx.discovery().nodes(topVer), null);
+                        ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
 
-                        if (oldest.isLocal()) {
+                        if (oldest != null && oldest.isLocal()) {
                             final Collection<GridServiceDeployment> retries = new ConcurrentLinkedQueue<>();
 
                             if (ctx.deploy().enabled())
@@ -1128,7 +1129,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                         ctx.cache().internalCache(UTILITY_CACHE_NAME).context().affinity().
                                             affinityReadyFuture(topVer).get();
 
-                                        reassign(dep, topVer);
+                                        reassign(dep, topVer.topologyVersion());
                                     }
                                     catch (IgniteCheckedException ex) {
                                         if (!(e instanceof ClusterTopologyCheckedException))
@@ -1145,7 +1146,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                             }
 
                             if (!retries.isEmpty())
-                                onReassignmentFailed(topVer, retries);
+                                onReassignmentFailed(topVer.topologyVersion(), retries);
                         }
 
                         // Clean up zombie assignments.


[17/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Added documentation.

Posted by se...@apache.org.
# IGNITE-943 Added documentation.


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

Branch: refs/heads/ignite-sprint-5
Commit: 77d1dcbdf014b65898b358f18e56ce6ec4e1b27a
Parents: b3ac88f
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 13:04:07 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 14:46:08 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpDiscoverySpi.java    | 15 +++++++++++++++
 1 file changed, 15 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/77d1dcbd/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 71ba0f1..aa8fb76 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
@@ -53,6 +53,10 @@ import java.util.concurrent.atomic.*;
  * Nodes are organized in ring. So almost all network exchange (except few cases) is
  * done across it.
  * <p>
+ * Node may be started in client mode, in this case node does not insert to the ring,
+ * it connects to any node in the ring router and communicated with that node only.
+ * Thereby slowing or shutdown of client node will not affect whole cluster.
+ * <p>
  * At startup SPI tries to send messages to random IP taken from
  * {@link TcpDiscoveryIpFinder} about self start (stops when send succeeds)
  * and then this info goes to coordinator. When coordinator processes join request
@@ -481,6 +485,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      * is reached, then the process of message sending is considered as failed.
      * <p>
      * If not specified, default is {@link #DFLT_MAX_ACK_TIMEOUT}.
+     * <p>
+     * Affected server nodes only.
      *
      * @param maxAckTimeout Maximum acknowledgement timeout.
      */
@@ -502,6 +508,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      * Sets local port to listen to.
      * <p>
      * If not specified, default is {@link #DFLT_PORT}.
+     * <p>
+     * Affected server nodes only.
      *
      * @param locPort Local port to bind.
      */
@@ -523,6 +531,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      * <tt>{@link #getLocalPort()} {@code + locPortRange}</tt>.
      * <p>
      * If not specified, default is {@link #DFLT_PORT_RANGE}.
+     * <p>
+     * Affected server nodes only.
+
      *
      * @param locPortRange Local port range to bind.
      */
@@ -542,6 +553,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      * Sets max heartbeats count node can miss without initiating status check.
      * <p>
      * If not provided, default value is {@link #DFLT_MAX_MISSED_HEARTBEATS}.
+     * <p>
+     * Affected server nodes only.
      *
      * @param maxMissedHbs Max missed heartbeats.
      */
@@ -603,6 +616,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      * Sets IP finder clean frequency in milliseconds.
      * <p>
      * If not provided, default value is {@link #DFLT_IP_FINDER_CLEAN_FREQ}
+     * <p>
+     * Affected server nodes only.
      *
      * @param ipFinderCleanFreq IP finder clean frequency.
      */


[44/53] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-943

Posted by se...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-943


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

Branch: refs/heads/ignite-sprint-5
Commit: 477a2150ddc597b79846f862759dd6aed2cf4294
Parents: eb810196 07b6cb5
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 16:07:40 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 16:07:40 2015 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionMap.java      |   2 +-
 .../GridDhtPartitionsExchangeFuture.java        |   2 +-
 ...niteCacheClientNodeChangingTopologyTest.java |   6 ++
 .../IgniteCacheClientNodeConcurrentStart.java   | 105 +++++++++++++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |   6 +-
 .../junits/common/GridCommonAbstractTest.java   |  61 +++++++++--
 .../testsuites/IgniteCacheTestSuite2.java       |   1 +
 7 files changed, 170 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/477a2150/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/477a2150/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------


[35/53] [abbrv] incubator-ignite git commit: # ignite-709 Fix tests

Posted by se...@apache.org.
# ignite-709 Fix tests


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

Branch: refs/heads/ignite-sprint-5
Commit: 2799c3a69c2ddfb0b7c5f8f1c88808948fa40664
Parents: cfe5621
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 19:26:44 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 19:26:44 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAbstractFullApiSelfTest.java        | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2799c3a6/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 70d8f9c..5c31396 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -33,6 +33,7 @@ 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.spi.discovery.tcp.*;
 import org.apache.ignite.spi.swapspace.inmemory.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.transactions.*;
@@ -130,6 +131,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         if (memoryMode() == OFFHEAP_TIERED || memoryMode() == OFFHEAP_VALUES)
             cfg.setSwapSpaceSpi(new GridTestSwapSpaceSpi());
 


[30/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix tests

Posted by se...@apache.org.
# IGNITE-943 Fix  tests


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

Branch: refs/heads/ignite-sprint-5
Commit: 7f98e291fe4b55957d7d26fe6e778dad23f5510d
Parents: 0e7fc0a
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 17:14:53 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 17:14:53 2015 +0300

----------------------------------------------------------------------
 .../internal/processors/cache/GridCachePutAllFailoverSelfTest.java  | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/7f98e291/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
index ae36c4de..fc9e17b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
@@ -617,6 +617,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
 
         discoverySpi.setAckTimeout(60000);
         discoverySpi.setIpFinder(ipFinder);
+        discoverySpi.setForceServerMode(true);
 
         cfg.setDiscoverySpi(discoverySpi);
 


[50/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix example tests.

Posted by se...@apache.org.
# IGNITE-943 Fix example tests.


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

Branch: refs/heads/ignite-sprint-5
Commit: f499cfeaf140cf50355be775a79c186102833dfa
Parents: d10120d
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 20:07:10 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 20:07:10 2015 +0300

----------------------------------------------------------------------
 .../apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java  | 10 ++++++++--
 .../ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java | 10 ++++++----
 2 files changed, 14 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f499cfea/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 af85050..56fb63f 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
@@ -1614,8 +1614,14 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
         U.interrupt(sockTimeoutWorker);
         U.join(sockTimeoutWorker, log);
 
-        if (ipFinder != null)
-            ipFinder.close();
+        if (ipFinder != null) {
+            try {
+                ipFinder.close();
+            }
+            catch (Exception e) {
+                log.error("Failed to close ipFinder", e);
+            }
+        }
 
         unregisterMBean();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f499cfea/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
index 6cf06ab..45d0816 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
@@ -495,11 +495,13 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
 
     /** {@inheritDoc} */
     @Override public void close() {
-        for (AddressSender addrSnd : addrSnds)
-            U.interrupt(addrSnd);
+        if (addrSnds != null) {
+            for (AddressSender addrSnd : addrSnds)
+                U.interrupt(addrSnd);
 
-        for (AddressSender addrSnd : addrSnds)
-            U.join(addrSnd, log);
+            for (AddressSender addrSnd : addrSnds)
+                U.join(addrSnd, log);
+        }
     }
 
     /** {@inheritDoc} */


[34/53] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-943

Posted by se...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-943


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

Branch: refs/heads/ignite-sprint-5
Commit: cfe56213e1dfc011de47e0425627b3ff73a57763
Parents: 9cddb6f b6fc8a9
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 18:02:47 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 18:02:47 2015 +0300

----------------------------------------------------------------------
 dev-tools/gradle/wrapper/gradle-wrapper.jar     | Bin 51017 -> 0 bytes
 .../gradle/wrapper/gradle-wrapper.properties    |  18 +-
 dev-tools/gradlew                               | 163 ++++++++----
 dev-tools/slurp.sh                              |   2 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  73 ++++--
 examples/pom.xml                                |   2 +-
 modules/aop/pom.xml                             |   2 +-
 modules/aws/pom.xml                             |   2 +-
 modules/clients/pom.xml                         |   2 +-
 modules/cloud/pom.xml                           |   2 +-
 modules/codegen/pom.xml                         |   2 +-
 modules/core/pom.xml                            |   2 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   8 +-
 .../java/org/apache/ignite/IgniteServices.java  |   5 +-
 .../apache/ignite/internal/IgniteKernal.java    |   4 +-
 .../org/apache/ignite/services/Service.java     |   5 +-
 .../core/src/main/resources/ignite.properties   |   2 +-
 .../cache/IgniteDynamicCacheStartSelfTest.java  |  20 +-
 .../service/ClosureServiceClientsNodesTest.java | 245 +++++++++++++++++++
 .../ignite/testsuites/IgniteBasicTestSuite.java |   2 +
 modules/extdata/p2p/pom.xml                     |   2 +-
 modules/extdata/uri/pom.xml                     |   2 +-
 modules/gce/pom.xml                             |   2 +-
 modules/geospatial/pom.xml                      |   2 +-
 modules/hadoop/pom.xml                          |   2 +-
 modules/hibernate/pom.xml                       |   2 +-
 modules/indexing/pom.xml                        |   2 +-
 .../query/h2/sql/GridSqlQuerySplitter.java      |   4 +
 .../query/h2/sql/BaseH2CompareQueryTest.java    |  16 ++
 modules/jcl/pom.xml                             |   2 +-
 modules/jta/pom.xml                             |   2 +-
 modules/log4j/pom.xml                           |   2 +-
 modules/rest-http/pom.xml                       |   2 +-
 modules/scalar/pom.xml                          |   2 +-
 modules/schedule/pom.xml                        |   2 +-
 modules/schema-import/pom.xml                   |   2 +-
 modules/slf4j/pom.xml                           |   2 +-
 modules/spring/pom.xml                          |   2 +-
 modules/ssh/pom.xml                             |   2 +-
 modules/tools/pom.xml                           |   2 +-
 modules/urideploy/pom.xml                       |   2 +-
 .../licenses/jcraft-revised-bsd.txt             |  28 ---
 modules/visor-console/pom.xml                   |   2 +-
 modules/visor-plugins/pom.xml                   |   2 +-
 modules/web/pom.xml                             |   2 +-
 modules/yardstick/pom.xml                       |   2 +-
 pom.xml                                         |   2 +-
 47 files changed, 498 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cfe56213/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cfe56213/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------


[40/53] [abbrv] incubator-ignite git commit: ignite-943 Fix tests: add setForceServerMode(true)

Posted by se...@apache.org.
ignite-943 Fix tests: add setForceServerMode(true)


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

Branch: refs/heads/ignite-sprint-5
Commit: 1b0a051d4f91adc827e84ac816e3082ada6081ef
Parents: 6b1265c
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 13:15:25 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 13:15:59 2015 +0300

----------------------------------------------------------------------
 .../internal/GridReleaseTypeSelfTest.java       | 33 ++++++--------------
 ...GridCacheMixedPartitionExchangeSelfTest.java |  2 +-
 .../GridCacheReturnValueTransferSelfTest.java   |  3 ++
 .../IgniteCacheAbstractStopBusySelfTest.java    |  6 +---
 .../cache/IgniteCacheAbstractTest.java          |  2 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |  5 +--
 .../cache/IgniteCacheAbstractQuerySelfTest.java |  6 +---
 7 files changed, 18 insertions(+), 39 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
index 79e8905..96d5f1a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
@@ -45,32 +45,19 @@ public class GridReleaseTypeSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi;
-
-        if (clientMode) {
-            discoSpi = new TcpDiscoverySpi() {
-                @Override public void setNodeAttributes(Map<String, Object> attrs,
-                    IgniteProductVersion ver) {
-                    super.setNodeAttributes(attrs, ver);
+        if (clientMode)
+            cfg.setClientMode(true);
 
-                    attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
-                }
-            };
+        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi() {
+            @Override public void setNodeAttributes(Map<String, Object> attrs,
+                IgniteProductVersion ver) {
+                super.setNodeAttributes(attrs, ver);
 
-            cfg.setClientMode(true);
-        }
-        else {
-            discoSpi = new TcpDiscoverySpi() {
-                @Override public void setNodeAttributes(Map<String, Object> attrs,
-                    IgniteProductVersion ver) {
-                    super.setNodeAttributes(attrs, ver);
-
-                    attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
-                }
-            };
-        }
+                attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
+            }
+        };
 
-        discoSpi.setIpFinder(IP_FINDER);
+        discoSpi.setIpFinder(IP_FINDER).setForceServerMode(true);
 
         cfg.setDiscoverySpi(discoSpi);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
index 266937f..2fad523 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
@@ -56,7 +56,7 @@ public class GridCacheMixedPartitionExchangeSelfTest extends GridCommonAbstractT
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
 
         if (cache)
             cfg.setCacheConfiguration(cacheConfiguration());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
index 3e5987a..7723319 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReturnValueTransferSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 
 import javax.cache.processor.*;
@@ -62,6 +63,8 @@ public class GridCacheReturnValueTransferSelfTest extends GridCommonAbstractTest
 
         cfg.setCacheConfiguration(ccfg);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         if (!cache)
             cfg.setClientMode(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
index aa8e2f7..10b14cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractStopBusySelfTest.java
@@ -101,11 +101,7 @@ public abstract class IgniteCacheAbstractStopBusySelfTest extends GridCommonAbst
 
         cfg.setCommunicationSpi(commSpi);
 
-        TcpDiscoverySpi spi = new TcpDiscoverySpi();
-
-        spi.setIpFinder(finder);
-
-        cfg.setDiscoverySpi(spi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(finder).setForceServerMode(true));
 
         cfg.setCacheConfiguration(cacheCfg);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
index 371e547..1a62d4b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractTest.java
@@ -82,7 +82,7 @@ public abstract class IgniteCacheAbstractTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
+        TcpDiscoverySpi disco = new TcpDiscoverySpi().setForceServerMode(true);
 
         disco.setMaxMissedHeartbeats(Integer.MAX_VALUE);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
index d3440fc..c0f73af 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/igfs/IgfsNearOnlyMultiNodeSelfTest.java
@@ -73,10 +73,7 @@ public class IgfsNearOnlyMultiNodeSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER).setForceServerMode(true));
 
         FileSystemConfiguration igfsCfg = new FileSystemConfiguration();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/1b0a051d/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
index 1a60bbd..6224cb9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractQuerySelfTest.java
@@ -99,11 +99,7 @@ public abstract class IgniteCacheAbstractQuerySelfTest extends GridCommonAbstrac
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration c = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi disco = new TcpDiscoverySpi();
-
-        disco.setIpFinder(ipFinder);
-
-        c.setDiscoverySpi(disco);
+        c.setDiscoverySpi(new TcpDiscoverySpi().setForceServerMode(true).setIpFinder(ipFinder));
 
         // Otherwise noop swap space will be chosen on Windows.
         c.setSwapSpaceSpi(new FileSwapSpaceSpi());


[38/53] [abbrv] incubator-ignite git commit: # ignite-709 can not skip preload on server nodes

Posted by se...@apache.org.
# ignite-709 can not skip preload on server nodes


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

Branch: refs/heads/ignite-sprint-5
Commit: 2c3ce158de266dfd02c91e3e6e377ce667fd598f
Parents: 581f4d9
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 10:35:24 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 10:35:24 2015 +0300

----------------------------------------------------------------------
 .../GridDhtPartitionsExchangeFuture.java        |   2 +-
 .../IgniteCacheClientNodeConcurrentStart.java   | 105 +++++++++++++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |   6 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   1 +
 4 files changed, 110 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3ce158/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 162c7b3..145def8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -517,7 +517,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
                         onDone(exchId.topologyVersion());
 
-                        skipPreload = true;
+                        skipPreload = cctx.kernalContext().clientNode();
 
                         return;
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3ce158/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java
new file mode 100644
index 0000000..bd74ece
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeConcurrentStart.java
@@ -0,0 +1,105 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.util.*;
+import java.util.concurrent.*;
+
+import static org.apache.ignite.cache.CacheRebalanceMode.*;
+
+/**
+ *
+ */
+public class IgniteCacheClientNodeConcurrentStart extends GridCommonAbstractTest {
+    /** */
+    protected static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES_CNT = 5;
+
+    /** */
+    private Set<Integer> clientNodes;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        assertNotNull(clientNodes);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        boolean client = false;
+
+        for (Integer clientIdx : clientNodes) {
+            if (getTestGridName(clientIdx).equals(gridName)) {
+                client = true;
+
+                break;
+            }
+        }
+
+        cfg.setClientMode(client);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setBackups(0);
+        ccfg.setRebalanceMode(SYNC);
+
+        cfg.setCacheConfiguration(ccfg);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testConcurrentStart() throws Exception {
+        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+        for (int i = 0; i < 3; i++) {
+            try {
+                clientNodes = new HashSet<>();
+
+                while (clientNodes.size() < 2)
+                    clientNodes.add(rnd.nextInt(0, NODES_CNT));
+
+                clientNodes.add(NODES_CNT - 1);
+
+                log.info("Test iteration [iter=" + i + ", clients=" + clientNodes + ']');
+
+                startGridsMultiThreaded(NODES_CNT, true);
+
+                for (int node : clientNodes) {
+                    Ignite ignite = grid(node);
+
+                    assertTrue(ignite.configuration().isClientMode());
+                }
+            }
+            finally {
+                stopAllGrids();
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3ce158/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
index 162aa81..68ae211 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
@@ -158,11 +158,11 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
 
         Ignite ignite1 = startGrid(1);
 
-        assertFalse(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
 
         ignite1.close();
 
-        assertFalse(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
 
         ignite1 = startGrid(1);
 
@@ -178,7 +178,7 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
             }
         }, EventType.EVT_CACHE_REBALANCE_STARTED, EventType.EVT_CACHE_REBALANCE_STOPPED);
 
-        assertFalse(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
+        assertTrue(evtLatch0.await(1000, TimeUnit.MILLISECONDS));
 
         client = false;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/2c3ce158/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
index 4664c66..037af9d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite2.java
@@ -136,6 +136,7 @@ public class IgniteCacheTestSuite2 extends TestSuite {
         suite.addTest(new TestSuite(IgniteCachePartitionMapUpdateTest.class));
         suite.addTest(new TestSuite(IgniteCacheClientNodePartitionsExchangeTest.class));
         suite.addTest(new TestSuite(IgniteCacheClientNodeChangingTopologyTest.class));
+        suite.addTest(new TestSuite(IgniteCacheClientNodeConcurrentStart.class));
 
         return suite;
     }


[14/53] [abbrv] incubator-ignite git commit: # ignite-943 minor

Posted by se...@apache.org.
# ignite-943 minor


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

Branch: refs/heads/ignite-sprint-5
Commit: 92b2a57b040d49536be03e36d0fa69edc9e32771
Parents: 5ecc08e
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed May 27 19:13:07 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed May 27 19:13:07 2015 +0300

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/ignite/cluster/ClusterNode.java  | 2 +-
 .../core/src/main/java/org/apache/ignite/internal/IgnitionEx.java  | 2 +-
 .../java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java  | 1 -
 3 files changed, 2 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/92b2a57b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
index 8f56372..85fd08a 100644
--- a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
@@ -248,9 +248,9 @@ public interface ClusterNode {
      * while topology clients connect to topology in a different way.
      *
      * @return {@code True} if this node is a client node, {@code false} otherwise.
-     * @see DiscoverySpi#isClientMode()
      * @see IgniteConfiguration#isClientMode()
      * @see Ignition#isClientMode()
+     * @see DiscoverySpi#isClientMode()
      */
     public boolean isClient();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/92b2a57b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 42d9cf9..7eb6262 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1817,7 +1817,7 @@ public class IgnitionEx {
             if (cfg.getDiscoverySpi() == null) {
                 cfg.setDiscoverySpi(new TcpDiscoverySpi());
 
-                if (cfg.isClientMode() != null && cfg.isClientMode())
+                if (Boolean.TRUE.equals(cfg.isClientMode()))
                     ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientMode(true);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/92b2a57b/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 d995717..5676fd5 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
@@ -1877,5 +1877,4 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
             return S.toString(SocketTimeoutObject.class, this);
         }
     }
-
 }


[52/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Move static class instances.

Posted by se...@apache.org.
# IGNITE-943 Move static class instances.


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

Branch: refs/heads/ignite-sprint-5
Commit: ab4c18caca9f3bd5dbc3174eb49ed79c1744a79d
Parents: 6c837c0
Author: sevdokimov <se...@gridgain.com>
Authored: Mon Jun 1 14:41:08 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Mon Jun 1 14:41:08 2015 +0300

----------------------------------------------------------------------
 .../service/GridServiceProcessor.java           | 22 +++++++++++---------
 1 file changed, 12 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ab4c18ca/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 8eff6bd..2e31b69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -60,12 +60,6 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /** Time to wait before reassignment retries. */
     private static final long RETRY_TIMEOUT = 1000;
 
-    /** */
-    private static final ServiceDeploymentPredicate DEPLOYMENT_PREDICATE = new ServiceDeploymentPredicate();
-
-    /** */
-    private static final ServiceAssignmentsPredicate ASSIGNMENTS_PREDICATE = new ServiceAssignmentsPredicate();
-
     /** Local service instances. */
     private final Map<String, Collection<ServiceContextImpl>> locSvcs = new HashMap<>();
 
@@ -352,7 +346,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']'));
                         }
                         else {
-                            Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ASSIGNMENTS_PREDICATE);
+                            Iterator<Cache.Entry<Object, Object>> it = serviceEntries(
+                                ServiceAssignmentsPredicate.INSTANCE);
 
                             while (it.hasNext()) {
                                 Cache.Entry<Object, Object> e = it.next();
@@ -448,7 +443,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public IgniteInternalFuture<?> cancelAll() {
         Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
-        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ServiceDeploymentPredicate.INSTANCE);
 
         while (it.hasNext()) {
             Cache.Entry<Object, Object> e = it.next();
@@ -471,7 +466,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public Collection<ServiceDescriptor> serviceDescriptors() {
         Collection<ServiceDescriptor> descs = new ArrayList<>();
 
-        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ServiceDeploymentPredicate.INSTANCE);
 
         while (it.hasNext()) {
             Cache.Entry<Object, Object> e = it.next();
@@ -1113,7 +1108,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 ctx.cache().context().deploy().ignoreOwnership(true);
 
                             try {
-                                Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+                                Iterator<Cache.Entry<Object, Object>> it = serviceEntries(
+                                    ServiceDeploymentPredicate.INSTANCE);
 
                                 while (it.hasNext()) {
                                     Cache.Entry<Object, Object> e = it.next();
@@ -1332,6 +1328,9 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      */
     static class ServiceDeploymentPredicate implements IgniteBiPredicate<Object, Object> {
         /** */
+        static final ServiceDeploymentPredicate INSTANCE = new ServiceDeploymentPredicate();
+
+        /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
@@ -1350,6 +1349,9 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      */
     static class ServiceAssignmentsPredicate implements IgniteBiPredicate<Object, Object> {
         /** */
+        static final ServiceAssignmentsPredicate INSTANCE = new ServiceAssignmentsPredicate();
+
+        /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */


[15/53] [abbrv] incubator-ignite git commit: # IGNITE-943 DiscoverySpi.setNodeAttributes should return void

Posted by se...@apache.org.
# IGNITE-943 DiscoverySpi.setNodeAttributes should return void


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

Branch: refs/heads/ignite-sprint-5
Commit: 966276e11b593f09d8f211aa1d2e6e51671d6bf7
Parents: 0e192ef
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 12:28:49 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 14:46:07 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/spi/discovery/DiscoverySpi.java   | 2 +-
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 4 +---
 .../org/apache/ignite/internal/GridReleaseTypeSelfTest.java  | 8 ++------
 ...acheAffinityFunctionExcludeNeighborsAbstractSelfTest.java | 4 +---
 4 files changed, 5 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/966276e1/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index 4996d16..f9cec34 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -81,7 +81,7 @@ public interface DiscoverySpi extends IgniteSpi {
      *  @param attrs Map of node attributes.
      * @param ver Product version.
      */
-    public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver);
+    public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver);
 
     /**
      * Sets a listener for discovery events. Refer to

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/966276e1/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 5676fd5..71ba0f1 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
@@ -769,7 +769,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /** {@inheritDoc} */
-    @Override public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+    @Override public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
         assert locNodeAttrs == null;
         assert locNodeVer == null;
 
@@ -780,8 +780,6 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
 
         locNodeAttrs = attrs;
         locNodeVer = ver;
-
-        return this;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/966276e1/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
index 603f66c..e6a69cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridReleaseTypeSelfTest.java
@@ -49,13 +49,11 @@ public class GridReleaseTypeSelfTest extends GridCommonAbstractTest {
 
         if (clientMode) {
             discoSpi = new TcpDiscoverySpi() {
-                @Override public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs,
+                @Override public void setNodeAttributes(Map<String, Object> attrs,
                     IgniteProductVersion ver) {
                     super.setNodeAttributes(attrs, ver);
 
                     attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
-
-                    return this;
                 }
             };
 
@@ -65,13 +63,11 @@ public class GridReleaseTypeSelfTest extends GridCommonAbstractTest {
         }
         else {
             discoSpi = new TcpDiscoverySpi() {
-                @Override public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs,
+                @Override public void setNodeAttributes(Map<String, Object> attrs,
                     IgniteProductVersion ver) {
                     super.setNodeAttributes(attrs, ver);
 
                     attrs.put(IgniteNodeAttributes.ATTR_BUILD_VER, nodeVer);
-
-                    return this;
                 }
             };
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/966276e1/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
index e7daec3..7cdf265 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest.java
@@ -55,7 +55,7 @@ public abstract class GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest
 
         // Override node attributes in discovery spi.
         TcpDiscoverySpi spi = new TcpDiscoverySpi() {
-            @Override public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs,
+            @Override public void setNodeAttributes(Map<String, Object> attrs,
                 IgniteProductVersion ver) {
                 super.setNodeAttributes(attrs, ver);
 
@@ -65,8 +65,6 @@ public abstract class GridCacheAffinityFunctionExcludeNeighborsAbstractSelfTest
                 attrs.put(IgniteNodeAttributes.ATTR_MACS, macAddrs);
 
                 gridInstanceNum++;
-
-                return this;
             }
         };
 


[11/53] [abbrv] incubator-ignite git commit: IGNITE-943 Fix tests.

Posted by se...@apache.org.
IGNITE-943 Fix tests.


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

Branch: refs/heads/ignite-sprint-5
Commit: e66578ea8fdc14ce25dace68600c2e5317818bad
Parents: bb35c88
Author: sevdokimov <se...@gridgain.com>
Authored: Wed May 27 18:16:59 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Wed May 27 18:16:59 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java  | 4 ++--
 .../org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java  | 7 +++++++
 2 files changed, 9 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e66578ea/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 46a89aa..2cd8212 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
@@ -4080,7 +4080,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     // Ping.
                     if (msg instanceof TcpDiscoveryPingRequest) {
-                        if (!adapter.getSpiContext().isStopping()) {
+                        if (!adapter.isNodeStopping0()) {
                             TcpDiscoveryPingRequest req = (TcpDiscoveryPingRequest)msg;
 
                             TcpDiscoveryPingResponse res = new TcpDiscoveryPingResponse(locNodeId);
@@ -4649,7 +4649,7 @@ class ServerImpl extends TcpDiscoveryImpl {
          *
          */
         public boolean ping() throws InterruptedException {
-            if (adapter.getSpiContext().isStopping())
+            if (adapter.isNodeStopping0())
                 return false;
 
             GridFutureAdapter<Boolean> fut;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e66578ea/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 90ab05a..93a1ec3 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
@@ -1610,6 +1610,13 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /**
+     * @return {@code True} if node is stopping.
+     */
+    boolean isNodeStopping0() {
+        return super.isNodeStopping();
+    }
+
+    /**
      * @throws IgniteSpiException If any error occurs.
      * @return {@code true} if IP finder contains local address.
      */


[19/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix performance suggestion, move it to GridDiscoveryManager

Posted by se...@apache.org.
# IGNITE-943 Fix performance suggestion, move it to GridDiscoveryManager


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

Branch: refs/heads/ignite-sprint-5
Commit: b0d9ef8ab3f7880c04b0dd8e301e808a66e3d800
Parents: 966276e
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 12:38:38 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 14:46:08 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/IgniteKernal.java | 3 ---
 .../internal/managers/discovery/GridDiscoveryManager.java      | 6 ++++++
 2 files changed, 6 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0d9ef8a/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 d6e3ca4..9d752ce 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
@@ -1072,9 +1072,6 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         if (cfg.getIncludeEventTypes() != null && cfg.getIncludeEventTypes().length != 0)
             perf.add("Disable grid events (remove 'includeEventTypes' from configuration)");
 
-        if (Boolean.TRUE.equals(cfg.isClientMode()) && !cfg.getDiscoverySpi().isClientMode())
-            perf.add("Use TcpDiscoverySpi in client mode for client node");
-
         if (OptimizedMarshaller.available() && !(cfg.getMarshaller() instanceof OptimizedMarshaller))
             perf.add("Enable optimized marshaller (set 'marshaller' to " +
                 OptimizedMarshaller.class.getSimpleName() + ')');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/b0d9ef8a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index 9619c1c..0f79084 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -283,6 +283,12 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     }
 
     /** {@inheritDoc} */
+    @Override protected void onKernalStart0() throws IgniteCheckedException {
+        if (Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode())
+            ctx.performance().add("Enable client mode for TcpDiscoverySpi (set TcpDiscoverySpi.clientMode to true)");
+    }
+
+    /** {@inheritDoc} */
     @Override public void start() throws IgniteCheckedException {
         long totSysMemory = -1;
 


[23/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix Dr* tests

Posted by se...@apache.org.
# IGNITE-943 Fix Dr* tests


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

Branch: refs/heads/ignite-sprint-5
Commit: a43522baa9eb93c500c65f92c047850e8a6cd37d
Parents: 5cec2ba
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 15:50:04 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 15:50:04 2015 +0300

----------------------------------------------------------------------
 .../processors/affinity/GridAffinityProcessorAbstractSelfTest.java  | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a43522ba/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
index e7fab8e..7da3728 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
@@ -54,6 +54,7 @@ public abstract class GridAffinityProcessorAbstractSelfTest extends GridCommonAb
 
         TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
 
+        discoSpi.setForceServerMode(true);
         discoSpi.setIpFinder(ipFinder);
 
         cfg.setDiscoverySpi(discoSpi);


[24/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix tests

Posted by se...@apache.org.
# IGNITE-943 Fix tests


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

Branch: refs/heads/ignite-sprint-5
Commit: cc1612dda53cb566ba251a6d241914b4d22079ce
Parents: a43522b
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 16:02:16 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 16:02:16 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/internal/GridAffinitySelfTest.java     | 1 +
 .../processors/cache/IgniteCacheConfigurationTemplateTest.java    | 2 +-
 .../processors/cache/IgniteCacheNearLockValueSelfTest.java        | 3 +++
 .../processors/datastreamer/DataStreamProcessorSelfTest.java      | 1 +
 4 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cc1612dd/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
index 6735021..7bc8dd6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridAffinitySelfTest.java
@@ -47,6 +47,7 @@ public class GridAffinitySelfTest extends GridCommonAbstractTest {
 
         disco.setMaxMissedHeartbeats(Integer.MAX_VALUE);
         disco.setIpFinder(IP_FINDER);
+        disco.setForceServerMode(true);
 
         cfg.setDiscoverySpi(disco);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cc1612dd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
index a030039..937a3b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheConfigurationTemplateTest.java
@@ -59,7 +59,7 @@ public class IgniteCacheConfigurationTemplateTest extends GridCommonAbstractTest
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
 
         if (addTemplate) {
             CacheConfiguration dfltCfg = new CacheConfiguration();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cc1612dd/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
index 5cc9d04..cd04433 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheNearLockValueSelfTest.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.plugin.extensions.communication.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.apache.ignite.transactions.*;
 
@@ -54,6 +55,8 @@ public class IgniteCacheNearLockValueSelfTest extends GridCommonAbstractTest {
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setForceServerMode(true));
+
         if (getTestGridName(0).equals(gridName))
             cfg.setClientMode(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/cc1612dd/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
index 1f0319f..81751f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessorSelfTest.java
@@ -84,6 +84,7 @@ public class DataStreamProcessorSelfTest extends GridCommonAbstractTest {
 
         TcpDiscoverySpi spi = new TcpDiscoverySpi();
 
+        spi.setForceServerMode(true);
         spi.setIpFinder(ipFinder);
 
         cfg.setDiscoverySpi(spi);


[13/53] [abbrv] incubator-ignite git commit: Merge branches 'ignite-709_2' and 'ignite-943' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-943

Posted by se...@apache.org.
Merge branches 'ignite-709_2' and 'ignite-943' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into ignite-943


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

Branch: refs/heads/ignite-sprint-5
Commit: 5ecc08ef15a170593f755c2e2bdd18045c71a7aa
Parents: 982cb0c 9d3ab16
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed May 27 18:35:34 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed May 27 18:35:34 2015 +0300

----------------------------------------------------------------------
 dev-tools/slurp.sh                              |   14 +-
 dev-tools/src/main/groovy/jiraslurp.groovy      |  265 ++-
 examples/pom.xml                                |    2 +-
 modules/aop/pom.xml                             |    2 +-
 modules/aws/pom.xml                             |    2 +-
 modules/clients/pom.xml                         |    2 +-
 modules/cloud/pom.xml                           |    2 +-
 modules/codegen/pom.xml                         |    2 +-
 modules/core/pom.xml                            |    2 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../discovery/GridDiscoveryManager.java         |   73 +-
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../processors/cache/GridCacheAdapter.java      |    2 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../processors/cache/GridCacheContext.java      |    6 +-
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   67 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |    6 +-
 .../cache/GridCacheSharedContext.java           |    1 +
 .../processors/cache/GridCacheUtils.java        |  228 +--
 .../processors/cache/IgniteCacheProxy.java      |    3 +-
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../GridDistributedTxRemoteAdapter.java         |   10 +-
 .../dht/GridClientPartitionTopology.java        |    8 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   22 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 ++-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   78 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  207 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  305 ++-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   28 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 ++-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   77 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |   10 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   15 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../processors/query/GridQueryIndexing.java     |   16 +
 .../processors/query/GridQueryProcessor.java    |   52 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../core/src/main/resources/ignite.properties   |    2 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |    4 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../GridCacheAbstractRemoveFailureTest.java     |   20 +
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +++
 ...niteCacheClientNodeChangingTopologyTest.java | 1797 ++++++++++++++++++
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++++++
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   47 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   17 +-
 .../junits/common/GridCommonAbstractTest.java   |    4 +-
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../testsuites/IgniteCacheTestSuite2.java       |   10 +-
 modules/extdata/p2p/pom.xml                     |    2 +-
 modules/extdata/uri/pom.xml                     |    2 +-
 modules/gce/pom.xml                             |    2 +-
 modules/geospatial/pom.xml                      |    2 +-
 .../query/h2/GridH2IndexingGeoSelfTest.java     |   20 +-
 modules/hadoop/pom.xml                          |    2 +-
 modules/hibernate/pom.xml                       |    2 +-
 modules/indexing/pom.xml                        |    2 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   17 +
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |   18 +-
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |    4 +-
 .../query/h2/sql/GridSqlOperationType.java      |    2 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |    2 +-
 .../IgniteCacheQueryMultiThreadedSelfTest.java  |    2 +-
 .../local/IgniteCacheLocalQuerySelfTest.java    |    6 +
 modules/jcl/pom.xml                             |    2 +-
 modules/jta/pom.xml                             |    2 +-
 modules/log4j/pom.xml                           |    2 +-
 modules/rest-http/pom.xml                       |    2 +-
 modules/scalar/pom.xml                          |    2 +-
 modules/schedule/pom.xml                        |    2 +-
 modules/schema-import/pom.xml                   |    2 +-
 modules/slf4j/pom.xml                           |    2 +-
 modules/spring/pom.xml                          |    2 +-
 modules/ssh/pom.xml                             |    2 +-
 modules/tools/pom.xml                           |    2 +-
 modules/urideploy/pom.xml                       |    2 +-
 modules/visor-console/pom.xml                   |    2 +-
 .../visor/commands/node/VisorNodeCommand.scala  |    2 +-
 .../commands/tasks/VisorTasksCommand.scala      |    2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |   63 +-
 modules/visor-plugins/pom.xml                   |    2 +-
 modules/web/pom.xml                             |    2 +-
 modules/yardstick/pom.xml                       |    2 +-
 pom.xml                                         |   13 +-
 119 files changed, 4860 insertions(+), 1105 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5ecc08ef/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------


[07/53] [abbrv] incubator-ignite git commit: # ignite-23

Posted by se...@apache.org.
# ignite-23


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

Branch: refs/heads/ignite-sprint-5
Commit: 9188f4ed312d4a0289a3c2b63ebe478341be6424
Parents: cf6c420
Author: sboikov <sb...@gridgain.com>
Authored: Wed May 27 16:51:07 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed May 27 17:01:14 2015 +0300

----------------------------------------------------------------------
 ...niteCacheClientNodeChangingTopologyTest.java | 29 ++++++++++++++++++--
 .../junits/common/GridCommonAbstractTest.java   |  2 +-
 2 files changed, 27 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9188f4ed/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index e13b283..47c1d7e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.processors.cache.version.*;
+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.plugin.extensions.communication.*;
@@ -851,8 +852,8 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
         ccfg.setRebalanceMode(SYNC);
         ccfg.setNearConfiguration(nearCfg);
 
-        IgniteEx ignite0 = startGrid(0);
-        IgniteEx ignite1 = startGrid(1);
+        final IgniteEx ignite0 = startGrid(0);
+        final IgniteEx ignite1 = startGrid(1);
 
         client = true;
 
@@ -920,10 +921,32 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
 
         lockFut.get();
 
+        boolean wait = GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return unlocked(ignite0) && unlocked(ignite1);
+            }
+
+            private boolean unlocked(Ignite ignite) {
+                IgniteCache<Integer, Integer> cache = ignite.cache(null);
+
+                for (Integer key : keys) {
+                    if (cache.isLocalLocked(key, false)) {
+                        log.info("Key is locked [key=" + key + ", node=" + ignite.name() + ']');
+
+                        return false;
+                    }
+                }
+
+                return true;
+            }
+        }, 10_000);
+
+        assertTrue(wait);
+
         for (Integer key : keys) {
             Lock lock = cache0.lock(key);
 
-            assertTrue(lock.tryLock());
+            assertTrue("Failed to lock: " + key, lock.tryLock());
 
             lock.unlock();
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9188f4ed/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index a4d3355..1f4c7b6 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -391,7 +391,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                 if (i == 0)
                                     start = System.currentTimeMillis();
 
-                                if (i >= 50)
+                                if (System.currentTimeMillis() - start > 30_000)
                                     throw new IgniteException("Timeout of waiting for topology map update [grid="
                                         + g.name() + ", p=" + p + ", nodes=" + exp + ", owners=" + actual +
                                             ", affNodes=" + affNodes + ", owners=" + owners + ", locNode="


[53/53] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-943' into ignite-sprint-5

Posted by se...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-943' into ignite-sprint-5


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

Branch: refs/heads/ignite-sprint-5
Commit: 3c0046e8e040ab53a5f39819b1c86487f645d25f
Parents: 9a16d19 ab4c18c
Author: sevdokimov <se...@gridgain.com>
Authored: Mon Jun 1 14:42:34 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Mon Jun 1 14:42:34 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |   18 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |   24 +-
 .../configuration/CacheConfiguration.java       |   27 +-
 .../configuration/IgniteConfiguration.java      |    4 +-
 .../ignite/internal/ClusterMetricsSnapshot.java |   14 +
 .../internal/GridEventConsumeHandler.java       |  100 +-
 .../ignite/internal/GridKernalContext.java      |    5 +
 .../ignite/internal/GridKernalContextImpl.java  |    5 +
 .../apache/ignite/internal/IgniteKernal.java    |   23 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   20 +-
 .../ignite/internal/MarshallerContextImpl.java  |    2 +-
 .../internal/events/DiscoveryCustomEvent.java   |   18 +-
 .../managers/discovery/CustomEventListener.java |   31 +
 .../discovery/CustomMessageWrapper.java         |   63 +
 .../discovery/DiscoveryCustomMessage.java       |   48 +
 .../discovery/GridDiscoveryManager.java         |  214 +-
 .../affinity/GridAffinityAssignmentCache.java   |   32 +
 .../cache/DynamicCacheChangeBatch.java          |   20 +-
 .../cache/DynamicCacheDescriptor.java           |    2 +
 .../processors/cache/GridCacheAdapter.java      |   17 +-
 .../cache/GridCacheAffinityManager.java         |   14 +
 .../cache/GridCacheConcurrentMap.java           |   21 +-
 .../processors/cache/GridCacheContext.java      |    6 +-
 .../processors/cache/GridCacheGateway.java      |    2 +-
 .../processors/cache/GridCacheIoManager.java    |    8 +-
 .../processors/cache/GridCacheMvccManager.java  |   32 +-
 .../GridCachePartitionExchangeManager.java      |   71 +-
 .../processors/cache/GridCachePreloader.java    |    6 +-
 .../cache/GridCachePreloaderAdapter.java        |   11 +-
 .../processors/cache/GridCacheProcessor.java    |   74 +-
 .../processors/cache/GridCacheProxyImpl.java    |   12 +
 .../cache/GridCacheSharedContext.java           |    1 +
 .../processors/cache/GridCacheUtils.java        |  234 +-
 .../processors/cache/IgniteInternalCache.java   |    5 +
 .../cache/affinity/GridCacheAffinityImpl.java   |   10 +-
 .../CacheDataStructuresManager.java             |    2 +-
 .../distributed/GridDistributedTxMapping.java   |   17 +
 .../dht/GridClientPartitionTopology.java        |    8 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   27 +-
 .../distributed/dht/GridDhtLockFuture.java      |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |   22 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  224 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |    8 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |    3 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   18 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |   10 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   78 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |  112 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   12 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  213 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   26 +-
 .../dht/preloader/GridDhtPartitionMap.java      |    2 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |   29 +-
 .../GridDhtPartitionsExchangeFuture.java        |  313 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    4 +-
 .../GridDhtPartitionsSingleMessage.java         |   33 +-
 .../dht/preloader/GridDhtPreloader.java         |   28 +-
 .../preloader/GridDhtPreloaderAssignments.java  |    3 +-
 .../distributed/near/GridNearAtomicCache.java   |    5 +
 .../distributed/near/GridNearCacheAdapter.java  |    2 +-
 .../distributed/near/GridNearGetFuture.java     |    2 +-
 .../distributed/near/GridNearLockFuture.java    |  271 +-
 .../distributed/near/GridNearLockRequest.java   |   68 +-
 .../distributed/near/GridNearLockResponse.java  |   48 +-
 .../near/GridNearOptimisticTxPrepareFuture.java |   83 +-
 .../GridNearPessimisticTxPrepareFuture.java     |    5 +-
 .../near/GridNearTransactionalCache.java        |    4 +-
 .../cache/distributed/near/GridNearTxLocal.java |   43 +-
 .../near/GridNearTxPrepareRequest.java          |   72 +-
 .../near/GridNearTxPrepareResponse.java         |   70 +-
 .../processors/cache/local/GridLocalCache.java  |    6 +-
 .../local/atomic/GridLocalAtomicCache.java      |    6 +-
 .../cache/query/GridCacheQueryAdapter.java      |   12 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    2 +
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../continuous/CacheContinuousQueryManager.java |   28 +-
 .../cache/transactions/IgniteInternalTx.java    |    5 +
 .../cache/transactions/IgniteTxAdapter.java     |   15 +-
 .../cache/transactions/IgniteTxHandler.java     |  148 +-
 .../cacheobject/IgniteCacheObjectProcessor.java |    5 +-
 .../IgniteCacheObjectProcessorImpl.java         |    2 +-
 .../continuous/AbstractContinuousMessage.java   |   54 +
 .../continuous/GridContinuousMessageType.java   |   12 -
 .../continuous/GridContinuousProcessor.java     |  836 +--
 .../processors/continuous/StartRequestData.java |  267 +
 .../StartRoutineAckDiscoveryMessage.java        |   63 +
 .../StartRoutineDiscoveryMessage.java           |   85 +
 .../StopRoutineAckDiscoveryMessage.java         |   49 +
 .../continuous/StopRoutineDiscoveryMessage.java |   49 +
 .../service/GridServiceProcessor.java           |  121 +-
 .../internal/util/future/GridFutureAdapter.java |    4 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |    2 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   20 +-
 .../discovery/DiscoverySpiCustomMessage.java    |   40 +
 .../spi/discovery/DiscoverySpiListener.java     |    5 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1478 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4766 ++++++++++++++
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1264 ----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  164 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  170 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5799 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1160 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    9 +
 .../tcp/internal/TcpDiscoveryNode.java          |    7 +-
 .../tcp/internal/TcpDiscoveryNodesRing.java     |    2 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   10 +-
 .../messages/TcpDiscoveryAbstractMessage.java   |   24 +-
 .../TcpDiscoveryClientHeartbeatMessage.java     |   67 +
 .../messages/TcpDiscoveryClientPingRequest.java |   56 +
 .../TcpDiscoveryClientPingResponse.java         |   67 +
 .../TcpDiscoveryCustomEventMessage.java         |   41 +-
 .../messages/TcpDiscoveryHeartbeatMessage.java  |   28 +-
 .../TcpDiscoveryNodeAddFinishedMessage.java     |   43 +
 .../messages/TcpDiscoveryNodeAddedMessage.java  |    2 +-
 .../tcp/messages/TcpDiscoveryPingRequest.java   |    6 +
 .../tcp/messages/TcpDiscoveryPingResponse.java  |   15 +-
 .../affinity/IgniteClientNodeAffinityTest.java  |  182 +
 .../ignite/internal/GridAffinitySelfTest.java   |    1 +
 .../internal/GridDiscoveryEventSelfTest.java    |    7 +-
 ...ridFailFastNodeFailureDetectionSelfTest.java |    7 +-
 .../internal/GridProjectionAbstractTest.java    |   16 +
 .../GridProjectionForCachesSelfTest.java        |   11 +-
 .../internal/GridReleaseTypeSelfTest.java       |   77 +-
 .../apache/ignite/internal/GridSelfTest.java    |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   62 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |  122 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   46 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    1 +
 .../cache/CacheRemoveAllSelfTest.java           |    2 +-
 .../GridCacheAbstractFailoverSelfTest.java      |    2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    3 +
 .../GridCacheAbstractRemoveFailureTest.java     |   23 +
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../GridCacheAtomicMessageCountSelfTest.java    |    1 +
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |    1 +
 .../GridCacheReturnValueTransferSelfTest.java   |    3 +
 ...acheTcpClientDiscoveryMultiThreadedTest.java |  190 +
 .../GridCacheVariableTopologySelfTest.java      |   12 +-
 .../IgniteCacheAbstractStopBusySelfTest.java    |    6 +-
 .../cache/IgniteCacheAbstractTest.java          |    2 +-
 .../IgniteCacheConfigurationTemplateTest.java   |    2 +-
 .../cache/IgniteCacheNearLockValueSelfTest.java |    3 +
 .../IgniteCacheP2pUnmarshallingErrorTest.java   |   29 +-
 ...gniteCacheP2pUnmarshallingNearErrorTest.java |   13 +-
 .../IgniteCachePartitionMapUpdateTest.java      |  226 +
 .../IgniteDynamicClientCacheStartSelfTest.java  |  283 +
 .../cache/IgniteSystemCacheOnClientTest.java    |   97 +
 .../GridCacheQueueApiSelfAbstractTest.java      |    4 +-
 .../IgniteClientDataStructuresAbstractTest.java |  283 +
 .../IgniteClientDataStructuresTest.java         |   28 +
 ...IgniteClientDiscoveryDataStructuresTest.java |   28 +
 .../GridCacheClientModesAbstractSelfTest.java   |   94 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |  168 +
 .../distributed/GridCacheMixedModeSelfTest.java |    3 +
 ...niteCacheClientNodeChangingTopologyTest.java | 1803 ++++++
 .../IgniteCacheClientNodeConcurrentStart.java   |  105 +
 ...teCacheClientNodePartitionsExchangeTest.java |  632 ++
 .../dht/GridCacheClientOnlySelfTest.java        |   60 +-
 .../GridCacheDhtClientRemoveFailureTest.java    |   28 +
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |   47 +-
 ...cClientInvalidPartitionHandlingSelfTest.java |   29 +
 .../GridCacheAtomicClientRemoveFailureTest.java |   28 +
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   23 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    3 +-
 .../near/GridCacheAtomicNearOnlySelfTest.java   |   32 -
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    2 +
 .../near/GridCacheNearOnlySelfTest.java         |   63 +-
 .../near/GridCacheNearOnlyTopologySelfTest.java |    1 +
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    5 +-
 ...idCacheRendezvousAffinityClientSelfTest.java |    4 +
 .../GridCacheReplicatedClientOnlySelfTest.java  |   43 -
 .../GridCacheReplicatedNearOnlySelfTest.java    |   43 -
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 ...heNearOnlyLruNearEvictionPolicySelfTest.java |   25 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |   93 +-
 .../DataStreamProcessorSelfTest.java            |    1 +
 .../igfs/IgfsClientCacheSelfTest.java           |    3 +-
 .../processors/igfs/IgfsCommonAbstractTest.java |   10 -
 .../processors/igfs/IgfsOneClientNodeTest.java  |    8 +-
 .../service/ClosureServiceClientsNodesTest.java |   16 +-
 .../service/GridServiceClientNodeTest.java      |   81 +
 .../OptimizedMarshallerNodeFailoverTest.java    |    4 +-
 ...GridMessagingNoPeerClassLoadingSelfTest.java |    7 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   13 +-
 .../discovery/AbstractDiscoverySelfTest.java    |    8 +-
 ...pClientDiscoveryMarshallerCheckSelfTest.java |   76 +
 .../tcp/TcpClientDiscoverySelfTest.java         |  700 ---
 .../tcp/TcpClientDiscoverySpiSelfTest.java      | 1171 ++++
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |   61 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |   18 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../ignite/testframework/GridTestUtils.java     |   15 +
 .../testframework/junits/GridAbstractTest.java  |   52 +-
 .../junits/common/GridCommonAbstractTest.java   |   59 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +-
 .../IgniteCacheDataStructuresSelfTestSuite.java |    3 +
 .../IgniteCacheFailoverTestSuite.java           |    4 +-
 .../IgniteCacheNearOnlySelfTestSuite.java       |   16 +-
 ...gniteCacheP2pUnmarshallingErrorTestSuit.java |   41 -
 ...niteCacheP2pUnmarshallingErrorTestSuite.java |   41 +
 .../IgniteCacheTcpClientDiscoveryTestSuite.java |   47 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |    4 +
 .../testsuites/IgniteCacheTestSuite2.java       |   11 +-
 .../testsuites/IgniteCacheTestSuite4.java       |    2 +
 .../testsuites/IgniteKernalSelfTestSuite.java   |    7 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    3 +-
 .../igfs/IgfsNearOnlyMultiNodeSelfTest.java     |    5 +-
 .../cache/IgniteCacheAbstractQuerySelfTest.java |    6 +-
 ...niteCacheP2pUnmarshallingQueryErrorTest.java |    3 +-
 214 files changed, 17497 insertions(+), 10220 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/3c0046e8/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------


[26/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Remove SocketMultiConnector

Posted by se...@apache.org.
# IGNITE-943 Remove SocketMultiConnector


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

Branch: refs/heads/ignite-sprint-5
Commit: 6adc9743d4912ad59a50832f4869e8e41f5f9a04
Parents: 154bd9e
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 17:03:39 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 17:03:39 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  21 +--
 .../spi/discovery/tcp/SocketMultiConnector.java | 144 -------------------
 2 files changed, 13 insertions(+), 152 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6adc9743/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 a966363..59e25fc 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
@@ -810,17 +810,22 @@ class ServerImpl extends TcpDiscoveryImpl {
             boolean retry = false;
             Collection<Exception> errs = new ArrayList<>();
 
-            try (SocketMultiConnector multiConnector = new SocketMultiConnector(spi, addrs, 2)) {
-                GridTuple3<InetSocketAddress, Socket, Exception> tuple;
+            for (int j = 2; --j >= 0;) {
+                for (InetSocketAddress addr : addrs) {
+                    Socket sock = null;
+                    Exception ex = null;
 
-                while ((tuple = multiConnector.next()) != null) {
-                    InetSocketAddress addr = tuple.get1();
-                    Socket sock = tuple.get2();
-                    Exception ex = tuple.get3();
+                    try {
+                        sock = spi.openSocket(addr);
+                    }
+                    catch (Exception e) {
+                        if (j > 0)
+                            continue;
 
-                    if (ex == null) {
-                        assert sock != null;
+                        ex = e;
+                    }
 
+                    if (ex == null) {
                         try {
                             Integer res = sendMessageDirectly(joinReq, addr, sock);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6adc9743/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/SocketMultiConnector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/SocketMultiConnector.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/SocketMultiConnector.java
deleted file mode 100644
index 698735e..0000000
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/SocketMultiConnector.java
+++ /dev/null
@@ -1,144 +0,0 @@
-/*
- * 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.internal.util.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.spi.*;
-import org.jetbrains.annotations.*;
-
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Allow to connect to addresses parallel.
- */
-class SocketMultiConnector implements AutoCloseable {
-    /** */
-    private int connInProgress;
-
-    /** */
-    private final ExecutorService executor;
-
-    /** */
-    private final CompletionService<GridTuple3<InetSocketAddress, Socket, Exception>> completionSrvc;
-
-    /**
-     * @param spi Discovery SPI.
-     * @param addrs Addresses.
-     * @param retryCnt Retry count.
-     */
-    SocketMultiConnector(final TcpDiscoverySpi spi, Collection<InetSocketAddress> addrs,
-        final int retryCnt) {
-        connInProgress = addrs.size();
-
-        executor = Executors.newFixedThreadPool(Math.min(1, addrs.size()));
-
-        completionSrvc = new ExecutorCompletionService<>(executor);
-
-        for (final InetSocketAddress addr : addrs) {
-            completionSrvc.submit(new Callable<GridTuple3<InetSocketAddress, Socket, Exception>>() {
-                @Override public GridTuple3<InetSocketAddress, Socket, Exception> call() {
-                    Exception ex = null;
-                    Socket sock = null;
-
-                    for (int i = 0; i < retryCnt; i++) {
-                        if (Thread.currentThread().isInterrupted())
-                            return null; // Executor is shutdown.
-
-                        try {
-                            sock = spi.openSocket(addr);
-
-                            break;
-                        }
-                        catch (Exception e) {
-                            ex = e;
-                        }
-                    }
-
-                    return new GridTuple3<>(addr, sock, ex);
-                }
-            });
-        }
-    }
-
-    /**
-     *
-     */
-    @Nullable public GridTuple3<InetSocketAddress, Socket, Exception> next() {
-        if (connInProgress == 0)
-            return null;
-
-        try {
-            Future<GridTuple3<InetSocketAddress, Socket, Exception>> fut = completionSrvc.take();
-
-            connInProgress--;
-
-            return fut.get();
-        }
-        catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-
-            throw new IgniteSpiException("Thread has been interrupted.", e);
-        }
-        catch (ExecutionException e) {
-            throw new IgniteSpiException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close() {
-        List<Runnable> unstartedTasks = executor.shutdownNow();
-
-        connInProgress -= unstartedTasks.size();
-
-        if (connInProgress > 0) {
-            Thread thread = new Thread(new Runnable() {
-                @Override public void run() {
-                    try {
-                        executor.awaitTermination(5, TimeUnit.MINUTES);
-
-                        Future<GridTuple3<InetSocketAddress, Socket, Exception>> fut;
-
-                        while ((fut = completionSrvc.poll()) != null) {
-                            try {
-                                GridTuple3<InetSocketAddress, Socket, Exception> tuple3 = fut.get();
-
-                                if (tuple3 != null)
-                                    IgniteUtils.closeQuiet(tuple3.get2());
-                            }
-                            catch (ExecutionException ignore) {
-
-                            }
-                        }
-                    }
-                    catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-
-                        throw new RuntimeException(e);
-                    }
-                }
-            });
-
-            thread.setDaemon(true);
-
-            thread.start();
-        }
-    }
-}


[29/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix tests

Posted by se...@apache.org.
# IGNITE-943 Fix  tests


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

Branch: refs/heads/ignite-sprint-5
Commit: 0e7fc0a2a1b5f81a4fdc7012d9ee47cdd1c2c912
Parents: a02cb41
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 17:13:10 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 17:13:10 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAbstractRemoveFailureTest.java   | 3 +++
 .../GridCacheAtomicInvalidPartitionHandlingSelfTest.java       | 6 +-----
 2 files changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0e7fc0a2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
index d5d80ab..2b6a6b0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.internal.*;
 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.spi.discovery.tcp.*;
 import org.apache.ignite.testframework.*;
 import org.jsr166.*;
 
@@ -75,6 +76,8 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
+
         if (testClientNode() && getTestGridName(0).equals(gridName))
             cfg.setClientMode(true);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0e7fc0a2/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
index 55125a2..054a110 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
@@ -68,11 +68,7 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(IP_FINDER);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(IP_FINDER).setForceServerMode(true));
 
         cfg.setCacheConfiguration(cacheConfiguration());
 


[49/53] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-943

Posted by se...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-943


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

Branch: refs/heads/ignite-sprint-5
Commit: d10120d67d3d6e20b25e494b03b829b20f263b5b
Parents: d10fe3e 5c30f9c
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 18:57:20 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 18:57:20 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/cache/CacheMetrics.java   | 187 +++++++--
 .../org/apache/ignite/igfs/IgfsUserContext.java | 119 ++++++
 .../igfs/secondary/IgfsSecondaryFileSystem.java |   7 +
 .../internal/igfs/common/IgfsMarshaller.java    |  35 +-
 .../igfs/common/IgfsPathControlRequest.java     |  22 +
 .../internal/managers/GridManagerAdapter.java   |  59 +--
 .../processors/cache/CacheMetricsImpl.java      | 367 ++++++++++++++++-
 .../cache/CacheMetricsMXBeanImpl.java           | 100 +++++
 .../processors/cache/CacheMetricsSnapshot.java  | 380 +++++++++++++----
 .../processors/cache/GridCacheAdapter.java      |  12 +-
 .../processors/cache/GridCacheSwapManager.java  | 118 ++++--
 .../internal/processors/hadoop/HadoopJob.java   |   2 +-
 .../ignite/internal/processors/igfs/IgfsEx.java |   8 +-
 .../internal/processors/igfs/IgfsImpl.java      |   8 +-
 .../processors/igfs/IgfsIpcHandler.java         | 184 +++++----
 .../igfs/IgfsSecondaryFileSystemImpl.java       |   9 +-
 .../internal/processors/igfs/IgfsServer.java    |   4 +-
 .../internal/processors/igfs/IgfsUtils.java     |  16 +
 .../ignite/internal/util/GridJavaProcess.java   |  30 +-
 .../ignite/mxbean/CacheMetricsMXBean.java       |  80 ++++
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |  35 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |  47 ---
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   8 +-
 ...CacheLocalOffHeapAndSwapMetricsSelfTest.java | 412 +++++++++++++++++++
 .../testframework/GridSpiTestContext.java       |  25 +-
 .../IgniteCacheMetricsSelfTestSuite.java        |   1 +
 .../fs/IgniteHadoopIgfsSecondaryFileSystem.java | 165 +++++---
 .../hadoop/fs/v1/IgniteHadoopFileSystem.java    | 107 +++--
 .../hadoop/fs/v2/IgniteHadoopFileSystem.java    |  32 +-
 .../internal/processors/hadoop/HadoopUtils.java |  10 +-
 .../hadoop/SecondaryFileSystemProvider.java     |  53 ++-
 .../hadoop/fs/HadoopDistributedFileSystem.java  |  91 ----
 .../hadoop/fs/HadoopFileSystemsUtils.java       |  17 -
 .../hadoop/fs/HadoopLazyConcurrentMap.java      | 204 +++++++++
 .../processors/hadoop/igfs/HadoopIgfsEx.java    |   6 +
 .../hadoop/igfs/HadoopIgfsInProc.java           | 170 ++++++--
 .../processors/hadoop/igfs/HadoopIgfsIpcIo.java |   2 +-
 .../hadoop/igfs/HadoopIgfsOutProc.java          |  33 +-
 .../hadoop/igfs/HadoopIgfsWrapper.java          |  19 +-
 .../hadoop/v2/HadoopV2TaskContext.java          |   4 +-
 .../HadoopIgfs20FileSystemAbstractSelfTest.java |  56 ++-
 ...oopSecondaryFileSystemConfigurationTest.java |   4 +-
 .../IgniteHadoopFileSystemAbstractSelfTest.java |  63 ++-
 .../IgniteHadoopFileSystemClientSelfTest.java   |   2 +-
 .../IgniteHadoopFileSystemIpcCacheSelfTest.java |   2 +
 .../hadoop/HadoopFileSystemsTest.java           |  23 +-
 .../collections/HadoopSkipListSelfTest.java     |   4 +-
 47 files changed, 2537 insertions(+), 805 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d10120d6/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d10120d6/modules/core/src/main/java/org/apache/ignite/spi/IgniteSpiAdapter.java
----------------------------------------------------------------------


[02/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Merge TcpDiscoverySpi and TcpClientDiscoverySpi

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/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 8ceac1c..2b2c691 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
@@ -18,22 +18,17 @@
 package org.apache.ignite.spi.discovery.tcp;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.events.*;
-import org.apache.ignite.internal.processors.security.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.io.*;
-import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.plugin.security.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.discovery.*;
@@ -45,22 +40,13 @@ import org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.spi.discovery.tcp.messages.*;
 import org.jetbrains.annotations.*;
-import org.jsr166.*;
 
 import java.io.*;
 import java.net.*;
-import java.text.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.internal.IgniteNodeAttributes.*;
-import static org.apache.ignite.spi.IgnitePortProtocol.*;
-import static org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoverySpiState.*;
-import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHeartbeatMessage.*;
-import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage.*;
-
 /**
  * Discovery SPI implementation that uses TCP/IP for node discovery.
  * <p>
@@ -150,10 +136,43 @@ import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusChe
 @IgniteSpiMultipleInstancesSupport(true)
 @DiscoverySpiOrderSupport(true)
 @DiscoverySpiHistorySupport(true)
-public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscoverySpiMBean {
+public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, TcpDiscoverySpiMBean {
+    /** Node attribute that is mapped to node's external addresses (value is <tt>disc.tcp.ext-addrs</tt>). */
+    public static final String ATTR_EXT_ADDRS = "disc.tcp.ext-addrs";
+
     /** Default local port range (value is <tt>100</tt>). */
     public static final int DFLT_PORT_RANGE = 100;
 
+    /** Default port to listen (value is <tt>47500</tt>). */
+    public static final int DFLT_PORT = 47500;
+
+    /** Default timeout for joining topology (value is <tt>0</tt>). */
+    public static final long DFLT_JOIN_TIMEOUT = 0;
+
+    /** Default network timeout in milliseconds (value is <tt>5000ms</tt>). */
+    public static final long DFLT_NETWORK_TIMEOUT = 5000;
+
+    /** Default value for thread priority (value is <tt>10</tt>). */
+    public static final int DFLT_THREAD_PRI = 10;
+
+    /** Default heartbeat messages issuing frequency (value is <tt>100ms</tt>). */
+    public static final long DFLT_HEARTBEAT_FREQ = 100;
+
+    /** Default size of topology snapshots history. */
+    public static final int DFLT_TOP_HISTORY_SIZE = 1000;
+
+    /** Default socket operations timeout in milliseconds (value is <tt>200ms</tt>). */
+    public static final long DFLT_SOCK_TIMEOUT = 200;
+
+    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>50ms</tt>). */
+    public static final long DFLT_ACK_TIMEOUT = 50;
+
+    /** Default socket operations timeout in milliseconds (value is <tt>700ms</tt>). */
+    public static final long DFLT_SOCK_TIMEOUT_CLIENT = 700;
+
+    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>700ms</tt>). */
+    public static final long DFLT_ACK_TIMEOUT_CLIENT = 700;
+
     /** Default reconnect attempts count (value is <tt>10</tt>). */
     public static final int DFLT_RECONNECT_CNT = 10;
 
@@ -172,154 +191,236 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
     /** Maximum ack timeout value for receiving message acknowledgement in milliseconds (value is <tt>600,000ms</tt>). */
     public static final long DFLT_MAX_ACK_TIMEOUT = 10 * 60 * 1000;
 
-    /** Default socket operations timeout in milliseconds (value is <tt>200ms</tt>). */
-    public static final long DFLT_SOCK_TIMEOUT = 200;
-
-    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>50ms</tt>). */
-    public static final long DFLT_ACK_TIMEOUT = 50;
-
-    /** Node attribute that is mapped to node's external addresses (value is <tt>disc.tcp.ext-addrs</tt>). */
-    public static final String ATTR_EXT_ADDRS = "disc.tcp.ext-addrs";
+    /** Local address. */
+    protected String locAddr;
 
     /** Address resolver. */
     private AddressResolver addrRslvr;
 
-    /** Local port which node uses. */
-    private int locPort = DFLT_PORT;
+    /** IP finder. */
+    protected TcpDiscoveryIpFinder ipFinder;
 
-    /** Local port range. */
-    private int locPortRange = DFLT_PORT_RANGE;
+    /** Socket operations timeout. */
+    protected long sockTimeout; // Must be initialized in the constructor of child class.
 
-    /** Statistics print frequency. */
-    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized", "RedundantFieldInitialization"})
-    private long statsPrintFreq = DFLT_STATS_PRINT_FREQ;
+    /** Message acknowledgement timeout. */
+    protected long ackTimeout; // Must be initialized in the constructor of child class.
 
-    /** Maximum message acknowledgement timeout. */
-    private long maxAckTimeout = DFLT_MAX_ACK_TIMEOUT;
+    /** Network timeout. */
+    protected long netTimeout = DFLT_NETWORK_TIMEOUT;
 
-    /** Max heartbeats count node can miss without initiating status check. */
-    private int maxMissedHbs = DFLT_MAX_MISSED_HEARTBEATS;
+    /** Join timeout. */
+    @SuppressWarnings("RedundantFieldInitialization")
+    protected long joinTimeout = DFLT_JOIN_TIMEOUT;
 
-    /** Max heartbeats count node can miss without failing client node. */
-    private int maxMissedClientHbs = DFLT_MAX_MISSED_CLIENT_HEARTBEATS;
+    /** Thread priority for all threads started by SPI. */
+    protected int threadPri = DFLT_THREAD_PRI;
 
-    /** IP finder clean frequency. */
-    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
-    private long ipFinderCleanFreq = DFLT_IP_FINDER_CLEAN_FREQ;
+    /** Heartbeat messages issuing frequency. */
+    protected long hbFreq = DFLT_HEARTBEAT_FREQ;
 
-    /** Reconnect attempts count. */
-    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
-    private int reconCnt = DFLT_RECONNECT_CNT;
+    /** Size of topology snapshots history. */
+    protected int topHistSize = DFLT_TOP_HISTORY_SIZE;
 
-    /** */
-    private final Executor utilityPool = new ThreadPoolExecutor(0, 1, 2000, TimeUnit.MILLISECONDS,
-        new LinkedBlockingQueue<Runnable>());
+    /** Grid discovery listener. */
+    protected volatile DiscoverySpiListener lsnr;
 
-    /** Nodes ring. */
-    @GridToStringExclude
-    private final TcpDiscoveryNodesRing ring = new TcpDiscoveryNodesRing();
+    /** Data exchange. */
+    protected DiscoverySpiDataExchange exchange;
+
+    /** Metrics provider. */
+    protected DiscoveryMetricsProvider metricsProvider;
 
-    /** Topology snapshots history. */
-    private final SortedMap<Long, Collection<ClusterNode>> topHist = new TreeMap<>();
+    /** Local node attributes. */
+    protected Map<String, Object> locNodeAttrs;
 
-    /** Socket readers. */
-    private final Collection<SocketReader> readers = new LinkedList<>();
+    /** Local node version. */
+    protected IgniteProductVersion locNodeVer;
 
-    /** TCP server for discovery SPI. */
-    private TcpServer tcpSrvr;
+    /** Local node. */
+    protected TcpDiscoveryNode locNode;
 
-    /** Message worker. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private RingMessageWorker msgWorker;
+    /** Local host. */
+    protected InetAddress locHost;
 
-    /** Client message workers. */
-    private ConcurrentMap<UUID, ClientMessageWorker> clientMsgWorkers = new ConcurrentHashMap8<>();
+    /** Internal and external addresses of local node. */
+    protected Collection<InetSocketAddress> locNodeAddrs;
 
-    /** Metrics sender. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private HeartbeatsSender hbsSnd;
+    /** Socket timeout worker. */
+    protected SocketTimeoutWorker sockTimeoutWorker;
 
-    /** Status checker. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private CheckStatusSender chkStatusSnd;
+    /** Start time of the very first grid node. */
+    protected volatile long gridStartTime;
 
-    /** IP finder cleaner. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private IpFinderCleaner ipFinderCleaner;
+    /** Marshaller. */
+    protected final Marshaller marsh = new JdkMarshaller();
 
-    /** Statistics printer thread. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private StatisticsPrinter statsPrinter;
+    /** Statistics. */
+    protected final TcpDiscoveryStatistics stats = new TcpDiscoveryStatistics();
 
-    /** Failed nodes (but still in topology). */
-    private Collection<TcpDiscoveryNode> failedNodes = new HashSet<>();
+    /** Local port which node uses. */
+    protected int locPort = DFLT_PORT;
+
+    /** Local port range. */
+    protected int locPortRange = DFLT_PORT_RANGE;
+
+    /** Reconnect attempts count. */
+    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
+    protected int reconCnt = DFLT_RECONNECT_CNT;
 
-    /** Leaving nodes (but still in topology). */
-    private Collection<TcpDiscoveryNode> leavingNodes = new HashSet<>();
+    /** Statistics print frequency. */
+    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized", "RedundantFieldInitialization"})
+    protected long statsPrintFreq = DFLT_STATS_PRINT_FREQ;
 
-    /** If non-shared IP finder is used this flag shows whether IP finder contains local address. */
-    private boolean ipFinderHasLocAddr;
+    /** Maximum message acknowledgement timeout. */
+    protected long maxAckTimeout = DFLT_MAX_ACK_TIMEOUT;
 
-    /** Addresses that do not respond during join requests send (for resolving concurrent start). */
-    private final Collection<SocketAddress> noResAddrs = new GridConcurrentHashSet<>();
+    /** Max heartbeats count node can miss without initiating status check. */
+    protected int maxMissedHbs = DFLT_MAX_MISSED_HEARTBEATS;
 
-    /** Addresses that incoming join requests send were send from (for resolving concurrent start). */
-    private final Collection<SocketAddress> fromAddrs = new GridConcurrentHashSet<>();
+    /** Max heartbeats count node can miss without failing client node. */
+    protected int maxMissedClientHbs = DFLT_MAX_MISSED_CLIENT_HEARTBEATS;
 
-    /** Response on join request from coordinator (in case of duplicate ID or auth failure). */
-    private final GridTuple<TcpDiscoveryAbstractMessage> joinRes = F.t1();
+    /** IP finder clean frequency. */
+    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
+    protected long ipFinderCleanFreq = DFLT_IP_FINDER_CLEAN_FREQ;
 
     /** Context initialization latch. */
     @GridToStringExclude
     private final CountDownLatch ctxInitLatch = new CountDownLatch(1);
 
-    /** Node authenticator. */
-    private DiscoverySpiNodeAuthenticator nodeAuth;
+    /** */
+    protected final CopyOnWriteArrayList<IgniteInClosure<TcpDiscoveryAbstractMessage>> sendMsgLsnrs =
+        new CopyOnWriteArrayList<>();
+
+    /** */
+    protected final CopyOnWriteArrayList<IgniteInClosure<Socket>> incomeConnLsnrs =
+        new CopyOnWriteArrayList<>();
+
+    /** Logger. */
+    @LoggerResource
+    protected IgniteLogger log;
+
+    /** */
+    protected TcpDiscoveryImpl impl;
+
+    /** */
+    private boolean clientMode;
+
+    /** {@inheritDoc} */
+    @Override public String getSpiState() {
+        return impl.getSpiState();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMessageWorkerQueueSize() {
+        return impl.getMessageWorkerQueueSize();
+    }
 
-    /** Mutex. */
-    private final Object mux = new Object();
+    /** {@inheritDoc} */
+    @Nullable @Override public UUID getCoordinator() {
+        return impl.getCoordinator();
+    }
 
-    /** Discovery state. */
-    protected TcpDiscoverySpiState spiState = DISCONNECTED;
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> getRemoteNodes() {
+        return impl.getRemoteNodes();
+    }
 
-    /** Map with proceeding ping requests. */
-    private final ConcurrentMap<InetSocketAddress, IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>>> pingMap =
-        new ConcurrentHashMap8<>();
+    /** {@inheritDoc} */
+    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
+        return impl.getNode(nodeId);
+    }
 
-    /** Debug mode. */
-    private boolean debugMode;
+    /** {@inheritDoc} */
+    @Override public boolean pingNode(UUID nodeId) {
+        return impl.pingNode(nodeId);
+    }
 
-    /** Debug messages history. */
-    private int debugMsgHist = 512;
+    /** {@inheritDoc} */
+    @Override public void disconnect() throws IgniteSpiException {
+        impl.disconnect();
+    }
 
-    /** Received messages. */
-    @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-    private ConcurrentLinkedDeque<String> debugLog;
+    /** {@inheritDoc} */
+    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
+        impl.setAuthenticator(auth);
+    }
 
-    /** */
-    private final CopyOnWriteArrayList<IgniteInClosure<TcpDiscoveryAbstractMessage>> sendMsgLsnrs =
-        new CopyOnWriteArrayList<>();
+    /** {@inheritDoc} */
+    @Override public void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException {
+        impl.sendCustomEvent(msg);
+    }
 
-    /** */
-    private final CopyOnWriteArrayList<IgniteInClosure<Socket>> incomeConnLsnrs =
-        new CopyOnWriteArrayList<>();
+    /** {@inheritDoc} */
+    @Override public void failNode(UUID nodeId) {
+        impl.failNode(nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void dumpDebugInfo() {
+        impl.dumpDebugInfo(log);
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isClientMode() {
+        return clientMode;
+    }
 
     /**
-     * Default constructor.
+     * @param clientMode New client mode.
      */
-    public TcpDiscoverySpi() {
-        ackTimeout = DFLT_ACK_TIMEOUT;
-        sockTimeout = DFLT_SOCK_TIMEOUT;
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setClientMode(boolean clientMode) {
+        if (impl != null)
+            throw new IllegalStateException("You cannot change mode, TcpDiscoverySpi already started.");
+
+        this.clientMode = clientMode;
+
+        return this;
     }
 
-    /** {@inheritDoc} */
+    /**
+     * Inject resources
+     *
+     * @param ignite Ignite.
+     */
     @IgniteInstanceResource
-    @Override public void injectResources(Ignite ignite) {
+    @Override protected void injectResources(Ignite ignite) {
         super.injectResources(ignite);
 
         // Inject resource.
-        if (ignite != null)
+        if (ignite != null) {
+            setLocalAddress(ignite.configuration().getLocalHost());
             setAddressResolver(ignite.configuration().getAddressResolver());
+        }
+    }
+
+    /**
+     * Sets local host IP address that discovery SPI uses.
+     * <p>
+     * If not provided, by default a first found non-loopback address
+     * will be used. If there is no non-loopback address available,
+     * then {@link InetAddress#getLocalHost()} will be used.
+     *
+     * @param locAddr IP address.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setLocalAddress(String locAddr) {
+        // Injection should not override value already set by Spring or user.
+        if (this.locAddr == null)
+            this.locAddr = locAddr;
+
+        return this;
+    }
+
+    /**
+     * Gets local address that was set to SPI with {@link #setLocalAddress(String)} method.
+     *
+     * @return local address.
+     */
+    public String getLocalAddress() {
+        return locAddr;
     }
 
     /**
@@ -360,8 +461,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @see #setAckTimeout(long)
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setReconnectCount(int reconCnt) {
+    public TcpDiscoverySpi setReconnectCount(int reconCnt) {
         this.reconCnt = reconCnt;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -382,8 +485,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @param maxAckTimeout Maximum acknowledgement timeout.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setMaxAckTimeout(long maxAckTimeout) {
+    public TcpDiscoverySpi setMaxAckTimeout(long maxAckTimeout) {
         this.maxAckTimeout = maxAckTimeout;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -401,8 +506,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @param locPort Local port to bind.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setLocalPort(int locPort) {
+    public TcpDiscoverySpi setLocalPort(int locPort) {
         this.locPort = locPort;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -420,8 +527,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @param locPortRange Local port range to bind.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setLocalPortRange(int locPortRange) {
+    public TcpDiscoverySpi setLocalPortRange(int locPortRange) {
         this.locPortRange = locPortRange;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -437,8 +546,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @param maxMissedHbs Max missed heartbeats.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setMaxMissedHeartbeats(int maxMissedHbs) {
+    public TcpDiscoverySpi setMaxMissedHeartbeats(int maxMissedHbs) {
         this.maxMissedHbs = maxMissedHbs;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -454,8 +565,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @param maxMissedClientHbs Max missed client heartbeats.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setMaxMissedClientHeartbeats(int maxMissedClientHbs) {
+    public TcpDiscoverySpi setMaxMissedClientHeartbeats(int maxMissedClientHbs) {
         this.maxMissedClientHbs = maxMissedClientHbs;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -475,8 +588,10 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @param statsPrintFreq Statistics print frequency in milliseconds.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setStatisticsPrintFrequency(long statsPrintFreq) {
+    public TcpDiscoverySpi setStatisticsPrintFrequency(long statsPrintFreq) {
         this.statsPrintFreq = statsPrintFreq;
+
+        return this;
     }
 
     /** {@inheritDoc} */
@@ -492,112 +607,187 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
      * @param ipFinderCleanFreq IP finder clean frequency.
      */
     @IgniteSpiConfiguration(optional = true)
-    public void setIpFinderCleanFrequency(long ipFinderCleanFreq) {
+    public TcpDiscoverySpi setIpFinderCleanFrequency(long ipFinderCleanFreq) {
         this.ipFinderCleanFreq = ipFinderCleanFreq;
+
+        return this;
     }
 
     /**
-     * This method is intended for troubleshooting purposes only.
+     * Gets IP finder for IP addresses sharing and storing.
      *
-     * @param debugMode {code True} to start SPI in debug mode.
+     * @return IP finder for IP addresses sharing and storing.
      */
-    public void setDebugMode(boolean debugMode) {
-        this.debugMode = debugMode;
+    public TcpDiscoveryIpFinder getIpFinder() {
+        return ipFinder;
     }
 
     /**
-     * This method is intended for troubleshooting purposes only.
+     * Sets IP finder for IP addresses sharing and storing.
+     * <p>
+     * If not provided {@link org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.TcpDiscoveryMulticastIpFinder} will be used by default.
      *
-     * @param debugMsgHist Message history log size.
+     * @param ipFinder IP finder.
      */
-    public void setDebugMessageHistory(int debugMsgHist) {
-        this.debugMsgHist = debugMsgHist;
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setIpFinder(TcpDiscoveryIpFinder ipFinder) {
+        this.ipFinder = ipFinder;
+
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public String getSpiState() {
-        synchronized (mux) {
-            return spiState.name();
-        }
+    /**
+     * Sets socket operations timeout. This timeout is used to limit connection time and
+     * write-to-socket time.
+     * <p>
+     * Note that when running Ignite on Amazon EC2, socket timeout must be set to a value
+     * significantly greater than the default (e.g. to {@code 30000}).
+     * <p>
+     * If not specified, default is {@link #DFLT_SOCK_TIMEOUT} or {@link #DFLT_SOCK_TIMEOUT_CLIENT}.
+     *
+     * @param sockTimeout Socket connection timeout.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setSocketTimeout(long sockTimeout) {
+        this.sockTimeout = sockTimeout;
+
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public int getMessageWorkerQueueSize() {
-        return msgWorker.queueSize();
+    /**
+     * Sets timeout for receiving acknowledgement for sent message.
+     * <p>
+     * If acknowledgement is not received within this timeout, sending is considered as failed
+     * and SPI tries to repeat message sending.
+     * <p>
+     * If not specified, default is {@link #DFLT_ACK_TIMEOUT} or {@link #DFLT_ACK_TIMEOUT_CLIENT}.
+     *
+     * @param ackTimeout Acknowledgement timeout.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setAckTimeout(long ackTimeout) {
+        this.ackTimeout = ackTimeout;
+
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Nullable @Override public UUID getCoordinator() {
-        TcpDiscoveryNode crd = resolveCoordinator();
+    /**
+     * Sets maximum network timeout to use for network operations.
+     * <p>
+     * If not specified, default is {@link #DFLT_NETWORK_TIMEOUT}.
+     *
+     * @param netTimeout Network timeout.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setNetworkTimeout(long netTimeout) {
+        this.netTimeout = netTimeout;
 
-        return crd != null ? crd.id() : null;
+        return this;
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
-        assert nodeId != null;
-
-        UUID locNodeId0 = getLocalNodeId();
+    @Override public long getJoinTimeout() {
+        return joinTimeout;
+    }
 
-        if (locNodeId0 != null && locNodeId0.equals(nodeId))
-            // Return local node directly.
-            return locNode;
+    /**
+     * Sets join timeout.
+     * <p>
+     * If non-shared IP finder is used and node fails to connect to
+     * any address from IP finder, node keeps trying to join within this
+     * timeout. If all addresses are still unresponsive, exception is thrown
+     * and node startup fails.
+     * <p>
+     * If not specified, default is {@link #DFLT_JOIN_TIMEOUT}.
+     *
+     * @param joinTimeout Join timeout ({@code 0} means wait forever).
+     *
+     * @see TcpDiscoveryIpFinder#isShared()
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setJoinTimeout(long joinTimeout) {
+        this.joinTimeout = joinTimeout;
 
-        TcpDiscoveryNode node = ring.node(nodeId);
+        return this;
+    }
 
-        if (node != null && !node.visible())
-            return null;
+    /**
+     * Sets thread priority. All threads within SPI will be started with it.
+     * <p>
+     * If not provided, default value is {@link #DFLT_THREAD_PRI}
+     *
+     * @param threadPri Thread priority.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setThreadPriority(int threadPri) {
+        this.threadPri = threadPri;
 
-        return node;
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public Collection<ClusterNode> getRemoteNodes() {
-        return F.upcast(ring.visibleRemoteNodes());
+    /**
+     * Sets delay between issuing of heartbeat messages. SPI sends heartbeat messages
+     * in configurable time interval to other nodes to notify them about its state.
+     * <p>
+     * If not provided, default value is {@link #DFLT_HEARTBEAT_FREQ}.
+     *
+     * @param hbFreq Heartbeat frequency in milliseconds.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setHeartbeatFrequency(long hbFreq) {
+        this.hbFreq = hbFreq;
+
+        return this;
     }
 
-    /** {@inheritDoc} */
-    @Override public void spiStart(String gridName) throws IgniteSpiException {
-        spiStart0(false);
+    /**
+     * @return Size of topology snapshots history.
+     */
+    public long getTopHistorySize() {
+        return topHistSize;
     }
 
     /**
-     * Starts or restarts SPI after stop (to reconnect).
+     * Sets size of topology snapshots history. Specified size should be greater than or equal to default size
+     * {@link #DFLT_TOP_HISTORY_SIZE}.
      *
-     * @param restart {@code True} if SPI is restarted after stop.
-     * @throws IgniteSpiException If failed.
+     * @param topHistSize Size of topology snapshots history.
      */
-    private void spiStart0(boolean restart) throws IgniteSpiException {
-        if (!restart)
-            // It is initial start.
-            onSpiStart();
+    @IgniteSpiConfiguration(optional = true)
+    public TcpDiscoverySpi setTopHistorySize(int topHistSize) {
+        if (topHistSize < DFLT_TOP_HISTORY_SIZE) {
+            U.warn(log, "Topology history size should be greater than or equal to default size. " +
+                "Specified size will not be set [curSize=" + this.topHistSize + ", specifiedSize=" + topHistSize +
+                ", defaultSize=" + DFLT_TOP_HISTORY_SIZE + ']');
 
-        synchronized (mux) {
-            spiState = DISCONNECTED;
+            return this;
         }
 
-        if (debugMode) {
-            if (!log.isInfoEnabled())
-                throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " +
-                    "in debug mode.");
+        this.topHistSize = topHistSize;
 
-            debugLog = new ConcurrentLinkedDeque<>();
-
-            U.quietAndWarn(log, "TCP discovery SPI is configured in debug mode.");
-        }
+        return this;
+    }
 
-        // Clear addresses collections.
-        fromAddrs.clear();
-        noResAddrs.clear();
+    /** {@inheritDoc} */
+    @Override public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver) {
+        assert locNodeAttrs == null;
+        assert locNodeVer == null;
 
-        sockTimeoutWorker = new SocketTimeoutWorker();
-        sockTimeoutWorker.start();
+        if (log.isDebugEnabled()) {
+            log.debug("Node attributes to set: " + attrs);
+            log.debug("Node version to set: " + ver);
+        }
 
-        msgWorker = new RingMessageWorker();
-        msgWorker.start();
+        locNodeAttrs = attrs;
+        locNodeVer = ver;
 
-        tcpSrvr = new TcpServer();
+        return this;
+    }
 
+    /**
+     * @param srvPort Server port.
+     */
+    void initLocalNode(int srvPort, boolean addExtAddrAttr) {
         // Init local node.
         IgniteBiTuple<Collection<String>, Collection<String>> addrs;
 
@@ -612,166 +802,190 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
             getLocalNodeId(),
             addrs.get1(),
             addrs.get2(),
-            tcpSrvr.port,
+            srvPort,
             metricsProvider,
             locNodeVer);
 
-        Collection<InetSocketAddress> extAddrs = addrRslvr == null ? null :
-            U.resolveAddresses(addrRslvr, F.flat(Arrays.asList(addrs.get1(), addrs.get2())),
-                locNode.discoveryPort());
+        if (addExtAddrAttr) {
+            Collection<InetSocketAddress> extAddrs = addrRslvr == null ? null :
+                U.resolveAddresses(addrRslvr, F.flat(Arrays.asList(addrs.get1(), addrs.get2())),
+                    locNode.discoveryPort());
 
-        if (extAddrs != null)
-            locNodeAttrs.put(createSpiAttributeName(ATTR_EXT_ADDRS), extAddrs);
+            locNodeAddrs = new LinkedHashSet<>();
+            locNodeAddrs.addAll(locNode.socketAddresses());
 
-        locNode.setAttributes(locNodeAttrs);
+            if (extAddrs != null) {
+                locNodeAttrs.put(createSpiAttributeName(ATTR_EXT_ADDRS), extAddrs);
 
-        locNode.local(true);
+                locNodeAddrs.addAll(extAddrs);
+            }
+        }
 
-        locNodeAddrs = getNodeAddresses(locNode);
+        locNode.setAttributes(locNodeAttrs);
+        locNode.local(true);
 
         if (log.isDebugEnabled())
             log.debug("Local node initialized: " + locNode);
+    }
 
-        // Start TCP server thread after local node is initialized.
-        tcpSrvr.start();
-
-        ring.localNode(locNode);
+    /**
+     * @param node Node.
+     * @return {@link LinkedHashSet} of internal and external addresses of provided node.
+     *      Internal addresses placed before external addresses.
+     */
+    @SuppressWarnings("TypeMayBeWeakened")
+    LinkedHashSet<InetSocketAddress> getNodeAddresses(TcpDiscoveryNode node) {
+        LinkedHashSet<InetSocketAddress> res = new LinkedHashSet<>(node.socketAddresses());
 
-        if (ipFinder.isShared())
-            registerLocalNodeAddress();
-        else {
-            if (F.isEmpty(ipFinder.getRegisteredAddresses()))
-                throw new IgniteSpiException("Non-shared IP finder must have IP addresses specified in " +
-                    "GridTcpDiscoveryIpFinder.getRegisteredAddresses() configuration property " +
-                    "(specify list of IP addresses in configuration).");
+        Collection<InetSocketAddress> extAddrs = node.attribute(createSpiAttributeName(ATTR_EXT_ADDRS));
 
-            ipFinderHasLocAddr = ipFinderHasLocalAddress();
-        }
+        if (extAddrs != null)
+            res.addAll(extAddrs);
 
-        if (statsPrintFreq > 0 && log.isInfoEnabled()) {
-            statsPrinter = new StatisticsPrinter();
-            statsPrinter.start();
-        }
+        return res;
+    }
 
-        stats.onJoinStarted();
+    /**
+     * @param node Node.
+     * @param sameHost Same host flag.
+     * @return {@link LinkedHashSet} of internal and external addresses of provided node.
+     *      Internal addresses placed before external addresses.
+     *      Internal addresses will be sorted with {@code inetAddressesComparator(sameHost)}.
+     */
+    @SuppressWarnings("TypeMayBeWeakened")
+    LinkedHashSet<InetSocketAddress> getNodeAddresses(TcpDiscoveryNode node, boolean sameHost) {
+        List<InetSocketAddress> addrs = U.arrayList(node.socketAddresses());
 
-        joinTopology();
+        Collections.sort(addrs, U.inetAddressesComparator(sameHost));
 
-        stats.onJoinFinished();
+        LinkedHashSet<InetSocketAddress> res = new LinkedHashSet<>(addrs);
 
-        hbsSnd = new HeartbeatsSender();
-        hbsSnd.start();
+        Collection<InetSocketAddress> extAddrs = node.attribute(createSpiAttributeName(ATTR_EXT_ADDRS));
 
-        chkStatusSnd = new CheckStatusSender();
-        chkStatusSnd.start();
+        if (extAddrs != null)
+            res.addAll(extAddrs);
 
-        if (ipFinder.isShared()) {
-            ipFinderCleaner = new IpFinderCleaner();
-            ipFinderCleaner.start();
-        }
+        return res;
+    }
 
-        if (log.isDebugEnabled() && !restart)
-            log.debug(startInfo());
+    /** {@inheritDoc} */
+    @Override public Collection<Object> injectables() {
+        return F.<Object>asList(ipFinder);
+    }
 
-        if (restart)
-            getSpiContext().registerPort(tcpSrvr.port, TCP);
+    /** {@inheritDoc} */
+    @Override public long getSocketTimeout() {
+        return sockTimeout;
     }
 
-    /**
-     * @throws IgniteSpiException If failed.
-     */
-    @SuppressWarnings("BusyWait")
-    private void registerLocalNodeAddress() throws IgniteSpiException {
-        // Make sure address registration succeeded.
-        while (true) {
-            try {
-                ipFinder.initializeLocalAddresses(locNode.socketAddresses());
+    /** {@inheritDoc} */
+    @Override public long getAckTimeout() {
+        return ackTimeout;
+    }
 
-                // Success.
-                break;
-            }
-            catch (IllegalStateException e) {
-                throw new IgniteSpiException("Failed to register local node address with IP finder: " +
-                    locNode.socketAddresses(), e);
-            }
-            catch (IgniteSpiException e) {
-                LT.error(log, e, "Failed to register local node address in IP finder on start " +
-                    "(retrying every 2000 ms).");
-            }
+    /** {@inheritDoc} */
+    @Override public long getNetworkTimeout() {
+        return netTimeout;
+    }
 
-            try {
-                U.sleep(2000);
-            }
-            catch (IgniteInterruptedCheckedException e) {
-                throw new IgniteSpiException("Thread has been interrupted.", e);
-            }
-        }
+    /** {@inheritDoc} */
+    @Override public int getThreadPriority() {
+        return threadPri;
     }
 
-    /**
-     * @throws IgniteSpiException If failed.
-     */
-    private void onSpiStart() throws IgniteSpiException {
-        startStopwatch();
+    /** {@inheritDoc} */
+    @Override public long getHeartbeatFrequency() {
+        return hbFreq;
+    }
 
-        checkParameters();
+    /** {@inheritDoc} */
+    @Override public String getIpFinderFormatted() {
+        return ipFinder.toString();
+    }
 
-        assertParameter(ipFinderCleanFreq > 0, "ipFinderCleanFreq > 0");
-        assertParameter(locPort > 1023, "localPort > 1023");
-        assertParameter(locPortRange >= 0, "localPortRange >= 0");
-        assertParameter(locPort + locPortRange <= 0xffff, "locPort + locPortRange <= 0xffff");
-        assertParameter(maxAckTimeout > ackTimeout, "maxAckTimeout > ackTimeout");
-        assertParameter(reconCnt > 0, "reconnectCnt > 0");
-        assertParameter(maxMissedHbs > 0, "maxMissedHeartbeats > 0");
-        assertParameter(maxMissedClientHbs > 0, "maxMissedClientHeartbeats > 0");
-        assertParameter(threadPri > 0, "threadPri > 0");
-        assertParameter(statsPrintFreq >= 0, "statsPrintFreq >= 0");
+    /** {@inheritDoc} */
+    @Override public long getNodesJoined() {
+        return stats.joinedNodesCount();
+    }
 
-        try {
-            locHost = U.resolveLocalHost(locAddr);
-        }
-        catch (IOException e) {
-            throw new IgniteSpiException("Unknown local address: " + locAddr, e);
-        }
+    /** {@inheritDoc} */
+    @Override public long getNodesLeft() {
+        return stats.leftNodesCount();
+    }
 
-        if (log.isDebugEnabled()) {
-            log.debug(configInfo("localHost", locHost.getHostAddress()));
-            log.debug(configInfo("localPort", locPort));
-            log.debug(configInfo("localPortRange", locPortRange));
-            log.debug(configInfo("threadPri", threadPri));
-            log.debug(configInfo("networkTimeout", netTimeout));
-            log.debug(configInfo("sockTimeout", sockTimeout));
-            log.debug(configInfo("ackTimeout", ackTimeout));
-            log.debug(configInfo("maxAckTimeout", maxAckTimeout));
-            log.debug(configInfo("reconnectCount", reconCnt));
-            log.debug(configInfo("ipFinder", ipFinder));
-            log.debug(configInfo("ipFinderCleanFreq", ipFinderCleanFreq));
-            log.debug(configInfo("heartbeatFreq", hbFreq));
-            log.debug(configInfo("maxMissedHeartbeats", maxMissedHbs));
-            log.debug(configInfo("statsPrintFreq", statsPrintFreq));
-        }
+    /** {@inheritDoc} */
+    @Override public long getNodesFailed() {
+        return stats.failedNodesCount();
+    }
 
-        // Warn on odd network timeout.
-        if (netTimeout < 3000)
-            U.warn(log, "Network timeout is too low (at least 3000 ms recommended): " + netTimeout);
+    /** {@inheritDoc} */
+    @Override public long getPendingMessagesRegistered() {
+        return stats.pendingMessagesRegistered();
+    }
 
-        registerMBean(gridName, this, TcpDiscoverySpiMBean.class);
+    /** {@inheritDoc} */
+    @Override public long getPendingMessagesDiscarded() {
+        return stats.pendingMessagesDiscarded();
+    }
 
-        if (ipFinder instanceof TcpDiscoveryMulticastIpFinder) {
-            TcpDiscoveryMulticastIpFinder mcastIpFinder = ((TcpDiscoveryMulticastIpFinder)ipFinder);
+    /** {@inheritDoc} */
+    @Override public long getAvgMessageProcessingTime() {
+        return stats.avgMessageProcessingTime();
+    }
 
-            if (mcastIpFinder.getLocalAddress() == null)
-                mcastIpFinder.setLocalAddress(locAddr);
-        }
+    /** {@inheritDoc} */
+    @Override public long getMaxMessageProcessingTime() {
+        return stats.maxMessageProcessingTime();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getTotalReceivedMessages() {
+        return stats.totalReceivedMessages();
     }
 
     /** {@inheritDoc} */
-    @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+    @Override public Map<String, Integer> getReceivedMessages() {
+        return stats.receivedMessages();
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getTotalProcessedMessages() {
+        return stats.totalProcessedMessages();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Map<String, Integer> getProcessedMessages() {
+        return stats.processedMessages();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getCoordinatorSinceTimestamp() {
+        return stats.coordinatorSinceTimestamp();
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
         super.onContextInitialized0(spiCtx);
 
         ctxInitLatch.countDown();
 
-        spiCtx.registerPort(tcpSrvr.port, TCP);
+        ipFinder.onSpiContextInitialized(spiCtx);
+
+        impl.onContextInitialized0(spiCtx);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void onContextDestroyed0() {
+        super.onContextDestroyed0();
+
+        if (ctxInitLatch.getCount() > 0)
+            // Safety.
+            ctxInitLatch.countDown();
+
+        if (ipFinder != null)
+            ipFinder.onSpiContextDestroyed();
+
+        getSpiContext().deregisterPorts();
     }
 
     /** {@inheritDoc} */
@@ -795,4621 +1009,858 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
     }
 
     /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        spiStop0(false);
+    @Override public ClusterNode getLocalNode() {
+        return locNode;
     }
 
-    /**
-     * Stops SPI finally or stops SPI for restart.
-     *
-     * @param disconnect {@code True} if SPI is being disconnected.
-     * @throws IgniteSpiException If failed.
-     */
-    private void spiStop0(boolean disconnect) throws IgniteSpiException {
-        if (ctxInitLatch.getCount() > 0)
-            // Safety.
-            ctxInitLatch.countDown();
+    /** {@inheritDoc} */
+    @Override public void setListener(@Nullable DiscoverySpiListener lsnr) {
+        this.lsnr = lsnr;
+    }
 
-        if (log.isDebugEnabled()) {
-            if (disconnect)
-                log.debug("Disconnecting SPI.");
-            else
-                log.debug("Preparing to start local node stop procedure.");
-        }
+    /** {@inheritDoc} */
+    @Override public TcpDiscoverySpi setDataExchange(DiscoverySpiDataExchange exchange) {
+        this.exchange = exchange;
 
-        if (disconnect) {
-            synchronized (mux) {
-                spiState = DISCONNECTING;
-            }
-        }
+        return this;
+    }
 
-        if (msgWorker != null && msgWorker.isAlive() && !disconnect) {
-            // Send node left message only if it is final stop.
-            msgWorker.addMessage(new TcpDiscoveryNodeLeftMessage(getLocalNodeId()));
+    /** {@inheritDoc} */
+    @Override public TcpDiscoverySpi setMetricsProvider(DiscoveryMetricsProvider metricsProvider) {
+        this.metricsProvider = metricsProvider;
 
-            synchronized (mux) {
-                long threshold = U.currentTimeMillis() + netTimeout;
+        return this;
+    }
 
-                long timeout = netTimeout;
+    /** {@inheritDoc} */
+    @Override public long getGridStartTime() {
+        assert gridStartTime != 0;
 
-                while (spiState != LEFT && timeout > 0) {
-                    try {
-                        mux.wait(timeout);
+        return gridStartTime;
+    }
 
-                        timeout = threshold - U.currentTimeMillis();
-                    }
-                    catch (InterruptedException ignored) {
-                        Thread.currentThread().interrupt();
+    /**
+     * @param sockAddr Remote address.
+     * @return Opened socket.
+     * @throws IOException If failed.
+     */
+    protected Socket openSocket(InetSocketAddress sockAddr) throws IOException {
+        assert sockAddr != null;
 
-                        break;
-                    }
-                }
+        InetSocketAddress resolved = sockAddr.isUnresolved() ?
+            new InetSocketAddress(InetAddress.getByName(sockAddr.getHostName()), sockAddr.getPort()) : sockAddr;
 
-                if (spiState == LEFT) {
-                    if (log.isDebugEnabled())
-                        log.debug("Verification for local node leave has been received from coordinator" +
-                            " (continuing stop procedure).");
-                }
-                else if (log.isInfoEnabled()) {
-                    log.info("No verification for local node leave has been received from coordinator" +
-                        " (will stop node anyway).");
-                }
-            }
-        }
+        InetAddress addr = resolved.getAddress();
 
-        U.interrupt(tcpSrvr);
-        U.join(tcpSrvr, log);
+        assert addr != null;
 
-        Collection<SocketReader> tmp;
+        Socket sock = new Socket();
 
-        synchronized (mux) {
-            tmp = U.arrayList(readers);
-        }
+        sock.bind(new InetSocketAddress(locHost, 0));
 
-        U.interrupt(tmp);
-        U.joinThreads(tmp, log);
+        sock.setTcpNoDelay(true);
 
-        U.interrupt(hbsSnd);
-        U.join(hbsSnd, log);
+        sock.connect(resolved, (int)sockTimeout);
 
-        U.interrupt(chkStatusSnd);
-        U.join(chkStatusSnd, log);
+        writeToSocket(sock, U.IGNITE_HEADER);
 
-        U.interrupt(ipFinderCleaner);
-        U.join(ipFinderCleaner, log);
+        return sock;
+    }
 
-        U.interrupt(msgWorker);
-        U.join(msgWorker, log);
+    /**
+     * Writes message to the socket.
+     *
+     * @param sock Socket.
+     * @param data Raw data to write.
+     * @throws IOException If IO failed or write timed out.
+     */
+    @SuppressWarnings("ThrowFromFinallyBlock")
+    protected void writeToSocket(Socket sock, byte[] data) throws IOException {
+        assert sock != null;
+        assert data != null;
 
-        U.interrupt(sockTimeoutWorker);
-        U.join(sockTimeoutWorker, log);
+        SocketTimeoutObject obj = new SocketTimeoutObject(sock, U.currentTimeMillis() + sockTimeout);
 
-        U.interrupt(statsPrinter);
-        U.join(statsPrinter, log);
+        sockTimeoutWorker.addTimeoutObject(obj);
 
-        if (ipFinder != null)
-            ipFinder.close();
+        IOException err = null;
 
-        Collection<TcpDiscoveryNode> rmts = null;
+        try {
+            OutputStream out = sock.getOutputStream();
 
-        if (!disconnect) {
-            // This is final stop.
-            unregisterMBean();
+            out.write(data);
 
-            if (log.isDebugEnabled())
-                log.debug(stopInfo());
+            out.flush();
         }
-        else {
-            getSpiContext().deregisterPorts();
-
-            rmts = ring.visibleRemoteNodes();
+        catch (IOException e) {
+            err = e;
         }
+        finally {
+            boolean cancelled = obj.cancel();
 
-        long topVer = ring.topologyVersion();
-
-        ring.clear();
-
-        if (rmts != null && !rmts.isEmpty()) {
-            // This is restart/disconnection and remote nodes are not empty.
-            // We need to fire FAIL event for each.
-            DiscoverySpiListener lsnr = this.lsnr;
+            if (cancelled)
+                sockTimeoutWorker.removeTimeoutObject(obj);
 
-            if (lsnr != null) {
-                Set<ClusterNode> processed = new HashSet<>();
+            // Throw original exception.
+            if (err != null)
+                throw err;
 
-                for (TcpDiscoveryNode n : rmts) {
-                    assert n.visible();
+            if (!cancelled)
+                throw new SocketTimeoutException("Write timed out (socket was concurrently closed).");
+        }
+    }
 
-                    processed.add(n);
+    /**
+     * Writes message to the socket.
+     *
+     * @param sock Socket.
+     * @param msg Message.
+     * @throws IOException If IO failed or write timed out.
+     * @throws IgniteCheckedException If marshalling failed.
+     */
+    protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg) throws IOException, IgniteCheckedException {
+        writeToSocket(sock, msg, new GridByteArrayOutputStream(8 * 1024)); // 8K.
+    }
 
-                    List<ClusterNode> top = U.arrayList(rmts, F.notIn(processed));
+    /**
+     * Writes message to the socket.
+     *
+     * @param sock Socket.
+     * @param msg Message.
+     * @param bout Byte array output stream.
+     * @throws IOException If IO failed or write timed out.
+     * @throws IgniteCheckedException If marshalling failed.
+     */
+    @SuppressWarnings("ThrowFromFinallyBlock")
+    protected void writeToSocket(Socket sock, TcpDiscoveryAbstractMessage msg, GridByteArrayOutputStream bout)
+        throws IOException, IgniteCheckedException {
+        assert sock != null;
+        assert msg != null;
+        assert bout != null;
 
-                    topVer++;
+        // Marshall message first to perform only write after.
+        marsh.marshal(msg, bout);
 
-                    Map<Long, Collection<ClusterNode>> hist = updateTopologyHistory(topVer,
-                        Collections.unmodifiableList(top));
+        SocketTimeoutObject obj = new SocketTimeoutObject(sock, U.currentTimeMillis() + sockTimeout);
 
-                    lsnr.onDiscovery(EVT_NODE_FAILED, topVer, n, top, hist, null);
-                }
-            }
-        }
+        sockTimeoutWorker.addTimeoutObject(obj);
 
-        printStatistics();
+        IOException err = null;
 
-        stats.clear();
+        try {
+            OutputStream out = sock.getOutputStream();
 
-        synchronized (mux) {
-            // Clear stored data.
-            leavingNodes.clear();
-            failedNodes.clear();
+            bout.writeTo(out);
 
-            spiState = DISCONNECTED;
+            out.flush();
         }
-    }
+        catch (IOException e) {
+            err = e;
+        }
+        finally {
+            boolean cancelled = obj.cancel();
 
-    /** {@inheritDoc} */
-    @Override protected void onContextDestroyed0() {
-        super.onContextDestroyed0();
+            if (cancelled)
+                sockTimeoutWorker.removeTimeoutObject(obj);
 
-        if (ctxInitLatch.getCount() > 0)
-            // Safety.
-            ctxInitLatch.countDown();
+            // Throw original exception.
+            if (err != null)
+                throw err;
 
-        getSpiContext().deregisterPorts();
+            if (!cancelled)
+                throw new SocketTimeoutException("Write timed out (socket was concurrently closed).");
+        }
     }
 
     /**
-     * @throws IgniteSpiException If any error occurs.
-     * @return {@code true} if IP finder contains local address.
+     * Writes response to the socket.
+     *
+     * @param sock Socket.
+     * @param res Integer response.
+     * @throws IOException If IO failed or write timed out.
      */
-    private boolean ipFinderHasLocalAddress() throws IgniteSpiException {
-        for (InetSocketAddress locAddr : locNodeAddrs) {
-            for (InetSocketAddress addr : registeredAddresses())
-                try {
-                    int port = addr.getPort();
-
-                    InetSocketAddress resolved = addr.isUnresolved() ?
-                        new InetSocketAddress(InetAddress.getByName(addr.getHostName()), port) :
-                        new InetSocketAddress(addr.getAddress(), port);
-
-                    if (resolved.equals(locAddr))
-                        return true;
-                }
-                catch (UnknownHostException e) {
-                    onException(e.getMessage(), e);
-                }
-        }
+    @SuppressWarnings("ThrowFromFinallyBlock")
+    protected void writeToSocket(Socket sock, int res) throws IOException {
+        assert sock != null;
 
-        return false;
-    }
+        SocketTimeoutObject obj = new SocketTimeoutObject(sock, U.currentTimeMillis() + sockTimeout);
 
-    /** {@inheritDoc} */
-    @Override public boolean pingNode(UUID nodeId) {
-        assert nodeId != null;
+        sockTimeoutWorker.addTimeoutObject(obj);
 
-        if (log.isDebugEnabled())
-            log.debug("Pinging node: " + nodeId + "].");
+        OutputStream out = sock.getOutputStream();
 
-        if (nodeId == getLocalNodeId())
-            return true;
+        IOException err = null;
 
-        TcpDiscoveryNode node = ring.node(nodeId);
+        try {
+            out.write(res);
 
-        if (node == null || !node.visible())
-            return false;
+            out.flush();
+        }
+        catch (IOException e) {
+            err = e;
+        }
+        finally {
+            boolean cancelled = obj.cancel();
 
-        boolean res = pingNode(node);
+            if (cancelled)
+                sockTimeoutWorker.removeTimeoutObject(obj);
 
-        if (!res && !node.isClient()) {
-            LT.warn(log, null, "Failed to ping node (status check will be initiated): " + nodeId);
+            // Throw original exception.
+            if (err != null)
+                throw err;
 
-            msgWorker.addMessage(new TcpDiscoveryStatusCheckMessage(locNode, node.id()));
+            if (!cancelled)
+                throw new SocketTimeoutException("Write timed out (socket was concurrently closed).");
         }
-
-        return res;
     }
 
     /**
-     * Pings the remote node to see if it's alive.
+     * Reads message from the socket limiting read time.
      *
-     * @param node Node.
-     * @return {@code True} if ping succeeds.
+     * @param sock Socket.
+     * @param in Input stream (in case socket stream was wrapped).
+     * @param timeout Socket timeout for this operation.
+     * @return Message.
+     * @throws IOException If IO failed or read timed out.
+     * @throws IgniteCheckedException If unmarshalling failed.
      */
-    private boolean pingNode(TcpDiscoveryNode node) {
-        assert node != null;
-
-        if (node.id().equals(getLocalNodeId()))
-            return true;
-
-        UUID clientNodeId = null;
+    protected <T> T readMessage(Socket sock, @Nullable InputStream in, long timeout) throws IOException, IgniteCheckedException {
+        assert sock != null;
 
-        if (node.isClient()) {
-            clientNodeId = node.id();
+        int oldTimeout = sock.getSoTimeout();
 
-            node = ring.node(node.clientRouterNodeId());
+        try {
+            sock.setSoTimeout((int)timeout);
 
-            if (node == null || !node.visible())
-                return false;
+            return marsh.unmarshal(in == null ? sock.getInputStream() : in, U.gridClassLoader());
         }
+        catch (IOException | IgniteCheckedException e) {
+            if (X.hasCause(e, SocketTimeoutException.class))
+                LT.warn(log, null, "Timed out waiting for message to be read (most probably, the reason is " +
+                    "in long GC pauses on remote node. Current timeout: " + timeout + '.');
 
-        for (InetSocketAddress addr : getNodeAddresses(node, U.sameMacs(locNode, node))) {
+            throw e;
+        }
+        finally {
+            // Quietly restore timeout.
             try {
-                // ID returned by the node should be the same as ID of the parameter for ping to succeed.
-                IgniteBiTuple<UUID, Boolean> t = pingNode(addr, clientNodeId);
-
-                return node.id().equals(t.get1()) && (clientNodeId == null || t.get2());
+                sock.setSoTimeout(oldTimeout);
             }
-            catch (IgniteCheckedException e) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to ping node [node=" + node + ", err=" + e.getMessage() + ']');
-
-                onException("Failed to ping node [node=" + node + ", err=" + e.getMessage() + ']', e);
-                // continue;
+            catch (SocketException ignored) {
+                // No-op.
             }
         }
-
-        return false;
     }
 
     /**
-     * Pings the node by its address to see if it's alive.
+     * Reads message delivery receipt from the socket.
      *
-     * @param addr Address of the node.
-     * @return ID of the remote node and "client exists" flag if node alive.
-     * @throws IgniteSpiException If an error occurs.
+     * @param sock Socket.
+     * @param timeout Socket timeout for this operation.
+     * @return Receipt.
+     * @throws IOException If IO failed or read timed out.
      */
-    private IgniteBiTuple<UUID, Boolean> pingNode(InetSocketAddress addr, @Nullable UUID clientNodeId)
-        throws IgniteCheckedException {
-        assert addr != null;
+    protected int readReceipt(Socket sock, long timeout) throws IOException {
+        assert sock != null;
 
-        UUID locNodeId = getLocalNodeId();
+        int oldTimeout = sock.getSoTimeout();
+
+        try {
+            sock.setSoTimeout((int)timeout);
 
-        if (F.contains(locNodeAddrs, addr)) {
-            if (clientNodeId == null)
-                return F.t(getLocalNodeId(), false);
+            int res = sock.getInputStream().read();
 
-            ClientMessageWorker clientWorker = clientMsgWorkers.get(clientNodeId);
+            if (res == -1)
+                throw new EOFException();
 
-            if (clientWorker == null)
-                return F.t(getLocalNodeId(), false);
+            return res;
+        }
+        catch (SocketTimeoutException e) {
+            LT.warn(log, null, "Timed out waiting for message delivery receipt (most probably, the reason is " +
+                "in long GC pauses on remote node; consider tuning GC and increasing 'ackTimeout' " +
+                "configuration property). Will retry to send message with increased timeout. " +
+                "Current timeout: " + timeout + '.');
 
-            boolean clientPingRes;
+            stats.onAckTimeout();
 
+            throw e;
+        }
+        finally {
+            // Quietly restore timeout.
             try {
-                clientPingRes = clientWorker.ping();
+                sock.setSoTimeout(oldTimeout);
             }
-            catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-
-                throw new IgniteInterruptedCheckedException(e);
+            catch (SocketException ignored) {
+                // No-op.
             }
-
-            return F.t(getLocalNodeId(), clientPingRes);
         }
+    }
 
-        GridFutureAdapter<IgniteBiTuple<UUID, Boolean>> fut = new GridFutureAdapter<>();
-
-        IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>> oldFut = pingMap.putIfAbsent(addr, fut);
+    /**
+     * Resolves addresses registered in the IP finder, removes duplicates and local host
+     * address and returns the collection of.
+     *
+     * @return Resolved addresses without duplicates and local address (potentially
+     *      empty but never null).
+     * @throws org.apache.ignite.spi.IgniteSpiException If an error occurs.
+     */
+    protected Collection<InetSocketAddress> resolvedAddresses() throws IgniteSpiException {
+        List<InetSocketAddress> res = new ArrayList<>();
 
-        if (oldFut != null)
-            return oldFut.get();
-        else {
-            Collection<Throwable> errs = null;
+        Collection<InetSocketAddress> addrs;
 
+        // Get consistent addresses collection.
+        while (true) {
             try {
-                Socket sock = null;
-
-                for (int i = 0; i < reconCnt; i++) {
-                    try {
-                        if (addr.isUnresolved())
-                            addr = new InetSocketAddress(InetAddress.getByName(addr.getHostName()), addr.getPort());
+                addrs = registeredAddresses();
 
-                        long tstamp = U.currentTimeMillis();
-
-                        sock = openSocket(addr);
+                break;
+            }
+            catch (IgniteSpiException e) {
+                LT.error(log, e, "Failed to get registered addresses from IP finder on start " +
+                    "(retrying every 2000 ms).");
+            }
 
-                        writeToSocket(sock, new TcpDiscoveryPingRequest(locNodeId, clientNodeId));
+            try {
+                U.sleep(2000);
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                throw new IgniteSpiException("Thread has been interrupted.", e);
+            }
+        }
 
-                        TcpDiscoveryPingResponse res = readMessage(sock, null, netTimeout);
+        for (InetSocketAddress addr : addrs) {
+            assert addr != null;
 
-                        if (locNodeId.equals(res.creatorNodeId())) {
-                            if (log.isDebugEnabled())
-                                log.debug("Ping response from local node: " + res);
+            try {
+                InetSocketAddress resolved = addr.isUnresolved() ?
+                    new InetSocketAddress(InetAddress.getByName(addr.getHostName()), addr.getPort()) : addr;
 
-                            break;
-                        }
+                if (locNodeAddrs == null || !locNodeAddrs.contains(resolved))
+                    res.add(resolved);
+            }
+            catch (UnknownHostException ignored) {
+                LT.warn(log, null, "Failed to resolve address from IP finder (host is unknown): " + addr);
 
-                        stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+                // Add address in any case.
+                res.add(addr);
+            }
+        }
 
-                        IgniteBiTuple<UUID, Boolean> t = F.t(res.creatorNodeId(), res.clientExists());
+        if (!res.isEmpty())
+            Collections.shuffle(res);
 
-                        fut.onDone(t);
-
-                        return t;
-                    }
-                    catch (IOException | IgniteCheckedException e) {
-                        if (errs == null)
-                            errs = new ArrayList<>();
-
-                        errs.add(e);
-                    }
-                    finally {
-                        U.closeQuiet(sock);
-                    }
-                }
-            }
-            catch (Throwable t) {
-                fut.onDone(t);
-
-                if (t instanceof Error)
-                    throw t;
-
-                throw U.cast(t);
-            }
-            finally {
-                if (!fut.isDone())
-                    fut.onDone(U.exceptionWithSuppressed("Failed to ping node by address: " + addr, errs));
-
-                boolean b = pingMap.remove(addr, fut);
-
-                assert b;
-            }
-
-            return fut.get();
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void disconnect() throws IgniteSpiException {
-        spiStop0(true);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator nodeAuth) {
-        this.nodeAuth = nodeAuth;
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
-        try {
-            msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, marsh.marshal(evt)));
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteSpiException("Failed to marshal custom event: " + evt, e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
-        ClusterNode node = ring.node(nodeId);
-
-        if (node != null) {
-            TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
-                node.id(), node.order());
-
-            msgWorker.addMessage(msg);
-        }
-    }
-
-    /**
-     * Tries to join this node to topology.
-     *
-     * @throws IgniteSpiException If any error occurs.
-     */
-    private void joinTopology() throws IgniteSpiException {
-        synchronized (mux) {
-            assert spiState == CONNECTING || spiState == DISCONNECTED;
-
-            spiState = CONNECTING;
-        }
-
-        SecurityCredentials locCred = (SecurityCredentials)locNode.getAttributes()
-            .get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
-
-        // Marshal credentials for backward compatibility and security.
-        marshalCredentials(locNode);
-
-        while (true) {
-            if (!sendJoinRequestMessage()) {
-                if (log.isDebugEnabled())
-                    log.debug("Join request message has not been sent (local node is the first in the topology).");
-
-                if (nodeAuth != null) {
-                    // Authenticate local node.
-                    try {
-                        SecurityContext subj = nodeAuth.authenticateNode(locNode, locCred);
-
-                        if (subj == null)
-                            throw new IgniteSpiException("Authentication failed for local node: " + locNode.id());
-
-                        Map<String, Object> attrs = new HashMap<>(locNode.attributes());
-
-                        attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT,
-                            ignite.configuration().getMarshaller().marshal(subj));
-                        attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
-
-                        locNode.setAttributes(attrs);
-                    }
-                    catch (IgniteException | IgniteCheckedException e) {
-                        throw new IgniteSpiException("Failed to authenticate local node (will shutdown local node).", e);
-                    }
-                }
-
-                locNode.order(1);
-                locNode.internalOrder(1);
-
-                gridStartTime = U.currentTimeMillis();
-
-                locNode.visible(true);
-
-                ring.clear();
-
-                ring.topologyVersion(1);
-
-                synchronized (mux) {
-                    topHist.clear();
-
-                    spiState = CONNECTED;
-
-                    mux.notifyAll();
-                }
-
-                notifyDiscovery(EVT_NODE_JOINED, 1, locNode);
-
-                break;
-            }
-
-            if (log.isDebugEnabled())
-                log.debug("Join request message has been sent (waiting for coordinator response).");
-
-            synchronized (mux) {
-                long threshold = U.currentTimeMillis() + netTimeout;
-
-                long timeout = netTimeout;
-
-                while (spiState == CONNECTING && timeout > 0) {
-                    try {
-                        mux.wait(timeout);
-
-                        timeout = threshold - U.currentTimeMillis();
-                    }
-                    catch (InterruptedException ignored) {
-                        Thread.currentThread().interrupt();
-
-                        throw new IgniteSpiException("Thread has been interrupted.");
-                    }
-                }
-
-                if (spiState == CONNECTED)
-                    break;
-                else if (spiState == DUPLICATE_ID)
-                    throw duplicateIdError((TcpDiscoveryDuplicateIdMessage)joinRes.get());
-                else if (spiState == AUTH_FAILED)
-                    throw authenticationFailedError((TcpDiscoveryAuthFailedMessage)joinRes.get());
-                else if (spiState == CHECK_FAILED)
-                    throw checkFailedError((TcpDiscoveryCheckFailedMessage)joinRes.get());
-                else if (spiState == LOOPBACK_PROBLEM) {
-                    TcpDiscoveryLoopbackProblemMessage msg = (TcpDiscoveryLoopbackProblemMessage)joinRes.get();
-
-                    boolean locHostLoopback = locHost.isLoopbackAddress();
-
-                    String firstNode = locHostLoopback ? "local" : "remote";
-
-                    String secondNode = locHostLoopback ? "remote" : "local";
-
-                    throw new IgniteSpiException("Failed to add node to topology because " + firstNode +
-                        " node is configured to use loopback address, but " + secondNode + " node is not " +
-                        "(consider changing 'localAddress' configuration parameter) " +
-                        "[locNodeAddrs=" + U.addressesAsString(locNode) + ", rmtNodeAddrs=" +
-                        U.addressesAsString(msg.addresses(), msg.hostNames()) + ']');
-                }
-                else
-                    LT.warn(log, null, "Node has not been connected to topology and will repeat join process. " +
-                        "Check remote nodes logs for possible error messages. " +
-                        "Note that large topology may require significant time to start. " +
-                        "Increase 'TcpDiscoverySpi.networkTimeout' configuration property " +
-                        "if getting this message on the starting nodes [networkTimeout=" + netTimeout + ']');
-            }
-        }
-
-        assert locNode.order() != 0;
-        assert locNode.internalOrder() != 0;
-
-        if (log.isDebugEnabled())
-            log.debug("Discovery SPI has been connected to topology with order: " + locNode.internalOrder());
-    }
+        return res;
+    }
 
     /**
-     * Tries to send join request message to a random node presenting in topology.
-     * Address is provided by {@link TcpDiscoveryIpFinder} and message is
-     * sent to first node connection succeeded to.
+     * Gets addresses registered in the IP finder, initializes addresses having no
+     * port (or 0 port) with {@link #DFLT_PORT}.
      *
-     * @return {@code true} if send succeeded.
-     * @throws IgniteSpiException If any error occurs.
+     * @return Registered addresses.
+     * @throws org.apache.ignite.spi.IgniteSpiException If an error occurs.
      */
-    @SuppressWarnings({"BusyWait"})
-    private boolean sendJoinRequestMessage() throws IgniteSpiException {
-        TcpDiscoveryAbstractMessage joinReq = new TcpDiscoveryJoinRequestMessage(locNode,
-            collectExchangeData(getLocalNodeId()));
-
-        // Time when it has been detected, that addresses from IP finder do not respond.
-        long noResStart = 0;
-
-        while (true) {
-            Collection<InetSocketAddress> addrs = resolvedAddresses();
-
-            if (F.isEmpty(addrs))
-                return false;
+    protected Collection<InetSocketAddress> registeredAddresses() throws IgniteSpiException {
+        Collection<InetSocketAddress> res = new ArrayList<>();
 
-            boolean retry = false;
-            Collection<Exception> errs = new ArrayList<>();
+        for (InetSocketAddress addr : ipFinder.getRegisteredAddresses()) {
+            if (addr.getPort() == 0) {
+                // TcpDiscoveryNode.discoveryPort() returns an correct port for a server node and 0 for client node.
+                int port = locNode.discoveryPort() != 0 ? locNode.discoveryPort() : DFLT_PORT;
 
-            try (SocketMultiConnector multiConnector = new SocketMultiConnector(this, addrs, 2)) {
-                GridTuple3<InetSocketAddress, Socket, Exception> tuple;
-
-                while ((tuple = multiConnector.next()) != null) {
-                    InetSocketAddress addr = tuple.get1();
-                    Socket sock = tuple.get2();
-                    Exception ex = tuple.get3();
-
-                    if (ex == null) {
-                        assert sock != null;
-
-                        try {
-                            Integer res = sendMessageDirectly(joinReq, addr, sock);
-
-                            assert res != null;
-
-                            noResAddrs.remove(addr);
-
-                            // Address is responsive, reset period start.
-                            noResStart = 0;
-
-                            switch (res) {
-                                case RES_WAIT:
-                                    // Concurrent startup, try sending join request again or wait if no success.
-                                    retry = true;
-
-                                    break;
-                                case RES_OK:
-                                    if (log.isDebugEnabled())
-                                        log.debug("Join request message has been sent to address [addr=" + addr +
-                                            ", req=" + joinReq + ']');
-
-                                    // Join request sending succeeded, wait for response from topology.
-                                    return true;
-
-                                default:
-                                    // Concurrent startup, try next node.
-                                    if (res == RES_CONTINUE_JOIN) {
-                                        if (!fromAddrs.contains(addr))
-                                            retry = true;
-                                    }
-                                    else {
-                                        if (log.isDebugEnabled())
-                                            log.debug("Unexpected response to join request: " + res);
-
-                                        retry = true;
-                                    }
-
-                                    break;
-                            }
-                        }
-                        catch (IgniteSpiException e) {
-                            e.printStackTrace();
-
-                            ex = e;
-                        }
-                    }
-
-                    if (ex != null) {
-                        errs.add(ex);
-
-                        if (log.isDebugEnabled()) {
-                            IOException ioe = X.cause(ex, IOException.class);
-
-                            log.debug("Failed to send join request message [addr=" + addr +
-                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']');
-
-                            onException("Failed to send join request message [addr=" + addr +
-                                ", msg=" + ioe != null ? ioe.getMessage() : ex.getMessage() + ']', ioe);
-                        }
-
-                        noResAddrs.add(addr);
-                    }
-                }
+                addr = addr.isUnresolved() ? new InetSocketAddress(addr.getHostName(), port) :
+                    new InetSocketAddress(addr.getAddress(), port);
             }
 
-            if (retry) {
-                if (log.isDebugEnabled())
-                    log.debug("Concurrent discovery SPI start has been detected (local node should wait).");
-
-                try {
-                    U.sleep(2000);
-                }
-                catch (IgniteInterruptedCheckedException e) {
-                    throw new IgniteSpiException("Thread has been interrupted.", e);
-                }
-            }
-            else if (!ipFinder.isShared() && !ipFinderHasLocAddr) {
-                IgniteCheckedException e = null;
-
-                if (!errs.isEmpty()) {
-                    e = new IgniteCheckedException("Multiple connection attempts failed.");
-
-                    for (Exception err : errs)
-                        e.addSuppressed(err);
-                }
-
-                if (e != null && X.hasCause(e, ConnectException.class))
-                    LT.warn(log, null, "Failed to connect to any address from IP finder " +
-                        "(make sure IP finder addresses are correct and firewalls are disabled on all host machines): " +
-                        addrs);
-
-                if (joinTimeout > 0) {
-                    if (noResStart == 0)
-                        noResStart = U.currentTimeMillis();
-                    else if (U.currentTimeMillis() - noResStart > joinTimeout)
-                        throw new IgniteSpiException(
-                            "Failed to connect to any address from IP finder within join timeout " +
-                                "(make sure IP finder addresses are correct, and operating system firewalls are disabled " +
-                                "on all host machines, or consider increasing 'joinTimeout' configuration property): " +
-                                addrs, e);
-                }
-
-                try {
-                    U.sleep(2000);
-                }
-                catch (IgniteInterruptedCheckedException ex) {
-                    throw new IgniteSpiException("Thread has been interrupted.", ex);
-                }
-            }
-            else
-                break;
+            res.add(addr);
         }
 
-        return false;
+        return res;
     }
 
     /**
-     * Establishes connection to an address, sends message and returns the response (if any).
-     *
-     * @param msg Message to send.
-     * @param addr Address to send message to.
-     * @return Response read from the recipient or {@code null} if no response is supposed.
-     * @throws IgniteSpiException If an error occurs.
+     * @param msg Message.
+     * @return Error.
      */
-    @Nullable private Integer sendMessageDirectly(TcpDiscoveryAbstractMessage msg, InetSocketAddress addr, Socket sock)
-        throws IgniteSpiException {
+    protected IgniteSpiException duplicateIdError(TcpDiscoveryDuplicateIdMessage msg) {
         assert msg != null;
-        assert addr != null;
-
-        Collection<Throwable> errs = null;
-
-        long ackTimeout0 = ackTimeout;
-
-        int connectAttempts = 1;
-
-        boolean joinReqSent = false;
-
-        UUID locNodeId = getLocalNodeId();
-
-        for (int i = 0; i < reconCnt; i++) {
-            // Need to set to false on each new iteration,
-            // since remote node may leave in the middle of the first iteration.
-            joinReqSent = false;
-
-            boolean openSock = false;
-
-            try {
-                long tstamp = U.currentTimeMillis();
-
-                if (sock == null)
-                    sock = openSocket(addr);
-
-                openSock = true;
-
-                // Handshake.
-                writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId));
-
-                TcpDiscoveryHandshakeResponse res = readMessage(sock, null, ackTimeout0);
-
-                if (locNodeId.equals(res.creatorNodeId())) {
-                    if (log.isDebugEnabled())
-                        log.debug("Handshake response from local node: " + res);
-
-                    break;
-                }
-
-                stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
-
-                // Send message.
-                tstamp = U.currentTimeMillis();
-
-                writeToSocket(sock, msg);
-
-                stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
-
-                if (debugMode)
-                    debugLog("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
-                        ", rmtNodeId=" + res.creatorNodeId() + ']');
-
-                if (log.isDebugEnabled())
-                    log.debug("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
-                        ", rmtNodeId=" + res.creatorNodeId() + ']');
-
-                // Connection has been established, but
-                // join request may not be unmarshalled on remote host.
-                // E.g. due to class not found issue.
-                joinReqSent = msg instanceof TcpDiscoveryJoinRequestMessage;
-
-                return readReceipt(sock, ackTimeout0);
-            }
-            catch (ClassCastException e) {
-                // This issue is rarely reproducible on AmazonEC2, but never
-                // on dedicated machines.
-                if (log.isDebugEnabled())
-                    U.error(log, "Class cast exception on direct send: " + addr, e);
-
-                onException("Class cast exception on direct send: " + addr, e);
-
-                if (errs == null)
-                    errs = new ArrayList<>();
-
-                errs.add(e);
-            }
-            catch (IOException | IgniteCheckedException e) {
-                if (log.isDebugEnabled())
-                    log.error("Exception on direct send: " + e.getMessage(), e);
-
-                onException("Exception on direct send: " + e.getMessage(), e);
-
-                if (errs == null)
-                    errs = new ArrayList<>();
-
-                errs.add(e);
-
-                if (!openSock) {
-                    // Reconnect for the second time, if connection is not established.
-                    if (connectAttempts < 2) {
-                        connectAttempts++;
-
-                        continue;
-                    }
-
-                    break; // Don't retry if we can not establish connection.
-                }
-
-                if (e instanceof SocketTimeoutException || X.hasCause(e, SocketTimeoutException.class)) {
-                    ackTimeout0 *= 2;
-
-                    if (!checkAckTimeout(ackTimeout0))
-                        break;
-                }
-            }
-            finally {
-                U.closeQuiet(sock);
-
-                sock = null;
-            }
-        }
-
-        if (joinReqSent) {
-            if (log.isDebugEnabled())
-                log.debug("Join request has been sent, but receipt has not been read (returning RES_WAIT).");
-
-            // Topology will not include this node,
-            // however, warning on timed out join will be output.
-            return RES_OK;
-        }
 
-        throw new IgniteSpiException(
-            "Failed to send message to address [addr=" + addr + ", msg=" + msg + ']',
-            U.exceptionWithSuppressed("Failed to send message to address " +
-                "[addr=" + addr + ", msg=" + msg + ']', errs));
+        return new IgniteSpiException("Local node has the same ID as existing node in topology " +
+            "(fix configuration and restart local node) [localNode=" + locNode +
+            ", existingNode=" + msg.node() + ']');
     }
 
     /**
-     * Marshalls credentials with discovery SPI marshaller (will replace attribute value).
-     *
-     * @param node Node to marshall credentials for.
-     * @throws IgniteSpiException If marshalling failed.
+     * @param msg Message.
+     * @return Error.
      */
-    private void marshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
-        try {
-            // Use security-unsafe getter.
-            Map<String, Object> attrs = new HashMap<>(node.getAttributes());
-
-            attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS,
-                marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS)));
+    protected IgniteSpiException authenticationFailedError(TcpDiscoveryAuthFailedMessage msg) {
+        assert msg != null;
 
-            node.setAttributes(attrs);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteSpiException("Failed to marshal node security credentials: " + node.id(), e);
-        }
+        return new IgniteSpiException(new IgniteAuthenticationException("Authentication failed [nodeId=" +
+            msg.creatorNodeId() + ", addr=" + msg.address().getHostAddress() + ']'));
     }
 
     /**
-     * Unmarshalls credentials with discovery SPI marshaller (will not replace attribute value).
-     *
-     * @param node Node to unmarshall credentials for.
-     * @return Security credentials.
-     * @throws IgniteSpiException If unmarshal fails.
+     * @param msg Message.
+     * @return Error.
      */
-    private SecurityCredentials unmarshalCredentials(TcpDiscoveryNode node) throws IgniteSpiException {
-        try {
-            byte[] credBytes = (byte[])node.getAttributes().get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
-
-            if (credBytes == null)
-                return null;
+    protected IgniteSpiException checkFailedError(TcpDiscoveryCheckFailedMessage msg) {
+        assert msg != null;
 
-            return marsh.unmarshal(credBytes, null);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteSpiException("Failed to unmarshal node security credentials: " + node.id(), e);
-        }
+        return versionCheckFailed(msg) ? new IgniteSpiVersionCheckException(msg.error()) :
+            new IgniteSpiException(msg.error());
     }
 
     /**
-     * @param ackTimeout Acknowledgement timeout.
-     * @return {@code True} if acknowledgement timeout is less or equal to
-     * maximum acknowledgement timeout, {@code false} otherwise.
+     * @param msg Message.
+     * @return Whether delivery of the message is ensured.
      */
-    private boolean checkAckTimeout(long ackTimeout) {
-        if (ackTimeout > maxAckTimeout) {
-            LT.warn(log, null, "Acknowledgement timeout is greater than maximum acknowledgement timeout " +
-                "(consider increasing 'maxAckTimeout' configuration property) " +
-                "[ackTimeout=" + ackTimeout + ", maxAckTimeout=" + maxAckTimeout + ']');
-
-            return false;
-        }
-
-        return true;
+    protected boolean ensured(TcpDiscoveryAbstractMessage msg) {
+        return U.getAnnotation(msg.getClass(), TcpDiscoveryEnsureDelivery.class) != null;
     }
 
     /**
-     * Notify external listener on discovery event.
-     *
-     * @param type Discovery event type. See {@link DiscoveryEvent} for more details.
-     * @param topVer Topology version.
-     * @param node Remote node this event is connected with.
+     * @param msg Failed message.
+     * @return {@code True} if specified failed message relates to version incompatibility, {@code false} otherwise.
+     * @deprecated Parsing of error message was used for preserving backward compatibility. We should remove it
+     *      and create separate message for failed version check with next major release.
      */
-    private void notifyDiscovery(int type, long topVer, TcpDiscoveryNode node) {
-        assert type > 0;
-        assert node != null;
-
-        DiscoverySpiListener lsnr = this.lsnr;
-
-        TcpDiscoverySpiState spiState = spiStateCopy();
-
-        if (lsnr != null && node.visible() && (spiState == CONNECTED || spiState == DISCONNECTING)) {
-            if (log.isDebugEnabled())
-                log.debug("Discovery notification [node=" + node + ", spiState=" + spiState +
-                    ", type=" + U.gridEventName(type) + ", topVer=" + topVer + ']');
-
-            Collection<ClusterNode> top = F.upcast(ring.visibleNodes());
-
-            Map<Long, Collection<ClusterNode>> hist = updateTopologyHistory(topVer, top);
-
-            lsnr.onDiscovery(type, topVer, node, top, hist, null);
-        }
-        else if (log.isDebugEnabled())
-            log.debug("Skipped discovery notification [node=" + node + ", spiState=" + spiState +
-                ", type=" + U.gridEventName(type) + ", topVer=" + topVer + ']');
+    @Deprecated
+    private static boolean versionCheckFailed(TcpDiscoveryCheckFailedMessage msg) {
+        return msg.error().contains("versions are not compatible");
     }
 
     /**
-     * Update topology history with new topology snapshots.
-     *
-     * @param topVer Topology version.
-     * @param top Topology snapshot.
-     * @return Copy of updated topology history.
+     * @param nodeId Node ID.
+     * @return Marshalled exchange data.
      */
-    @Nullable private Map<Long, Collection<ClusterNode>> updateTopologyHistory(long topVer, Collection<ClusterNode> top) {
-        synchronized (mux) {
-            if (topHist.containsKey(topVer))
-                return null;
+    protected Map<Integer, byte[]> collectExchangeData(UUID nodeId) {
+        Map<Integer, Serializable> data = exchange.collect(nodeId);
 
-            topHist.put(topVer, top);
+        if (data == null)
+            return null;
 
-            while (topHist.size() > topHistSize)
-                topHist.remove(topHist.firstKey());
+        Map<Integer, byte[]> data0 = U.newHashMap(data.size());
 
-            if (log.isDebugEnabled())
-                log.debug("Added topology snapshot to history, topVer=" + topVer + ", historySize=" + topHist.size());
+        for (Map.Entry<Integer, Serializable> entry : data.entrySet()) {
+            try {
+                byte[] bytes = marsh.marshal(entry.getValue());
 
-            return new TreeMap<>(topHist);
+                data0.put(entry.getKey(), bytes);
+            }
+            catch (IgniteCheckedException e) {
+                U.error(log, "Failed to marshal discovery data " +
+                    "[comp=" + entry.getKey() + ", data=" + entry.getValue() + ']', e);
+            }
         }
-    }
 
-    /**
-     * @param msg Error message.
-     * @param e Exception.
-     */
-    private void onException(String msg, Exception e){
-        getExceptionRegistry().onException(msg, e);
+        return data0;
     }
 
     /**
-     * @param node Node.
-     * @return {@link LinkedHashSet} of internal and external addresses of provided node.
-     *      Internal addresses placed before external addresses.
+     * @param joiningNodeID Joining node ID.
+     * @param nodeId Remote node ID for which data is provided.
+     * @param data Collection of marshalled discovery data objects from different components.
+     * @param clsLdr Class loader for discovery data unmarshalling.
      */
-    @SuppressWarnings("TypeMayBeWeakened")
-    private LinkedHashSet<InetSocketAddress> getNodeAddresses(TcpDiscoveryNode node) {
-        LinkedHashSet<InetSocketAddress> res = new LinkedHashSet<>(node.socketAddresses());
-
-        Collection<InetSocketAddress> extAddrs = node.attribute(createSpiAttributeName(ATTR_EXT_ADDRS));
+    p

<TRUNCATED>


[22/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix GridProjectionSelfTest

Posted by se...@apache.org.
# IGNITE-943 Fix GridProjectionSelfTest


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

Branch: refs/heads/ignite-sprint-5
Commit: 5cec2ba61b98cabd421aeb0091c5531b246f54e3
Parents: e2e90c4
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 15:43:34 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 15:43:34 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/TcpDiscoverySpi.java   |  4 +++-
 .../ignite/internal/GridProjectionAbstractTest.java | 16 ++++++++++++++++
 2 files changed, 19 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5cec2ba6/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 5a2ddd6..52ea78c 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
@@ -393,8 +393,10 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      *
      * @param forceSrvMode forceServerMode flag.
      */
-    public void setForceServerMode(boolean forceSrvMode) {
+    public TcpDiscoverySpi setForceServerMode(boolean forceSrvMode) {
         this.forceSrvMode = forceSrvMode;
+
+        return this;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5cec2ba6/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
index 0171290..cb1341c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
@@ -20,9 +20,13 @@ package org.apache.ignite.internal;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
+import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 
@@ -38,6 +42,9 @@ import static org.apache.ignite.events.EventType.*;
  */
 @SuppressWarnings("deprecation")
 public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest implements Externalizable {
+    /** VM ip finder for TCP discovery. */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
     /** Waiting timeout. */
     private static final int WAIT_TIMEOUT = 30000;
 
@@ -87,6 +94,15 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
         // No-op.
     }
 
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setForceServerMode(true).setIpFinder(ipFinder));
+
+        return cfg;
+    }
+
     /**
      * @param startGrid Start grid flag.
      */


[27/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix IgniteCacheClientNodePartitionsExchangeTest

Posted by se...@apache.org.
# IGNITE-943 Fix  IgniteCacheClientNodePartitionsExchangeTest


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

Branch: refs/heads/ignite-sprint-5
Commit: 9759b19ce5db74533e171a9eb3ba0bd2eee449b7
Parents: 6adc974
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 17:06:57 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 17:06:57 2015 +0300

----------------------------------------------------------------------
 .../distributed/IgniteCacheClientNodePartitionsExchangeTest.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9759b19c/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
index 162aa81..d1fbf5a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodePartitionsExchangeTest.java
@@ -64,7 +64,7 @@ public class IgniteCacheClientNodePartitionsExchangeTest extends GridCommonAbstr
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
-        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder).setForceServerMode(true);
 
         cfg.setClientMode(client);
 


[12/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix IDEA's warnings.

Posted by se...@apache.org.
# IGNITE-943 Fix IDEA's warnings.


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

Branch: refs/heads/ignite-sprint-5
Commit: 982cb0ca1acd1e34f79a387f41d4b25e34e4faab
Parents: e66578e
Author: sevdokimov <se...@gridgain.com>
Authored: Wed May 27 18:25:36 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Wed May 27 18:25:36 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spi/discovery/tcp/ClientImpl.java     | 2 ++
 .../org/apache/ignite/spi/discovery/tcp/ServerImpl.java     | 9 +++++----
 .../apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java    | 2 +-
 3 files changed, 8 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/982cb0ca/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 455b2af..aa254ec 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
@@ -541,6 +541,7 @@ class ClientImpl extends TcpDiscoveryImpl {
         U.closeQuiet(msgWorker.currSock);
     }
 
+    /** {@inheritDoc} */
     @Override protected IgniteSpiThread workerThread() {
         return msgWorker;
     }
@@ -548,6 +549,7 @@ class ClientImpl extends TcpDiscoveryImpl {
     /**
      * FOR TEST PURPOSE ONLY!
      */
+    @SuppressWarnings("BusyWait")
     public void waitForClientMessagePrecessed() {
         Object last = msgWorker.queue.peekLast();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/982cb0ca/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 2cd8212..184895b 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
@@ -55,6 +55,7 @@ import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusChe
 /**
  *
  */
+@SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
 class ServerImpl extends TcpDiscoveryImpl {
     /** */
     private final Executor utilityPool = new ThreadPoolExecutor(0, 1, 2000, TimeUnit.MILLISECONDS,
@@ -416,7 +417,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             DiscoverySpiListener lsnr = adapter.lsnr;
 
             if (lsnr != null) {
-                Set<ClusterNode> processed = new HashSet<>();
+                Collection<ClusterNode> processed = new HashSet<>();
 
                 for (TcpDiscoveryNode n : rmts) {
                     assert n.visible();
@@ -1412,7 +1413,7 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    public void dumpDebugInfo(IgniteLogger log) {
+    @Override public void dumpDebugInfo(IgniteLogger log) {
         if (!debugMode) {
             U.quietAndWarn(log, "Failed to dump debug info (discovery SPI was not configured " +
                 "in debug mode, consider setting 'debugMode' configuration property to 'true').");
@@ -2040,12 +2041,12 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 final boolean sameHost = U.sameMacs(locNode, next);
 
-                List<InetSocketAddress> localNodeAddresses = U.arrayList(locNode.socketAddresses());
+                List<InetSocketAddress> locNodeAddrs = U.arrayList(locNode.socketAddresses());
 
                 addr: for (InetSocketAddress addr : adapter.getNodeAddresses(next, sameHost)) {
                     long ackTimeout0 = adapter.ackTimeout;
 
-                    if (localNodeAddresses.contains(addr)){
+                    if (locNodeAddrs.contains(addr)){
                         if (log.isDebugEnabled())
                             log.debug("Skip to send message to the local node (probably remote node has the same " +
                                 "loopback address that local node): " + addr);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/982cb0ca/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 93a1ec3..d995717 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
@@ -1613,7 +1613,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
      * @return {@code True} if node is stopping.
      */
     boolean isNodeStopping0() {
-        return super.isNodeStopping();
+        return isNodeStopping();
     }
 
     /**


[05/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Merge TcpDiscoverySpi and TcpClientDiscoverySpi

Posted by se...@apache.org.
# IGNITE-943 Merge TcpDiscoverySpi and TcpClientDiscoverySpi


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

Branch: refs/heads/ignite-sprint-5
Commit: 838c0fd83b67b5e906144777d7340d5e61bdfa8a
Parents: 2f169f5
Author: sevdokimov <se...@gridgain.com>
Authored: Wed May 27 16:18:37 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Wed May 27 16:18:37 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |    9 +-
 .../org/apache/ignite/cluster/ClusterNode.java  |    6 +-
 .../configuration/IgniteConfiguration.java      |    4 +-
 .../apache/ignite/internal/IgniteKernal.java    |    5 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   14 +-
 .../ignite/spi/discovery/DiscoverySpi.java      |   17 +-
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 1481 +++++
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 4792 +++++++++++++++
 .../spi/discovery/tcp/SocketMultiConnector.java |    2 +-
 .../discovery/tcp/TcpClientDiscoverySpi.java    | 1573 -----
 .../tcp/TcpClientDiscoverySpiMBean.java         |  156 -
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  175 +
 .../spi/discovery/tcp/TcpDiscoverySpi.java      | 5777 ++++--------------
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   | 1185 ----
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java |    8 +
 .../internal/GridReleaseTypeSelfTest.java       |   16 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   17 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |   21 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |   21 +-
 ...acheTcpClientDiscoveryMultiThreadedTest.java |    8 +-
 .../IgniteClientDataStructuresAbstractTest.java |    9 +-
 ...ientModesTcpClientDiscoveryAbstractTest.java |   10 +-
 ...unctionExcludeNeighborsAbstractSelfTest.java |    5 +-
 .../GridCacheSyncReplicatedPreloadSelfTest.java |    1 -
 ...pClientDiscoveryMarshallerCheckSelfTest.java |    9 +-
 .../TcpClientDiscoverySpiConfigSelfTest.java    |   39 -
 .../tcp/TcpClientDiscoverySpiSelfTest.java      |   44 +-
 .../tcp/TcpDiscoveryConcurrentStartTest.java    |    4 +-
 .../tcp/TcpDiscoveryMultiThreadedTest.java      |    8 +-
 .../spi/discovery/tcp/TcpDiscoverySelfTest.java |    2 +-
 .../tcp/TcpDiscoverySpiConfigSelfTest.java      |    8 +
 .../testframework/junits/GridAbstractTest.java  |    8 +-
 .../IgniteSpiDiscoverySelfTestSuite.java        |    1 -
 33 files changed, 7693 insertions(+), 7742 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/Ignition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignition.java b/modules/core/src/main/java/org/apache/ignite/Ignition.java
index 3270f5c..35e0b51 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignition.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java
@@ -20,6 +20,7 @@ package org.apache.ignite;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.spi.discovery.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.jetbrains.annotations.*;
 
@@ -138,11 +139,11 @@ public class Ignition {
      * <p>
      * This flag used when node is started if {@link IgniteConfiguration#isClientMode()}
      * is {@code null}. When {@link IgniteConfiguration#isClientMode()} is set this flag is ignored.
-     * It is recommended to use {@link TcpClientDiscoverySpi} on client nodes.
+     * It is recommended to use {@link DiscoverySpi} in client mode too.
      *
      * @param clientMode Client mode flag.
      * @see IgniteConfiguration#isClientMode()
-     * @see TcpClientDiscoverySpi
+     * @see TcpDiscoverySpi#setClientMode(boolean)
      */
     public static void setClientMode(boolean clientMode) {
         IgnitionEx.setClientMode(clientMode);
@@ -153,11 +154,11 @@ public class Ignition {
      * <p>
      * This flag used when node is started if {@link IgniteConfiguration#isClientMode()}
      * is {@code null}. When {@link IgniteConfiguration#isClientMode()} is set this flag is ignored.
-     * It is recommended to use {@link TcpClientDiscoverySpi} on client nodes.
+     * It is recommended to use {@link DiscoverySpi} in client mode too.
      *
      * @return Client mode flag.
      * @see IgniteConfiguration#isClientMode()
-     * @see TcpClientDiscoverySpi
+     * @see TcpDiscoverySpi#setClientMode(boolean)
      */
     public static boolean isClientMode() {
         return IgnitionEx.isClientMode();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
index 13dc30a..8f56372 100644
--- a/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
+++ b/modules/core/src/main/java/org/apache/ignite/cluster/ClusterNode.java
@@ -20,7 +20,7 @@ package org.apache.ignite.cluster;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -243,12 +243,12 @@ public interface ClusterNode {
      * Tests whether or not this node is connected to cluster as a client.
      * <p>
      * Do not confuse client in terms of
-     * discovery {@link TcpClientDiscoverySpi} and client in terms of cache
+     * discovery {@link DiscoverySpi#isClientMode()} and client in terms of cache
      * {@link IgniteConfiguration#isClientMode()}. Cache clients cannot carry data,
      * while topology clients connect to topology in a different way.
      *
      * @return {@code True} if this node is a client node, {@code false} otherwise.
-     * @see TcpClientDiscoverySpi
+     * @see DiscoverySpi#isClientMode()
      * @see IgniteConfiguration#isClientMode()
      * @see Ignition#isClientMode()
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/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 e47d4b1..7ddfd71 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
@@ -1824,10 +1824,10 @@ public class IgniteConfiguration {
 
     /**
      * Gets client mode flag. Client node cannot hold data in the caches. It's recommended to use
-     * {@link TcpClientDiscoverySpi} on client nodes.
+     * {@link DiscoverySpi} in client mode if this property is {@code true}.
      *
      * @return Client mode flag.
-     * @see TcpClientDiscoverySpi
+     * @see TcpDiscoverySpi#setClientMode(boolean)
      */
     public Boolean isClientMode() {
         return clientMode;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/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 84d81d7..d6e3ca4 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
@@ -71,7 +71,6 @@ import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.mxbean.*;
 import org.apache.ignite.plugin.*;
 import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.discovery.tcp.*;
 import org.jetbrains.annotations.*;
 
 import javax.management.*;
@@ -1073,8 +1072,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         if (cfg.getIncludeEventTypes() != null && cfg.getIncludeEventTypes().length != 0)
             perf.add("Disable grid events (remove 'includeEventTypes' from configuration)");
 
-        if (Boolean.TRUE.equals(cfg.isClientMode()) && cfg.getDiscoverySpi() instanceof TcpDiscoverySpi)
-            perf.add("Use TcpClientDiscoverySpi instead of TcpDiscoverySpi to run client node");
+        if (Boolean.TRUE.equals(cfg.isClientMode()) && !cfg.getDiscoverySpi().isClientMode())
+            perf.add("Use TcpDiscoverySpi in client mode for client node");
 
         if (OptimizedMarshaller.available() && !(cfg.getMarshaller() instanceof OptimizedMarshaller))
             perf.add("Enable optimized marshaller (set 'marshaller' to " +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index 728fce6..13b015b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1744,8 +1744,8 @@ public class IgnitionEx {
             }
 
             if (myCfg.isClientMode() == null || !myCfg.isClientMode()) {
-                if (myCfg.getDiscoverySpi() instanceof TcpClientDiscoverySpi) {
-                    throw new IgniteCheckedException("TcpClientDiscoverySpi can be used in client mode only" +
+                if (myCfg.getDiscoverySpi().isClientMode()) {
+                    throw new IgniteCheckedException("DiscoverySpi is in client mode, but node is not in client mode" +
                         "(consider changing 'IgniteConfiguration.clientMode' to 'true').");
                 }
             }
@@ -1815,14 +1815,14 @@ public class IgnitionEx {
          */
         private void initializeDefaultSpi(IgniteConfiguration cfg) {
             if (cfg.getDiscoverySpi() == null) {
+                cfg.setDiscoverySpi(new TcpDiscoverySpi());
+
                 if (cfg.isClientMode() != null && cfg.isClientMode())
-                    cfg.setDiscoverySpi(new TcpClientDiscoverySpi());
-                else
-                    cfg.setDiscoverySpi(new TcpDiscoverySpi());
+                    ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setClientMode(true);
             }
 
-            if (cfg.getDiscoverySpi() instanceof TcpDiscoverySpiAdapter) {
-                TcpDiscoverySpiAdapter tcpDisco = (TcpDiscoverySpiAdapter)cfg.getDiscoverySpi();
+            if (cfg.getDiscoverySpi() instanceof TcpDiscoverySpi) {
+                TcpDiscoverySpi tcpDisco = (TcpDiscoverySpi)cfg.getDiscoverySpi();
 
                 if (tcpDisco.getIpFinder() == null)
                     tcpDisco.setIpFinder(new TcpDiscoveryMulticastIpFinder());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index 7836e0f..4996d16 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -77,11 +78,10 @@ public interface DiscoverySpi extends IgniteSpi {
     /**
      * Sets node attributes and node version which will be distributed in grid during
      * join process. Note that these attributes cannot be changed and set only once.
-     *
-     * @param attrs Map of node attributes.
+     *  @param attrs Map of node attributes.
      * @param ver Product version.
      */
-    public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver);
+    public TcpDiscoverySpi setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver);
 
     /**
      * Sets a listener for discovery events. Refer to
@@ -102,7 +102,7 @@ public interface DiscoverySpi extends IgniteSpi {
      *
      * @param exchange Discovery data exchange handler.
      */
-    public void setDataExchange(DiscoverySpiDataExchange exchange);
+    public TcpDiscoverySpi setDataExchange(DiscoverySpiDataExchange exchange);
 
     /**
      * Sets discovery metrics provider. Use metrics provided by
@@ -111,7 +111,7 @@ public interface DiscoverySpi extends IgniteSpi {
      *
      * @param metricsProvider Provider of metrics data.
      */
-    public void setMetricsProvider(DiscoveryMetricsProvider metricsProvider);
+    public TcpDiscoverySpi setMetricsProvider(DiscoveryMetricsProvider metricsProvider);
 
     /**
      * Tells discovery SPI to disconnect from topology. This is very close to calling
@@ -152,4 +152,11 @@ public interface DiscoverySpi extends IgniteSpi {
      * @param nodeId Node ID.
      */
     public void failNode(UUID nodeId);
+
+    /**
+     * Whether or not discovery is in client mode.
+     *
+     * @return {@code true} if node is in client mode.
+     */
+    public boolean isClientMode();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/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
new file mode 100644
index 0000000..455b2af
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -0,0 +1,1481 @@
+/*
+ * 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.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.internal.*;
+import org.apache.ignite.spi.discovery.tcp.messages.*;
+import org.jetbrains.annotations.*;
+import org.jsr166.*;
+
+import java.io.*;
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.EventType.*;
+import static org.apache.ignite.internal.events.DiscoveryCustomEvent.*;
+
+/**
+ *
+ */
+class ClientImpl extends TcpDiscoveryImpl {
+    /** */
+    private static final Object JOIN_TIMEOUT = "JOIN_TIMEOUT";
+
+    /** */
+    private static final Object SPI_STOP = "SPI_STOP";
+
+    /** */
+    private static final Object SPI_RECONNECT_FAILED = "SPI_RECONNECT_FAILED";
+
+    /** Remote nodes. */
+    private final ConcurrentMap<UUID, TcpDiscoveryNode> rmtNodes = new ConcurrentHashMap8<>();
+
+    /** Topology history. */
+    private final NavigableMap<Long, Collection<ClusterNode>> topHist = new TreeMap<>();
+
+    /** Remote nodes. */
+    private final ConcurrentMap<UUID, GridFutureAdapter<Boolean>> pingFuts = new ConcurrentHashMap8<>();
+
+    /** Socket writer. */
+    private SocketWriter sockWriter;
+
+    /** */
+    private SocketReader sockReader;
+
+    /** */
+    private boolean segmented;
+
+    /** Last message ID. */
+    private volatile IgniteUuid lastMsgId;
+
+    /** Current topology version. */
+    private volatile long topVer;
+
+    /** Join error. Contains error what occurs on join process. */
+    private IgniteSpiException joinErr;
+
+    /** Joined latch. */
+    private final CountDownLatch joinLatch = new CountDownLatch(1);
+
+    /** Left latch. */
+    private final CountDownLatch leaveLatch = new CountDownLatch(1);
+
+    /** */
+    private final Timer timer = new Timer("TcpDiscoverySpi.timer");
+
+    /** */
+    protected MessageWorker msgWorker;
+
+    /**
+     * @param adapter Adapter.
+     */
+    ClientImpl(TcpDiscoverySpi adapter) {
+        super(adapter);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void dumpDebugInfo(IgniteLogger log) {
+        StringBuilder b = new StringBuilder(U.nl());
+
+        b.append(">>>").append(U.nl());
+        b.append(">>>").append("Dumping discovery SPI debug info.").append(U.nl());
+        b.append(">>>").append(U.nl());
+
+        b.append("Local node ID: ").append(getLocalNodeId()).append(U.nl()).append(U.nl());
+        b.append("Local node: ").append(locNode).append(U.nl()).append(U.nl());
+
+        b.append("Internal threads: ").append(U.nl());
+
+        b.append("    Message worker: ").append(threadStatus(msgWorker)).append(U.nl());
+        b.append("    Socket reader: ").append(threadStatus(sockReader)).append(U.nl());
+        b.append("    Socket writer: ").append(threadStatus(sockWriter)).append(U.nl());
+        b.append("    Socket timeout worker: ").append(threadStatus(adapter.sockTimeoutWorker)).append(U.nl());
+
+        b.append(U.nl());
+
+        b.append("Nodes: ").append(U.nl());
+
+        for (ClusterNode node : allVisibleNodes())
+            b.append("    ").append(node.id()).append(U.nl());
+
+        b.append(U.nl());
+
+        b.append("Stats: ").append(adapter.stats).append(U.nl());
+
+        U.quietAndInfo(log, b.toString());
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSpiState() {
+
+        if (sockWriter.isOnline())
+            return "connected";
+
+        return "disconnected";
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getMessageWorkerQueueSize() {
+        return msgWorker.queueSize();
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID getCoordinator() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
+        adapter.initLocalNode(0, true);
+
+        locNode = adapter.locNode;
+
+        sockWriter = new SocketWriter();
+        sockWriter.start();
+
+        sockReader = new SocketReader();
+        sockReader.start();
+
+        msgWorker = new MessageWorker();
+        msgWorker.start();
+
+        try {
+            joinLatch.await();
+
+            if (joinErr != null)
+                throw joinErr;
+        }
+        catch (InterruptedException e) {
+            throw new IgniteSpiException("Thread has been interrupted.", e);
+        }
+
+        timer.schedule(new HeartbeatSender(), adapter.hbFreq, adapter.hbFreq);
+
+        adapter.printStartInfo();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void spiStop() throws IgniteSpiException {
+        timer.cancel();
+
+        if (msgWorker != null && msgWorker.isAlive()) { // Should always be alive
+            msgWorker.addMessage(SPI_STOP);
+
+            try {
+                if (!leaveLatch.await(adapter.netTimeout, MILLISECONDS))
+                    U.warn(log, "Failed to left node: timeout [nodeId=" + locNode + ']');
+            }
+            catch (InterruptedException ignored) {
+
+            }
+        }
+
+        for (GridFutureAdapter<Boolean> fut : pingFuts.values())
+            fut.onDone(false);
+
+        rmtNodes.clear();
+
+        U.interrupt(msgWorker);
+        U.interrupt(sockWriter);
+        U.interrupt(sockReader);
+
+        U.join(msgWorker, log);
+        U.join(sockWriter, log);
+        U.join(sockReader, log);
+
+        adapter.printStopInfo();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<ClusterNode> getRemoteNodes() {
+        return U.arrayList(rmtNodes.values(), TcpDiscoveryNodesRing.VISIBLE_NODES);
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
+        if (getLocalNodeId().equals(nodeId))
+            return locNode;
+
+        TcpDiscoveryNode node = rmtNodes.get(nodeId);
+
+        return node != null && node.visible() ? node : null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean pingNode(@NotNull final UUID nodeId) {
+        if (nodeId.equals(getLocalNodeId()))
+            return true;
+
+        TcpDiscoveryNode node = rmtNodes.get(nodeId);
+
+        if (node == null || !node.visible())
+            return false;
+
+        GridFutureAdapter<Boolean> fut = pingFuts.get(nodeId);
+
+        if (fut == null) {
+            fut = new GridFutureAdapter<>();
+
+            GridFutureAdapter<Boolean> oldFut = pingFuts.putIfAbsent(nodeId, fut);
+
+            if (oldFut != null)
+                fut = oldFut;
+            else {
+                if (adapter.getSpiContext().isStopping()) {
+                    if (pingFuts.remove(nodeId, fut))
+                        fut.onDone(false);
+
+                    return false;
+                }
+
+                final GridFutureAdapter<Boolean> finalFut = fut;
+
+                timer.schedule(new TimerTask() {
+                    @Override public void run() {
+                        if (pingFuts.remove(nodeId, finalFut))
+                            finalFut.onDone(false);
+                    }
+                }, adapter.netTimeout);
+
+                sockWriter.sendMessage(new TcpDiscoveryClientPingRequest(getLocalNodeId(), nodeId));
+            }
+        }
+
+        try {
+            return fut.get();
+        }
+        catch (IgniteInterruptedCheckedException ignored) {
+            return false;
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException(e); // Should newer occur
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void disconnect() throws IgniteSpiException {
+        U.interrupt(msgWorker);
+        U.interrupt(sockWriter);
+        U.interrupt(sockReader);
+
+        U.join(msgWorker, log);
+        U.join(sockWriter, log);
+        U.join(sockReader, log);
+
+        leaveLatch.countDown();
+        joinLatch.countDown();
+
+        adapter.getSpiContext().deregisterPorts();
+
+        Collection<ClusterNode> rmts = getRemoteNodes();
+
+        // This is restart/disconnection and remote nodes are not empty.
+        // We need to fire FAIL event for each.
+        DiscoverySpiListener lsnr = adapter.lsnr;
+
+        if (lsnr != null) {
+            for (ClusterNode n : rmts) {
+                rmtNodes.remove(n.id());
+
+                Collection<ClusterNode> top = updateTopologyHistory(topVer + 1);
+
+                lsnr.onDiscovery(EVT_NODE_FAILED, topVer, n, top, new TreeMap<>(topHist), null);
+            }
+        }
+
+        rmtNodes.clear();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
+        if (segmented)
+            throw new IgniteException("Failed to send custom message: client is disconnected");
+
+        try {
+            sockWriter.sendMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt,
+                adapter.marsh.marshal(evt)));
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteSpiException("Failed to marshal custom event: " + evt, e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void failNode(UUID nodeId) {
+        ClusterNode node = rmtNodes.get(nodeId);
+
+        if (node != null) {
+            TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
+                node.id(), node.order());
+
+            msgWorker.addMessage(msg);
+        }
+    }
+
+    /**
+     * @return Opened socket or {@code null} if timeout.
+     * @see TcpDiscoverySpi#joinTimeout
+     */
+    @SuppressWarnings("BusyWait")
+    @Nullable private Socket joinTopology(boolean recon) throws IgniteSpiException, InterruptedException {
+        Collection<InetSocketAddress> addrs = null;
+
+        long startTime = U.currentTimeMillis();
+
+        while (true) {
+            if (Thread.currentThread().isInterrupted())
+                throw new InterruptedException();
+
+            while (addrs == null || addrs.isEmpty()) {
+                addrs = adapter.resolvedAddresses();
+
+                if (!F.isEmpty(addrs)) {
+                    if (log.isDebugEnabled())
+                        log.debug("Resolved addresses from IP finder: " + addrs);
+                }
+                else {
+                    U.warn(log, "No addresses registered in the IP finder (will retry in 2000ms): " + adapter.ipFinder);
+
+                    if (adapter.joinTimeout > 0 && (U.currentTimeMillis() - startTime) > adapter.joinTimeout)
+                        return null;
+
+                    Thread.sleep(2000);
+                }
+            }
+
+            Collection<InetSocketAddress> addrs0 = new ArrayList<>(addrs);
+
+            Iterator<InetSocketAddress> it = addrs.iterator();
+
+            while (it.hasNext()) {
+                if (Thread.currentThread().isInterrupted())
+                    throw new InterruptedException();
+
+                InetSocketAddress addr = it.next();
+
+                Socket sock = null;
+
+                try {
+                    long ts = U.currentTimeMillis();
+
+                    IgniteBiTuple<Socket, UUID> t = initConnection(addr);
+
+                    sock = t.get1();
+
+                    UUID rmtNodeId = t.get2();
+
+                    adapter.stats.onClientSocketInitialized(U.currentTimeMillis() - ts);
+
+                    locNode.clientRouterNodeId(rmtNodeId);
+
+                    TcpDiscoveryAbstractMessage msg = recon ?
+                        new TcpDiscoveryClientReconnectMessage(getLocalNodeId(), rmtNodeId,
+                            lastMsgId) :
+                        new TcpDiscoveryJoinRequestMessage(locNode, adapter.collectExchangeData(getLocalNodeId()));
+
+                    msg.client(true);
+
+                    adapter.writeToSocket(sock, msg);
+
+                    int res = adapter.readReceipt(sock, adapter.ackTimeout);
+
+                    switch (res) {
+                        case RES_OK:
+                            return sock;
+
+                        case RES_CONTINUE_JOIN:
+                        case RES_WAIT:
+                            U.closeQuiet(sock);
+
+                            break;
+
+                        default:
+                            if (log.isDebugEnabled())
+                                log.debug("Received unexpected response to join request: " + res);
+
+                            U.closeQuiet(sock);
+                    }
+                }
+                catch (IOException | IgniteCheckedException e) {
+                    if (log.isDebugEnabled())
+                        U.error(log, "Failed to establish connection with address: " + addr, e);
+
+                    U.closeQuiet(sock);
+
+                    it.remove();
+                }
+            }
+
+            if (addrs.isEmpty()) {
+                U.warn(log, "Failed to connect to any address from IP finder (will retry to join topology " +
+                    "in 2000ms): " + addrs0);
+
+                if (adapter.joinTimeout > 0 && (U.currentTimeMillis() - startTime) > adapter.joinTimeout)
+                    return null;
+
+                Thread.sleep(2000);
+            }
+        }
+    }
+
+    /**
+     * @param topVer New topology version.
+     * @return Latest topology snapshot.
+     */
+    private NavigableSet<ClusterNode> updateTopologyHistory(long topVer) {
+        this.topVer = topVer;
+
+        NavigableSet<ClusterNode> allNodes = allVisibleNodes();
+
+        if (!topHist.containsKey(topVer)) {
+            assert topHist.isEmpty() || topHist.lastKey() == topVer - 1 :
+                "lastVer=" + topHist.lastKey() + ", newVer=" + topVer;
+
+            topHist.put(topVer, allNodes);
+
+            if (topHist.size() > adapter.topHistSize)
+                topHist.pollFirstEntry();
+
+            assert topHist.lastKey() == topVer;
+            assert topHist.size() <= adapter.topHistSize;
+        }
+
+        return allNodes;
+    }
+
+    /**
+     * @return All nodes.
+     */
+    private NavigableSet<ClusterNode> allVisibleNodes() {
+        NavigableSet<ClusterNode> allNodes = new TreeSet<>();
+
+        for (TcpDiscoveryNode node : rmtNodes.values()) {
+            if (node.visible())
+                allNodes.add(node);
+        }
+
+        allNodes.add(locNode);
+
+        return allNodes;
+    }
+
+    /**
+     * @param addr Address.
+     * @return Remote node ID.
+     * @throws IOException In case of I/O error.
+     * @throws IgniteCheckedException In case of other error.
+     */
+    private IgniteBiTuple<Socket, UUID> initConnection(InetSocketAddress addr) throws IOException, IgniteCheckedException {
+        assert addr != null;
+
+        Socket sock = adapter.openSocket(addr);
+
+        TcpDiscoveryHandshakeRequest req = new TcpDiscoveryHandshakeRequest(getLocalNodeId());
+
+        req.client(true);
+
+        adapter.writeToSocket(sock, req);
+
+        TcpDiscoveryHandshakeResponse res = adapter.readMessage(sock, null, adapter.ackTimeout);
+
+        UUID nodeId = res.creatorNodeId();
+
+        assert nodeId != null;
+        assert !getLocalNodeId().equals(nodeId);
+
+        return F.t(sock, nodeId);
+    }
+
+    /** {@inheritDoc} */
+    @Override void simulateNodeFailure() {
+        U.warn(log, "Simulating client node failure: " + getLocalNodeId());
+
+        U.interrupt(sockWriter);
+        U.interrupt(msgWorker);
+        U.interrupt(adapter.sockTimeoutWorker);
+
+        U.join(sockWriter, log);
+        U.join(msgWorker, log);
+        U.join(adapter.sockTimeoutWorker, log);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void brakeConnection() {
+        U.closeQuiet(msgWorker.currSock);
+    }
+
+    @Override protected IgniteSpiThread workerThread() {
+        return msgWorker;
+    }
+
+    /**
+     * FOR TEST PURPOSE ONLY!
+     */
+    public void waitForClientMessagePrecessed() {
+        Object last = msgWorker.queue.peekLast();
+
+        while (last != null && msgWorker.isAlive() && msgWorker.queue.contains(last)) {
+            try {
+                Thread.sleep(10);
+            }
+            catch (InterruptedException ignored) {
+                Thread.currentThread().interrupt();
+            }
+        }
+    }
+
+    /**
+     * Heartbeat sender.
+     */
+    private class HeartbeatSender extends TimerTask {
+        /** {@inheritDoc} */
+        @Override public void run() {
+            if (!adapter.getSpiContext().isStopping() && sockWriter.isOnline()) {
+                TcpDiscoveryClientHeartbeatMessage msg = new TcpDiscoveryClientHeartbeatMessage(getLocalNodeId(),
+                    adapter.metricsProvider.metrics());
+
+                msg.client(true);
+
+                sockWriter.sendMessage(msg);
+            }
+        }
+    }
+
+    /**
+     * Socket reader.
+     */
+    private class SocketReader extends IgniteSpiThread {
+        /** */
+        private final Object mux = new Object();
+
+        /** */
+        private Socket sock;
+
+        /** */
+        private UUID rmtNodeId;
+
+        /**
+         */
+        protected SocketReader() {
+            super(adapter.ignite().name(), "tcp-client-disco-sock-reader", log);
+        }
+
+        /**
+         * @param sock Socket.
+         * @param rmtNodeId Rmt node id.
+         */
+        public void setSocket(Socket sock, UUID rmtNodeId) {
+            synchronized (mux) {
+                this.sock = sock;
+
+                this.rmtNodeId = rmtNodeId;
+
+                mux.notifyAll();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            while (!isInterrupted()) {
+                Socket sock;
+                UUID rmtNodeId;
+
+                synchronized (mux) {
+                    if (this.sock == null) {
+                        mux.wait();
+
+                        continue;
+                    }
+
+                    sock = this.sock;
+                    rmtNodeId = this.rmtNodeId;
+                }
+
+                try {
+                    InputStream in = new BufferedInputStream(sock.getInputStream());
+
+                    sock.setKeepAlive(true);
+                    sock.setTcpNoDelay(true);
+
+                    while (!isInterrupted()) {
+                        TcpDiscoveryAbstractMessage msg;
+
+                        try {
+                            msg = adapter.marsh.unmarshal(in, U.gridClassLoader());
+                        }
+                        catch (IgniteCheckedException e) {
+                            if (log.isDebugEnabled())
+                                U.error(log, "Failed to read message [sock=" + sock + ", " +
+                                    "locNodeId=" + getLocalNodeId() + ", rmtNodeId=" + rmtNodeId + ']', e);
+
+                            IOException ioEx = X.cause(e, IOException.class);
+
+                            if (ioEx != null)
+                                throw ioEx;
+
+                            ClassNotFoundException clsNotFoundEx = X.cause(e, ClassNotFoundException.class);
+
+                            if (clsNotFoundEx != null)
+                                LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
+                                    "(make sure same versions of all classes are available on all nodes) " +
+                                    "[rmtNodeId=" + rmtNodeId + ", err=" + clsNotFoundEx.getMessage() + ']');
+                            else
+                                LT.error(log, e, "Failed to read message [sock=" + sock + ", locNodeId=" +
+                                    getLocalNodeId() + ", rmtNodeId=" + rmtNodeId + ']');
+
+                            continue;
+                        }
+
+                        msg.senderNodeId(rmtNodeId);
+
+                        if (log.isDebugEnabled())
+                            log.debug("Message has been received: " + msg);
+
+                        adapter.stats.onMessageReceived(msg);
+
+                        if (adapter.ensured(msg))
+                            lastMsgId = msg.id();
+
+                        msgWorker.addMessage(msg);
+                    }
+                }
+                catch (IOException e) {
+                    msgWorker.addMessage(new SocketClosedMessage(sock));
+
+                    if (log.isDebugEnabled())
+                        U.error(log, "Connection failed [sock=" + sock + ", locNodeId=" + getLocalNodeId() + ']', e);
+                }
+                finally {
+                    U.closeQuiet(sock);
+
+                    synchronized (mux) {
+                        if (this.sock == sock) {
+                            this.sock = null;
+                            this.rmtNodeId = null;
+                        }
+                    }
+                }
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private class SocketWriter extends IgniteSpiThread {
+        /** */
+        private final Object mux = new Object();
+
+        /** */
+        private Socket sock;
+
+        /** */
+        private final Queue<TcpDiscoveryAbstractMessage> queue = new ArrayDeque<>();
+
+        /**
+         *
+         */
+        protected SocketWriter() {
+            super(adapter.ignite().name(), "tcp-client-disco-sock-writer", log);
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void sendMessage(TcpDiscoveryAbstractMessage msg) {
+            synchronized (mux) {
+                queue.add(msg);
+
+                mux.notifyAll();
+            }
+        }
+
+        /**
+         * @param sock Socket.
+         */
+        private void setSocket(Socket sock) {
+            synchronized (mux) {
+                this.sock = sock;
+
+                mux.notifyAll();
+            }
+        }
+
+        /**
+         *
+         */
+        public boolean isOnline() {
+            synchronized (mux) {
+                return sock != null;
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            TcpDiscoveryAbstractMessage msg = null;
+
+            while (!Thread.currentThread().isInterrupted()) {
+                Socket sock;
+
+                synchronized (mux) {
+                    sock = this.sock;
+
+                    if (sock == null) {
+                        mux.wait();
+
+                        continue;
+                    }
+
+                    if (msg == null)
+                        msg = queue.poll();
+
+                    if (msg == null) {
+                        mux.wait();
+
+                        continue;
+                    }
+                }
+
+                for (IgniteInClosure<TcpDiscoveryAbstractMessage> msgLsnr : adapter.sendMsgLsnrs)
+                    msgLsnr.apply(msg);
+
+                try {
+                    adapter.writeToSocket(sock, msg);
+
+                    msg = null;
+                }
+                catch (IOException e) {
+                    if (log.isDebugEnabled())
+                        U.error(log, "Failed to send node left message (will stop anyway) [sock=" + sock + ']', e);
+
+                    U.closeQuiet(sock);
+
+                    synchronized (mux) {
+                        if (sock == this.sock)
+                            this.sock = null; // Connection has dead.
+                    }
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to send message: " + msg, e);
+
+                    msg = null;
+                }
+            }
+        }
+    }
+
+    /**
+     *
+     */
+    private class Reconnector extends IgniteSpiThread {
+        /** */
+        private volatile Socket sock;
+
+        /**
+         *
+         */
+        protected Reconnector() {
+            super(adapter.ignite().name(), "tcp-client-disco-msg-worker", log);
+        }
+
+        /**
+         *
+         */
+        public void cancel() {
+            interrupt();
+
+            U.closeQuiet(sock);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            assert !segmented;
+
+            boolean success = false;
+
+            try {
+                sock = joinTopology(true);
+
+                if (sock == null) {
+                    U.error(log, "Failed to reconnect to cluster: timeout.");
+
+                    return;
+                }
+
+                if (isInterrupted())
+                    throw new InterruptedException();
+
+                InputStream in = new BufferedInputStream(sock.getInputStream());
+
+                sock.setKeepAlive(true);
+                sock.setTcpNoDelay(true);
+
+                // Wait for
+                while (!isInterrupted()) {
+                    TcpDiscoveryAbstractMessage msg = adapter.marsh.unmarshal(in, U.gridClassLoader());
+
+                    if (msg instanceof TcpDiscoveryClientReconnectMessage) {
+                        TcpDiscoveryClientReconnectMessage res = (TcpDiscoveryClientReconnectMessage)msg;
+
+                        if (res.creatorNodeId().equals(getLocalNodeId())) {
+                            if (res.success()) {
+                                msgWorker.addMessage(res);
+
+                                success = true;
+                            }
+
+                            break;
+                        }
+                    }
+
+                }
+            }
+            catch (IOException | IgniteCheckedException e) {
+                U.error(log, "Failed to reconnect", e);
+            }
+            finally {
+                if (!success) {
+                    U.closeQuiet(sock);
+
+                    msgWorker.addMessage(SPI_RECONNECT_FAILED);
+                }
+            }
+        }
+    }
+
+    /**
+     * Message worker.
+     */
+    protected class MessageWorker extends IgniteSpiThread {
+        /** Message queue. */
+        private final BlockingDeque<Object> queue = new LinkedBlockingDeque<>();
+
+        /** */
+        private Socket currSock;
+
+        /** Indicates that pending messages are currently processed. */
+        private boolean pending;
+
+        /** */
+        private Reconnector reconnector;
+
+        /**
+         *
+         */
+        private MessageWorker() {
+            super(adapter.ignite().name(), "tcp-client-disco-msg-worker", log);
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("InfiniteLoopStatement")
+        @Override protected void body() throws InterruptedException {
+            adapter.stats.onJoinStarted();
+
+            try {
+                final Socket sock = joinTopology(false);
+
+                if (sock == null) {
+                    joinErr = new IgniteSpiException("Join process timed out");
+
+                    joinLatch.countDown();
+
+                    return;
+                }
+
+                currSock = sock;
+
+                sockWriter.setSocket(sock);
+
+                timer.schedule(new TimerTask() {
+                    @Override public void run() {
+                        if (joinLatch.getCount() > 0)
+                            queue.add(JOIN_TIMEOUT);
+                    }
+                }, adapter.netTimeout);
+
+                sockReader.setSocket(sock, locNode.clientRouterNodeId());
+
+                while (true) {
+                    Object msg = queue.take();
+
+                    if (msg == JOIN_TIMEOUT) {
+                        if (joinLatch.getCount() > 0) {
+                            joinErr = new IgniteSpiException("Join process timed out [sock=" + sock +
+                                ", timeout=" + adapter.netTimeout + ']');
+
+                            joinLatch.countDown();
+
+                            break;
+                        }
+                    }
+                    else if (msg == SPI_STOP) {
+                        assert adapter.getSpiContext().isStopping();
+
+                        if (currSock != null) {
+                            TcpDiscoveryAbstractMessage leftMsg = new TcpDiscoveryNodeLeftMessage(getLocalNodeId());
+
+                            leftMsg.client(true);
+
+                            sockWriter.sendMessage(leftMsg);
+                        }
+                        else
+                            leaveLatch.countDown();
+                    }
+                    else if (msg instanceof SocketClosedMessage) {
+                        if (((SocketClosedMessage)msg).sock == currSock) {
+                            currSock = null;
+
+                            if (joinLatch.getCount() > 0) {
+                                joinErr = new IgniteSpiException("Failed to connect to cluster: socket closed.");
+
+                                joinLatch.countDown();
+
+                                break;
+                            }
+                            else {
+                                if (adapter.getSpiContext().isStopping() || segmented)
+                                    leaveLatch.countDown();
+                                else {
+                                    assert reconnector == null;
+
+                                    final Reconnector reconnector = new Reconnector();
+                                    this.reconnector = reconnector;
+                                    reconnector.start();
+
+                                    timer.schedule(new TimerTask() {
+                                        @Override public void run() {
+                                            if (reconnector.isAlive())
+                                                reconnector.cancel();
+                                        }
+                                    }, adapter.netTimeout);
+                                }
+                            }
+                        }
+                    }
+                    else if (msg == SPI_RECONNECT_FAILED) {
+                        if (!segmented) {
+                            segmented = true;
+
+                            reconnector.cancel();
+                            reconnector.join();
+
+                            notifyDiscovery(EVT_NODE_SEGMENTED, topVer, locNode, allVisibleNodes());
+                        }
+                    }
+                    else {
+                        TcpDiscoveryAbstractMessage discoMsg = (TcpDiscoveryAbstractMessage)msg;
+
+                        if (joinLatch.getCount() > 0) {
+                            IgniteSpiException err = null;
+
+                            if (discoMsg instanceof TcpDiscoveryDuplicateIdMessage)
+                                err = adapter.duplicateIdError((TcpDiscoveryDuplicateIdMessage)msg);
+                            else if (discoMsg instanceof TcpDiscoveryAuthFailedMessage)
+                                err = adapter.authenticationFailedError((TcpDiscoveryAuthFailedMessage)msg);
+                            else if (discoMsg instanceof TcpDiscoveryCheckFailedMessage)
+                                err = adapter.checkFailedError((TcpDiscoveryCheckFailedMessage)msg);
+
+                            if (err != null) {
+                                joinErr = err;
+
+                                joinLatch.countDown();
+
+                                break;
+                            }
+                        }
+
+                        processDiscoveryMessage((TcpDiscoveryAbstractMessage)msg);
+                    }
+                }
+            }
+            finally {
+                U.closeQuiet(currSock);
+
+                if (joinLatch.getCount() > 0) {
+                    // This should not occurs.
+                    joinErr = new IgniteSpiException("Some error occurs in joinig process");
+
+                    joinLatch.countDown();
+                }
+
+                if (reconnector != null) {
+                    reconnector.cancel();
+
+                    reconnector.join();
+                }
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        protected void processDiscoveryMessage(TcpDiscoveryAbstractMessage msg) {
+            assert msg != null;
+            assert msg.verified() || msg.senderNodeId() == null;
+
+            adapter.stats.onMessageProcessingStarted(msg);
+
+            if (msg instanceof TcpDiscoveryNodeAddedMessage)
+                processNodeAddedMessage((TcpDiscoveryNodeAddedMessage)msg);
+            else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage)
+                processNodeAddFinishedMessage((TcpDiscoveryNodeAddFinishedMessage)msg);
+            else if (msg instanceof TcpDiscoveryNodeLeftMessage)
+                processNodeLeftMessage((TcpDiscoveryNodeLeftMessage)msg);
+            else if (msg instanceof TcpDiscoveryNodeFailedMessage)
+                processNodeFailedMessage((TcpDiscoveryNodeFailedMessage)msg);
+            else if (msg instanceof TcpDiscoveryHeartbeatMessage)
+                processHeartbeatMessage((TcpDiscoveryHeartbeatMessage)msg);
+            else if (msg instanceof TcpDiscoveryClientReconnectMessage)
+                processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg);
+            else if (msg instanceof TcpDiscoveryCustomEventMessage)
+                processCustomMessage((TcpDiscoveryCustomEventMessage)msg);
+            else if (msg instanceof TcpDiscoveryClientPingResponse)
+                processClientPingResponse((TcpDiscoveryClientPingResponse)msg);
+            else if (msg instanceof TcpDiscoveryPingRequest)
+                processPingRequest();
+
+            adapter.stats.onMessageProcessingFinished(msg);
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processNodeAddedMessage(TcpDiscoveryNodeAddedMessage msg) {
+            if (adapter.getSpiContext().isStopping())
+                return;
+
+            TcpDiscoveryNode node = msg.node();
+
+            UUID newNodeId = node.id();
+
+            if (getLocalNodeId().equals(newNodeId)) {
+                if (joinLatch.getCount() > 0) {
+                    Collection<TcpDiscoveryNode> top = msg.topology();
+
+                    if (top != null) {
+                        adapter.gridStartTime = msg.gridStartTime();
+
+                        for (TcpDiscoveryNode n : top) {
+                            if (n.order() > 0)
+                                n.visible(true);
+
+                            rmtNodes.put(n.id(), n);
+                        }
+
+                        topHist.clear();
+
+                        if (msg.topologyHistory() != null)
+                            topHist.putAll(msg.topologyHistory());
+                    }
+                    else if (log.isDebugEnabled())
+                        log.debug("Discarding node added message with empty topology: " + msg);
+                }
+                else if (log.isDebugEnabled())
+                    log.debug("Discarding node added message (this message has already been processed) " +
+                        "[msg=" + msg + ", locNode=" + locNode + ']');
+            }
+            else {
+                boolean topChanged = rmtNodes.putIfAbsent(newNodeId, node) == null;
+
+                if (topChanged) {
+                    if (log.isDebugEnabled())
+                        log.debug("Added new node to topology: " + node);
+
+                    Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
+
+                    if (data != null)
+                        adapter.onExchange(newNodeId, newNodeId, data, null);
+                }
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processNodeAddFinishedMessage(TcpDiscoveryNodeAddFinishedMessage msg) {
+            if (adapter.getSpiContext().isStopping())
+                return;
+
+            if (getLocalNodeId().equals(msg.nodeId())) {
+                if (joinLatch.getCount() > 0) {
+                    Map<UUID, Map<Integer, byte[]>> dataMap = msg.clientDiscoData();
+
+                    if (dataMap != null) {
+                        for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
+                            adapter.onExchange(getLocalNodeId(), entry.getKey(), entry.getValue(), null);
+                    }
+
+                    locNode.setAttributes(msg.clientNodeAttributes());
+                    locNode.visible(true);
+
+                    long topVer = msg.topologyVersion();
+
+                    locNode.order(topVer);
+
+                    notifyDiscovery(EVT_NODE_JOINED, topVer, locNode, updateTopologyHistory(topVer));
+
+                    joinErr = null;
+
+                    joinLatch.countDown();
+
+                    adapter.stats.onJoinFinished();
+                }
+                else if (log.isDebugEnabled())
+                    log.debug("Discarding node add finished message (this message has already been processed) " +
+                        "[msg=" + msg + ", locNode=" + locNode + ']');
+            }
+            else {
+                TcpDiscoveryNode node = rmtNodes.get(msg.nodeId());
+
+                if (node == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node add finished message since node is not found [msg=" + msg + ']');
+
+                    return;
+                }
+
+                long topVer = msg.topologyVersion();
+
+                node.order(topVer);
+                node.visible(true);
+
+                if (adapter.locNodeVer.equals(node.version()))
+                    node.version(adapter.locNodeVer);
+
+                NavigableSet<ClusterNode> top = updateTopologyHistory(topVer);
+
+                if (!pending && joinLatch.getCount() > 0) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node add finished message (join process is not finished): " + msg);
+
+                    return;
+                }
+
+                notifyDiscovery(EVT_NODE_JOINED, topVer, node, top);
+
+                adapter.stats.onNodeJoined();
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processNodeLeftMessage(TcpDiscoveryNodeLeftMessage msg) {
+            if (getLocalNodeId().equals(msg.creatorNodeId())) {
+                if (log.isDebugEnabled())
+                    log.debug("Received node left message for local node: " + msg);
+
+                leaveLatch.countDown();
+            }
+            else {
+                if (adapter.getSpiContext().isStopping())
+                    return;
+
+                TcpDiscoveryNode node = rmtNodes.remove(msg.creatorNodeId());
+
+                if (node == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node left message since node is not found [msg=" + msg + ']');
+
+                    return;
+                }
+
+                NavigableSet<ClusterNode> top = updateTopologyHistory(msg.topologyVersion());
+
+                if (!pending && joinLatch.getCount() > 0) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node left message (join process is not finished): " + msg);
+
+                    return;
+                }
+
+                notifyDiscovery(EVT_NODE_LEFT, msg.topologyVersion(), node, top);
+
+                adapter.stats.onNodeLeft();
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) {
+            if (adapter.getSpiContext().isStopping()) {
+                if (!getLocalNodeId().equals(msg.creatorNodeId()) && getLocalNodeId().equals(msg.failedNodeId())) {
+                    if (leaveLatch.getCount() > 0) {
+                        log.debug("Remote node fail this node while node is stopping [locNode=" + getLocalNodeId()
+                            + ", rmtNode=" + msg.creatorNodeId() + ']');
+
+                        leaveLatch.countDown();
+                    }
+                }
+
+                return;
+            }
+
+            if (!getLocalNodeId().equals(msg.creatorNodeId())) {
+                TcpDiscoveryNode node = rmtNodes.remove(msg.failedNodeId());
+
+                if (node == null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node failed message since node is not found [msg=" + msg + ']');
+
+                    return;
+                }
+
+                NavigableSet<ClusterNode> top = updateTopologyHistory(msg.topologyVersion());
+
+                if (!pending && joinLatch.getCount() > 0) {
+                    if (log.isDebugEnabled())
+                        log.debug("Discarding node failed message (join process is not finished): " + msg);
+
+                    return;
+                }
+
+                notifyDiscovery(EVT_NODE_FAILED, msg.topologyVersion(), node, top);
+
+                adapter.stats.onNodeFailed();
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) {
+            if (adapter.getSpiContext().isStopping())
+                return;
+
+            if (getLocalNodeId().equals(msg.creatorNodeId())) {
+                assert msg.senderNodeId() != null;
+
+                if (log.isDebugEnabled())
+                    log.debug("Received heartbeat response: " + msg);
+            }
+            else {
+                long tstamp = U.currentTimeMillis();
+
+                if (msg.hasMetrics()) {
+                    for (Map.Entry<UUID, TcpDiscoveryHeartbeatMessage.MetricsSet> e : msg.metrics().entrySet()) {
+                        UUID nodeId = e.getKey();
+
+                        TcpDiscoveryHeartbeatMessage.MetricsSet metricsSet = e.getValue();
+
+                        Map<Integer, CacheMetrics> cacheMetrics = msg.hasCacheMetrics() ?
+                            msg.cacheMetrics().get(nodeId) : Collections.<Integer, CacheMetrics>emptyMap();
+
+                        updateMetrics(nodeId, metricsSet.metrics(), cacheMetrics, tstamp);
+
+                        for (T2<UUID, ClusterMetrics> t : metricsSet.clientMetrics())
+                            updateMetrics(t.get1(), t.get2(), cacheMetrics, tstamp);
+                    }
+                }
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processClientReconnectMessage(TcpDiscoveryClientReconnectMessage msg) {
+            if (adapter.getSpiContext().isStopping())
+                return;
+
+            if (getLocalNodeId().equals(msg.creatorNodeId())) {
+                assert msg.success();
+
+                currSock = reconnector.sock;
+
+                sockWriter.setSocket(currSock);
+                sockReader.setSocket(currSock, locNode.clientRouterNodeId());
+
+                reconnector = null;
+
+                pending = true;
+
+                try {
+                    for (TcpDiscoveryAbstractMessage pendingMsg : msg.pendingMessages())
+                        processDiscoveryMessage(pendingMsg);
+                }
+                finally {
+                    pending = false;
+                }
+            }
+            else if (log.isDebugEnabled())
+                log.debug("Discarding reconnect message for another client: " + msg);
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) {
+            if (msg.verified() && joinLatch.getCount() == 0) {
+                DiscoverySpiListener lsnr = adapter.lsnr;
+
+                if (lsnr != null) {
+                    UUID nodeId = msg.creatorNodeId();
+
+                    TcpDiscoveryNode node = nodeId.equals(getLocalNodeId()) ? locNode : rmtNodes.get(nodeId);
+
+                    if (node != null && node.visible()) {
+                        try {
+                            DiscoverySpiCustomMessage msgObj = msg.message(adapter.marsh);
+
+                            notifyDiscovery(EVT_DISCOVERY_CUSTOM_EVT, topVer, node, allVisibleNodes(), msgObj);
+                        }
+                        catch (Throwable e) {
+                            U.error(log, "Failed to unmarshal discovery custom message.", e);
+                        }
+                    }
+                    else if (log.isDebugEnabled())
+                        log.debug("Received metrics from unknown node: " + nodeId);
+                }
+            }
+        }
+
+        /**
+         * @param msg Message.
+         */
+        private void processClientPingResponse(TcpDiscoveryClientPingResponse msg) {
+            GridFutureAdapter<Boolean> fut = pingFuts.remove(msg.nodeToPing());
+
+            if (fut != null)
+                fut.onDone(msg.result());
+        }
+
+        /**
+         * Router want to ping this client.
+         */
+        private void processPingRequest() {
+            TcpDiscoveryPingResponse res = new TcpDiscoveryPingResponse(getLocalNodeId());
+
+            res.client(true);
+
+            sockWriter.sendMessage(res);
+        }
+
+        /**
+         * @param nodeId Node ID.
+         * @param metrics Metrics.
+         * @param cacheMetrics Cache metrics.
+         * @param tstamp Timestamp.
+         */
+        private void updateMetrics(UUID nodeId,
+            ClusterMetrics metrics,
+            Map<Integer, CacheMetrics> cacheMetrics,
+            long tstamp)
+        {
+            assert nodeId != null;
+            assert metrics != null;
+            assert cacheMetrics != null;
+
+            TcpDiscoveryNode node = nodeId.equals(getLocalNodeId()) ? locNode : rmtNodes.get(nodeId);
+
+            if (node != null && node.visible()) {
+                node.setMetrics(metrics);
+                node.setCacheMetrics(cacheMetrics);
+
+                node.lastUpdateTime(tstamp);
+
+                notifyDiscovery(EVT_NODE_METRICS_UPDATED, topVer, node, allVisibleNodes());
+            }
+            else if (log.isDebugEnabled())
+                log.debug("Received metrics from unknown node: " + nodeId);
+        }
+
+        /**
+         * @param type Event type.
+         * @param topVer Topology version.
+         * @param node Node.
+         * @param top Topology snapshot.
+         */
+        private void notifyDiscovery(int type, long topVer, ClusterNode node, NavigableSet<ClusterNode> top) {
+            notifyDiscovery(type, topVer, node, top, null);
+        }
+
+        /**
+         * @param type Event type.
+         * @param topVer Topology version.
+         * @param node Node.
+         * @param top Topology snapshot.
+         */
+        private void notifyDiscovery(int type, long topVer, ClusterNode node, NavigableSet<ClusterNode> top,
+            @Nullable DiscoverySpiCustomMessage data) {
+            DiscoverySpiListener lsnr = adapter.lsnr;
+
+            if (lsnr != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Discovery notification [node=" + node + ", type=" + U.gridEventName(type) +
+                        ", topVer=" + topVer + ']');
+
+                lsnr.onDiscovery(type, topVer, node, top, new TreeMap<>(topHist), data);
+            }
+            else if (log.isDebugEnabled())
+                log.debug("Skipped discovery notification [node=" + node + ", type=" + U.gridEventName(type) +
+                    ", topVer=" + topVer + ']');
+        }
+
+        /**
+         * @param msg Message.
+         */
+        public void addMessage(Object msg) {
+            queue.add(msg);
+        }
+
+        /**
+         *
+         */
+        public int queueSize() {
+            return queue.size();
+        }
+    }
+
+    /**
+     *
+     */
+    private static class SocketClosedMessage {
+        /** */
+        private final Socket sock;
+
+        /**
+         * @param sock Socket.
+         */
+        private SocketClosedMessage(Socket sock) {
+            this.sock = sock;
+        }
+    }
+}


[03/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Merge TcpDiscoverySpi and TcpClientDiscoverySpi

Posted by se...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/SocketMultiConnector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/SocketMultiConnector.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/SocketMultiConnector.java
index b988ceb..698735e 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/SocketMultiConnector.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/SocketMultiConnector.java
@@ -44,7 +44,7 @@ class SocketMultiConnector implements AutoCloseable {
      * @param addrs Addresses.
      * @param retryCnt Retry count.
      */
-    SocketMultiConnector(final TcpDiscoverySpiAdapter spi, Collection<InetSocketAddress> addrs,
+    SocketMultiConnector(final TcpDiscoverySpi spi, Collection<InetSocketAddress> addrs,
         final int retryCnt) {
         connInProgress = addrs.size();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
deleted file mode 100644
index 52c9016..0000000
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
+++ /dev/null
@@ -1,1573 +0,0 @@
-/*
- * 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.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.discovery.*;
-import org.apache.ignite.spi.discovery.tcp.internal.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
-import org.apache.ignite.spi.discovery.tcp.messages.*;
-import org.jetbrains.annotations.*;
-import org.jsr166.*;
-
-import java.io.*;
-import java.net.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-import static java.util.concurrent.TimeUnit.*;
-import static org.apache.ignite.events.EventType.*;
-import static org.apache.ignite.internal.events.DiscoveryCustomEvent.*;
-import static org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryHeartbeatMessage.*;
-
-/**
- * Client discovery SPI implementation that uses TCP/IP for node discovery.
- * <p>
- * This discovery SPI requires at least one server node configured with
- * {@link TcpDiscoverySpi}. It will try to connect to random IP taken from
- * {@link TcpDiscoveryIpFinder} which should point to one of these server
- * nodes and will maintain connection only with this node (will not enter the ring).
- * If this connection is broken, it will try to reconnect using addresses from
- * the same IP finder.
- *
- * <h1 class="header">Configuration</h1>
- * <h2 class="header">Mandatory</h2>
- * There are no mandatory configuration parameters.
- * <h2 class="header">Optional</h2>
- * The following configuration parameters are optional:
- * <ul>
- * <li>IP finder to share info about nodes IP addresses
- * (see {@link #setIpFinder(TcpDiscoveryIpFinder)}).
- * See the following IP finder implementations for details on configuration:
- * <ul>
- * <li>{@link org.apache.ignite.spi.discovery.tcp.ipfinder.sharedfs.TcpDiscoverySharedFsIpFinder}</li>
- * <li>{@ignitelink org.apache.ignite.spi.discovery.tcp.ipfinder.s3.TcpDiscoveryS3IpFinder}</li>
- * <li>{@link org.apache.ignite.spi.discovery.tcp.ipfinder.jdbc.TcpDiscoveryJdbcIpFinder}</li>
- * <li>{@link org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder}</li>
- * <li>{@link TcpDiscoveryMulticastIpFinder} - default</li>
- * </ul>
- * </li>
- * </ul>
- * <ul>
- * <li>Local address (see {@link #setLocalAddress(String)})</li>
- * <li>Heartbeat frequency (see {@link #setHeartbeatFrequency(long)})</li>
- * <li>Network timeout (see {@link #setNetworkTimeout(long)})</li>
- * <li>Socket timeout (see {@link #setSocketTimeout(long)})</li>
- * <li>Message acknowledgement timeout (see {@link #setAckTimeout(long)})</li>
- * <li>Join timeout (see {@link #setJoinTimeout(long)})</li>
- * <li>Thread priority for threads started by SPI (see {@link #setThreadPriority(int)})</li>
- * </ul>
- * <h2 class="header">Java Example</h2>
- * <pre name="code" class="java">
- * TcpClientDiscoverySpi spi = new TcpClientDiscoverySpi();
- *
- * TcpDiscoveryVmIpFinder finder =
- *     new GridTcpDiscoveryVmIpFinder();
- *
- * spi.setIpFinder(finder);
- *
- * IgniteConfiguration cfg = new IgniteConfiguration();
- *
- * // Override default discovery SPI.
- * cfg.setDiscoverySpi(spi);
- *
- * // Start grid.
- * Ignition.start(cfg);
- * </pre>
- * <h2 class="header">Spring Example</h2>
- * TcpClientDiscoverySpi can be configured from Spring XML configuration file:
- * <pre name="code" class="xml">
- * &lt;bean id="grid.custom.cfg" class="org.apache.ignite.configuration.IgniteConfiguration" singleton="true"&gt;
- *         ...
- *         &lt;property name="discoverySpi"&gt;
- *             &lt;bean class="org.apache.ignite.spi.discovery.tcp.TcpClientDiscoverySpi"&gt;
- *                 &lt;property name="ipFinder"&gt;
- *                     &lt;bean class="org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder" /&gt;
- *                 &lt;/property&gt;
- *             &lt;/bean&gt;
- *         &lt;/property&gt;
- *         ...
- * &lt;/bean&gt;
- * </pre>
- * <p>
- * <img src="http://ignite.incubator.apache.org/images/spring-small.png">
- * <br>
- * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
- * @see DiscoverySpi
- */
-@SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-@IgniteSpiMultipleInstancesSupport(true)
-@DiscoverySpiOrderSupport(true)
-@DiscoverySpiHistorySupport(true)
-public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpClientDiscoverySpiMBean {
-    /** Default socket operations timeout in milliseconds (value is <tt>700ms</tt>). */
-    public static final long DFLT_SOCK_TIMEOUT = 700;
-
-    /** Default timeout for receiving message acknowledgement in milliseconds (value is <tt>700ms</tt>). */
-    public static final long DFLT_ACK_TIMEOUT = 700;
-
-    /** */
-    private static final Object JOIN_TIMEOUT = "JOIN_TIMEOUT";
-
-    /** */
-    private static final Object SPI_STOP = "SPI_STOP";
-
-    /** */
-    private static final Object SPI_RECONNECT_FAILED = "SPI_RECONNECT_FAILED";
-
-    /** Remote nodes. */
-    private final ConcurrentMap<UUID, TcpDiscoveryNode> rmtNodes = new ConcurrentHashMap8<>();
-
-    /** Topology history. */
-    private final NavigableMap<Long, Collection<ClusterNode>> topHist = new TreeMap<>();
-
-    /** Remote nodes. */
-    private final ConcurrentMap<UUID, GridFutureAdapter<Boolean>> pingFuts = new ConcurrentHashMap8<>();
-
-    /** Socket writer. */
-    private SocketWriter sockWriter;
-
-    /** */
-    private SocketReader sockReader;
-
-    /** */
-    private boolean segmented;
-
-    /** Last message ID. */
-    private volatile IgniteUuid lastMsgId;
-
-    /** Current topology version. */
-    private volatile long topVer;
-
-    /** Join error. Contains error what occurs on join process. */
-    private IgniteSpiException joinErr;
-
-    /** Joined latch. */
-    private final CountDownLatch joinLatch = new CountDownLatch(1);
-
-    /** Left latch. */
-    private final CountDownLatch leaveLatch = new CountDownLatch(1);
-
-    /** */
-    private final Timer timer = new Timer("TcpClientDiscoverySpi.timer");
-
-    /** */
-    protected MessageWorker msgWorker;
-
-    /**
-     * Default constructor.
-     */
-    public TcpClientDiscoverySpi() {
-        ackTimeout = DFLT_ACK_TIMEOUT;
-        sockTimeout = DFLT_SOCK_TIMEOUT;
-    }
-
-    /** {@inheritDoc} */
-    @Override public int getMessageWorkerQueueSize() {
-        return msgWorker.queueSize();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        startStopwatch();
-
-        checkParameters();
-
-        assertParameter(threadPri > 0, "threadPri > 0");
-
-        if (log.isDebugEnabled()) {
-            log.debug(configInfo("localHost", locHost.getHostAddress()));
-            log.debug(configInfo("threadPri", threadPri));
-            log.debug(configInfo("networkTimeout", netTimeout));
-            log.debug(configInfo("sockTimeout", sockTimeout));
-            log.debug(configInfo("ackTimeout", ackTimeout));
-            log.debug(configInfo("ipFinder", ipFinder));
-            log.debug(configInfo("heartbeatFreq", hbFreq));
-        }
-
-        // Warn on odd network timeout.
-        if (netTimeout < 3000)
-            U.warn(log, "Network timeout is too low (at least 3000 ms recommended): " + netTimeout);
-
-        registerMBean(gridName, this, TcpClientDiscoverySpiMBean.class);
-
-        try {
-            locHost = U.resolveLocalHost(locAddr);
-        }
-        catch (IOException e) {
-            throw new IgniteSpiException("Unknown local address: " + locAddr, e);
-        }
-
-        if (ipFinder instanceof TcpDiscoveryMulticastIpFinder) {
-            TcpDiscoveryMulticastIpFinder mcastIpFinder = ((TcpDiscoveryMulticastIpFinder)ipFinder);
-
-            if (mcastIpFinder.getLocalAddress() == null)
-                mcastIpFinder.setLocalAddress(locAddr);
-        }
-
-        IgniteBiTuple<Collection<String>, Collection<String>> addrs;
-
-        try {
-            addrs = U.resolveLocalAddresses(locHost);
-        }
-        catch (IOException | IgniteCheckedException e) {
-            throw new IgniteSpiException("Failed to resolve local host to set of external addresses: " + locHost, e);
-        }
-
-        locNode = new TcpDiscoveryNode(
-            getLocalNodeId(),
-            addrs.get1(),
-            addrs.get2(),
-            0,
-            metricsProvider,
-            locNodeVer);
-
-        locNode.setAttributes(locNodeAttrs);
-        locNode.local(true);
-
-        sockWriter = new SocketWriter();
-        sockWriter.start();
-
-        sockReader = new SocketReader();
-        sockReader.start();
-
-        sockTimeoutWorker = new SocketTimeoutWorker();
-        sockTimeoutWorker.start();
-
-        msgWorker = new MessageWorker();
-        msgWorker.start();
-
-        try {
-            joinLatch.await();
-
-            if (joinErr != null)
-                throw joinErr;
-        }
-        catch (InterruptedException e) {
-            throw new IgniteSpiException("Thread has been interrupted.", e);
-        }
-
-        timer.schedule(new HeartbeatSender(), hbFreq, hbFreq);
-
-        if (log.isDebugEnabled())
-            log.debug(startInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public void spiStop() throws IgniteSpiException {
-        timer.cancel();
-
-        if (msgWorker != null && msgWorker.isAlive()) { // Should always be alive
-            msgWorker.addMessage(SPI_STOP);
-
-            try {
-                if (!leaveLatch.await(netTimeout, MILLISECONDS))
-                    U.warn(log, "Failed to left node: timeout [nodeId=" + locNode + ']');
-            }
-            catch (InterruptedException ignored) {
-
-            }
-        }
-
-        for (GridFutureAdapter<Boolean> fut : pingFuts.values())
-            fut.onDone(false);
-
-        rmtNodes.clear();
-
-        U.interrupt(sockTimeoutWorker);
-        U.interrupt(msgWorker);
-        U.interrupt(sockWriter);
-        U.interrupt(sockReader);
-
-        U.join(msgWorker, log);
-        U.join(sockTimeoutWorker, log);
-        U.join(sockWriter, log);
-        U.join(sockReader, log);
-
-        unregisterMBean();
-
-        if (log.isDebugEnabled())
-            log.debug(stopInfo());
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<ClusterNode> getRemoteNodes() {
-        return U.arrayList(rmtNodes.values(), TcpDiscoveryNodesRing.VISIBLE_NODES);
-    }
-
-    /** {@inheritDoc} */
-    @Nullable @Override public ClusterNode getNode(UUID nodeId) {
-        if (getLocalNodeId().equals(nodeId))
-            return locNode;
-
-        TcpDiscoveryNode node = rmtNodes.get(nodeId);
-
-        return node != null && node.visible() ? node : null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean pingNode(@NotNull final UUID nodeId) {
-        if (nodeId.equals(getLocalNodeId()))
-            return true;
-
-        TcpDiscoveryNode node = rmtNodes.get(nodeId);
-
-        if (node == null || !node.visible())
-            return false;
-
-        GridFutureAdapter<Boolean> fut = pingFuts.get(nodeId);
-
-        if (fut == null) {
-            fut = new GridFutureAdapter<>();
-
-            GridFutureAdapter<Boolean> oldFut = pingFuts.putIfAbsent(nodeId, fut);
-
-            if (oldFut != null)
-                fut = oldFut;
-            else {
-                if (getSpiContext().isStopping()) {
-                    if (pingFuts.remove(nodeId, fut))
-                        fut.onDone(false);
-
-                    return false;
-                }
-
-                final GridFutureAdapter<Boolean> finalFut = fut;
-
-                timer.schedule(new TimerTask() {
-                    @Override public void run() {
-                        if (pingFuts.remove(nodeId, finalFut))
-                            finalFut.onDone(false);
-                    }
-                }, netTimeout);
-
-                sockWriter.sendMessage(new TcpDiscoveryClientPingRequest(getLocalNodeId(), nodeId));
-            }
-        }
-
-        try {
-            return fut.get();
-        }
-        catch (IgniteInterruptedCheckedException ignored) {
-            return false;
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteSpiException(e); // Should newer occur
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void disconnect() throws IgniteSpiException {
-        U.interrupt(msgWorker);
-        U.interrupt(sockWriter);
-        U.interrupt(sockReader);
-
-        U.join(msgWorker, log);
-        U.join(sockWriter, log);
-        U.join(sockReader, log);
-
-        leaveLatch.countDown();
-        joinLatch.countDown();
-
-        getSpiContext().deregisterPorts();
-
-        Collection<ClusterNode> rmts = getRemoteNodes();
-
-        // This is restart/disconnection and remote nodes are not empty.
-        // We need to fire FAIL event for each.
-        DiscoverySpiListener lsnr = this.lsnr;
-
-        if (lsnr != null) {
-            for (ClusterNode n : rmts) {
-                rmtNodes.remove(n.id());
-
-                Collection<ClusterNode> top = updateTopologyHistory(topVer + 1);
-
-                lsnr.onDiscovery(EVT_NODE_FAILED, topVer, n, top, new TreeMap<>(topHist), null);
-            }
-        }
-
-        rmtNodes.clear();
-    }
-
-    /** {@inheritDoc} */
-    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
-    @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
-        if (segmented)
-            throw new IgniteException("Failed to send custom message: client is disconnected");
-
-        try {
-            sockWriter.sendMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, marsh.marshal(evt)));
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteSpiException("Failed to marshal custom event: " + evt, e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void failNode(UUID nodeId) {
-        ClusterNode node = rmtNodes.get(nodeId);
-
-        if (node != null) {
-            TcpDiscoveryNodeFailedMessage msg = new TcpDiscoveryNodeFailedMessage(getLocalNodeId(),
-                node.id(), node.order());
-
-            msgWorker.addMessage(msg);
-        }
-    }
-
-    /**
-     * @return Opened socket or {@code null} if timeout.
-     * @see #joinTimeout
-     */
-    @SuppressWarnings("BusyWait")
-    @Nullable private Socket joinTopology(boolean recon) throws IgniteSpiException, InterruptedException {
-        Collection<InetSocketAddress> addrs = null;
-
-        long startTime = U.currentTimeMillis();
-
-        while (true) {
-            if (Thread.currentThread().isInterrupted())
-                throw new InterruptedException();
-
-            while (addrs == null || addrs.isEmpty()) {
-                addrs = resolvedAddresses();
-
-                if (!F.isEmpty(addrs)) {
-                    if (log.isDebugEnabled())
-                        log.debug("Resolved addresses from IP finder: " + addrs);
-                }
-                else {
-                    U.warn(log, "No addresses registered in the IP finder (will retry in 2000ms): " + ipFinder);
-
-                    if (joinTimeout > 0 && (U.currentTimeMillis() - startTime) > joinTimeout)
-                        return null;
-
-                    Thread.sleep(2000);
-                }
-            }
-
-            Collection<InetSocketAddress> addrs0 = new ArrayList<>(addrs);
-
-            Iterator<InetSocketAddress> it = addrs.iterator();
-
-            while (it.hasNext()) {
-                if (Thread.currentThread().isInterrupted())
-                    throw new InterruptedException();
-
-                InetSocketAddress addr = it.next();
-
-                Socket sock = null;
-
-                try {
-                    long ts = U.currentTimeMillis();
-
-                    IgniteBiTuple<Socket, UUID> t = initConnection(addr);
-
-                    sock = t.get1();
-
-                    UUID rmtNodeId = t.get2();
-
-                    stats.onClientSocketInitialized(U.currentTimeMillis() - ts);
-
-                    locNode.clientRouterNodeId(rmtNodeId);
-
-                    TcpDiscoveryAbstractMessage msg = recon ?
-                        new TcpDiscoveryClientReconnectMessage(getLocalNodeId(), rmtNodeId,
-                            lastMsgId) :
-                        new TcpDiscoveryJoinRequestMessage(locNode, collectExchangeData(getLocalNodeId()));
-
-                    msg.client(true);
-
-                    writeToSocket(sock, msg);
-
-                    int res = readReceipt(sock, ackTimeout);
-
-                    switch (res) {
-                        case RES_OK:
-                            return sock;
-
-                        case RES_CONTINUE_JOIN:
-                        case RES_WAIT:
-                            U.closeQuiet(sock);
-
-                            break;
-
-                        default:
-                            if (log.isDebugEnabled())
-                                log.debug("Received unexpected response to join request: " + res);
-
-                            U.closeQuiet(sock);
-                    }
-                }
-                catch (IOException | IgniteCheckedException e) {
-                    if (log.isDebugEnabled())
-                        U.error(log, "Failed to establish connection with address: " + addr, e);
-
-                    U.closeQuiet(sock);
-
-                    it.remove();
-                }
-            }
-
-            if (addrs.isEmpty()) {
-                U.warn(log, "Failed to connect to any address from IP finder (will retry to join topology " +
-                    "in 2000ms): " + addrs0);
-
-                if (joinTimeout > 0 && (U.currentTimeMillis() - startTime) > joinTimeout)
-                    return null;
-
-                Thread.sleep(2000);
-            }
-        }
-    }
-
-    /**
-     * @param topVer New topology version.
-     * @return Latest topology snapshot.
-     */
-    private NavigableSet<ClusterNode> updateTopologyHistory(long topVer) {
-        this.topVer = topVer;
-
-        NavigableSet<ClusterNode> allNodes = allVisibleNodes();
-
-        if (!topHist.containsKey(topVer)) {
-            assert topHist.isEmpty() || topHist.lastKey() == topVer - 1 :
-                "lastVer=" + topHist.lastKey() + ", newVer=" + topVer;
-
-            topHist.put(topVer, allNodes);
-
-            if (topHist.size() > topHistSize)
-                topHist.pollFirstEntry();
-
-            assert topHist.lastKey() == topVer;
-            assert topHist.size() <= topHistSize;
-        }
-
-        return allNodes;
-    }
-
-    /**
-     * @return All nodes.
-     */
-    private NavigableSet<ClusterNode> allVisibleNodes() {
-        NavigableSet<ClusterNode> allNodes = new TreeSet<>();
-
-        for (TcpDiscoveryNode node : rmtNodes.values()) {
-            if (node.visible())
-                allNodes.add(node);
-        }
-
-        allNodes.add(locNode);
-
-        return allNodes;
-    }
-
-    /**
-     * @param addr Address.
-     * @return Remote node ID.
-     * @throws IOException In case of I/O error.
-     * @throws IgniteCheckedException In case of other error.
-     */
-    private IgniteBiTuple<Socket, UUID> initConnection(InetSocketAddress addr) throws IOException, IgniteCheckedException {
-        assert addr != null;
-
-        Socket sock = openSocket(addr);
-
-        TcpDiscoveryHandshakeRequest req = new TcpDiscoveryHandshakeRequest(getLocalNodeId());
-
-        req.client(true);
-
-        writeToSocket(sock, req);
-
-        TcpDiscoveryHandshakeResponse res = readMessage(sock, null, ackTimeout);
-
-        UUID nodeId = res.creatorNodeId();
-
-        assert nodeId != null;
-        assert !getLocalNodeId().equals(nodeId);
-
-        return F.t(sock, nodeId);
-    }
-
-    /**
-     * FOR TEST PURPOSE ONLY!
-     */
-    void simulateNodeFailure() {
-        U.warn(log, "Simulating client node failure: " + getLocalNodeId());
-
-        U.interrupt(sockWriter);
-        U.interrupt(msgWorker);
-        U.interrupt(sockTimeoutWorker);
-
-        U.join(sockWriter, log);
-        U.join(msgWorker, log);
-        U.join(sockTimeoutWorker, log);
-    }
-
-    /**
-     * FOR TEST PURPOSE ONLY!
-     */
-    public void brakeConnection() {
-        U.closeQuiet(msgWorker.currSock);
-    }
-
-    /**
-     * FOR TEST PURPOSE ONLY!
-     */
-    public void waitForMessagePrecessed() {
-        Object last = msgWorker.queue.peekLast();
-
-        while (last != null && msgWorker.isAlive() && msgWorker.queue.contains(last)) {
-            try {
-                Thread.sleep(10);
-            }
-            catch (InterruptedException ignored) {
-                Thread.currentThread().interrupt();
-            }
-        }
-    }
-
-    /**
-     * Heartbeat sender.
-     */
-    private class HeartbeatSender extends TimerTask {
-        /** {@inheritDoc} */
-        @Override public void run() {
-            if (!getSpiContext().isStopping() && sockWriter.isOnline()) {
-                TcpDiscoveryClientHeartbeatMessage msg = new TcpDiscoveryClientHeartbeatMessage(getLocalNodeId(),
-                    metricsProvider.metrics());
-
-                msg.client(true);
-
-                sockWriter.sendMessage(msg);
-            }
-        }
-    }
-
-    /**
-     * Socket reader.
-     */
-    private class SocketReader extends IgniteSpiThread {
-        /** */
-        private final Object mux = new Object();
-
-        /** */
-        private Socket sock;
-
-        /** */
-        private UUID rmtNodeId;
-
-        /**
-         */
-        protected SocketReader() {
-            super(gridName, "tcp-client-disco-sock-reader", log);
-        }
-
-        /**
-         * @param sock Socket.
-         * @param rmtNodeId Rmt node id.
-         */
-        public void setSocket(Socket sock, UUID rmtNodeId) {
-            synchronized (mux) {
-                this.sock = sock;
-
-                this.rmtNodeId = rmtNodeId;
-
-                mux.notifyAll();
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            while (!isInterrupted()) {
-                Socket sock;
-                UUID rmtNodeId;
-
-                synchronized (mux) {
-                    if (this.sock == null) {
-                        mux.wait();
-
-                        continue;
-                    }
-
-                    sock = this.sock;
-                    rmtNodeId = this.rmtNodeId;
-                }
-
-                try {
-                    InputStream in = new BufferedInputStream(sock.getInputStream());
-
-                    sock.setKeepAlive(true);
-                    sock.setTcpNoDelay(true);
-
-                    while (!isInterrupted()) {
-                        TcpDiscoveryAbstractMessage msg;
-
-                        try {
-                            msg = marsh.unmarshal(in, U.gridClassLoader());
-                        }
-                        catch (IgniteCheckedException e) {
-                            if (log.isDebugEnabled())
-                                U.error(log, "Failed to read message [sock=" + sock + ", " +
-                                    "locNodeId=" + getLocalNodeId() + ", rmtNodeId=" + rmtNodeId + ']', e);
-
-                            IOException ioEx = X.cause(e, IOException.class);
-
-                            if (ioEx != null)
-                                throw ioEx;
-
-                            ClassNotFoundException clsNotFoundEx = X.cause(e, ClassNotFoundException.class);
-
-                            if (clsNotFoundEx != null)
-                                LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
-                                    "(make sure same versions of all classes are available on all nodes) " +
-                                    "[rmtNodeId=" + rmtNodeId + ", err=" + clsNotFoundEx.getMessage() + ']');
-                            else
-                                LT.error(log, e, "Failed to read message [sock=" + sock + ", locNodeId=" +
-                                    getLocalNodeId() + ", rmtNodeId=" + rmtNodeId + ']');
-
-                            continue;
-                        }
-
-                        msg.senderNodeId(rmtNodeId);
-
-                        if (log.isDebugEnabled())
-                            log.debug("Message has been received: " + msg);
-
-                        stats.onMessageReceived(msg);
-
-                        if (ensured(msg))
-                            lastMsgId = msg.id();
-
-                        msgWorker.addMessage(msg);
-                    }
-                }
-                catch (IOException e) {
-                    msgWorker.addMessage(new SocketClosedMessage(sock));
-
-                    if (log.isDebugEnabled())
-                        U.error(log, "Connection failed [sock=" + sock + ", locNodeId=" + getLocalNodeId() + ']', e);
-                }
-                finally {
-                    U.closeQuiet(sock);
-
-                    synchronized (mux) {
-                        if (this.sock == sock) {
-                            this.sock = null;
-                            this.rmtNodeId = null;
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    private class SocketWriter extends IgniteSpiThread {
-        /** */
-        private final Object mux = new Object();
-
-        /** */
-        private Socket sock;
-
-        /** */
-        private final Queue<TcpDiscoveryAbstractMessage> queue = new ArrayDeque<>();
-
-        /**
-         *
-         */
-        protected SocketWriter() {
-            super(gridName, "tcp-client-disco-sock-writer", log);
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void sendMessage(TcpDiscoveryAbstractMessage msg) {
-            synchronized (mux) {
-                queue.add(msg);
-
-                mux.notifyAll();
-            }
-        }
-
-        /**
-         * @param sock Socket.
-         */
-        private void setSocket(Socket sock) {
-            synchronized (mux) {
-                this.sock = sock;
-
-                mux.notifyAll();
-            }
-        }
-
-        /**
-         *
-         */
-        public boolean isOnline() {
-            synchronized (mux) {
-                return sock != null;
-            }
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            TcpDiscoveryAbstractMessage msg = null;
-
-            while (!Thread.currentThread().isInterrupted()) {
-                Socket sock;
-
-                synchronized (mux) {
-                    sock = this.sock;
-
-                    if (sock == null) {
-                        mux.wait();
-
-                        continue;
-                    }
-
-                    if (msg == null)
-                        msg = queue.poll();
-
-                    if (msg == null) {
-                        mux.wait();
-
-                        continue;
-                    }
-                }
-
-                try {
-                    writeToSocket(sock, msg);
-
-                    msg = null;
-                }
-                catch (IOException e) {
-                    if (log.isDebugEnabled())
-                        U.error(log, "Failed to send node left message (will stop anyway) [sock=" + sock + ']', e);
-
-                    U.closeQuiet(sock);
-
-                    synchronized (mux) {
-                        if (sock == this.sock)
-                            this.sock = null; // Connection has dead.
-                    }
-                }
-                catch (IgniteCheckedException e) {
-                    U.error(log, "Failed to send message: " + msg, e);
-
-                    msg = null;
-                }
-            }
-        }
-    }
-
-    /**
-     *
-     */
-    private class Reconnector extends IgniteSpiThread {
-        /** */
-        private volatile Socket sock;
-
-        /**
-         *
-         */
-        protected Reconnector() {
-            super(gridName, "tcp-client-disco-msg-worker", log);
-        }
-
-        /**
-         *
-         */
-        public void cancel() {
-            interrupt();
-
-            U.closeQuiet(sock);
-        }
-
-        /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException {
-            assert !segmented;
-
-            boolean success = false;
-
-            try {
-                sock = joinTopology(true);
-
-                if (sock == null) {
-                    U.error(log, "Failed to reconnect to cluster: timeout.");
-
-                    return;
-                }
-
-                if (isInterrupted())
-                    throw new InterruptedException();
-
-                InputStream in = new BufferedInputStream(sock.getInputStream());
-
-                sock.setKeepAlive(true);
-                sock.setTcpNoDelay(true);
-
-                // Wait for
-                while (!isInterrupted()) {
-                    TcpDiscoveryAbstractMessage msg = marsh.unmarshal(in, U.gridClassLoader());
-
-                    if (msg instanceof TcpDiscoveryClientReconnectMessage) {
-                        TcpDiscoveryClientReconnectMessage res = (TcpDiscoveryClientReconnectMessage)msg;
-
-                        if (res.creatorNodeId().equals(getLocalNodeId())) {
-                            if (res.success()) {
-                                msgWorker.addMessage(res);
-
-                                success = true;
-                            }
-
-                            break;
-                        }
-                    }
-
-                }
-            }
-            catch (IOException | IgniteCheckedException e) {
-                U.error(log, "Failed to reconnect", e);
-            }
-            finally {
-                if (!success) {
-                    U.closeQuiet(sock);
-
-                    msgWorker.addMessage(SPI_RECONNECT_FAILED);
-                }
-            }
-        }
-    }
-
-    /**
-     * Message worker.
-     */
-    protected class MessageWorker extends IgniteSpiThread {
-        /** Message queue. */
-        private final BlockingDeque<Object> queue = new LinkedBlockingDeque<>();
-
-        /** */
-        private Socket currSock;
-
-        /** Indicates that pending messages are currently processed. */
-        private boolean pending;
-
-        /** */
-        private Reconnector reconnector;
-
-        /**
-         *
-         */
-        private MessageWorker() {
-            super(gridName, "tcp-client-disco-msg-worker", log);
-        }
-
-        /** {@inheritDoc} */
-        @SuppressWarnings("InfiniteLoopStatement")
-        @Override protected void body() throws InterruptedException {
-            stats.onJoinStarted();
-
-            try {
-                final Socket sock = joinTopology(false);
-
-                if (sock == null) {
-                    joinErr = new IgniteSpiException("Join process timed out");
-
-                    joinLatch.countDown();
-
-                    return;
-                }
-
-                currSock = sock;
-
-                sockWriter.setSocket(sock);
-
-                timer.schedule(new TimerTask() {
-                    @Override public void run() {
-                        if (joinLatch.getCount() > 0)
-                           queue.add(JOIN_TIMEOUT);
-                    }
-                }, netTimeout);
-
-                sockReader.setSocket(sock, locNode.clientRouterNodeId());
-
-                while (true) {
-                    Object msg = queue.take();
-
-                    if (msg == JOIN_TIMEOUT) {
-                        if (joinLatch.getCount() > 0) {
-                            joinErr = new IgniteSpiException("Join process timed out [sock=" + sock +
-                                ", timeout=" + netTimeout + ']');
-
-                            joinLatch.countDown();
-
-                            break;
-                        }
-                    }
-                    else if (msg == SPI_STOP) {
-                        assert getSpiContext().isStopping();
-
-                        if (currSock != null) {
-                            TcpDiscoveryAbstractMessage leftMsg = new TcpDiscoveryNodeLeftMessage(getLocalNodeId());
-
-                            leftMsg.client(true);
-
-                            sockWriter.sendMessage(leftMsg);
-                        }
-                        else
-                            leaveLatch.countDown();
-                    }
-                    else if (msg instanceof SocketClosedMessage) {
-                        if (((SocketClosedMessage)msg).sock == currSock) {
-                            currSock = null;
-
-                            if (joinLatch.getCount() > 0) {
-                                joinErr = new IgniteSpiException("Failed to connect to cluster: socket closed.");
-
-                                joinLatch.countDown();
-
-                                break;
-                            }
-                            else {
-                                if (getSpiContext().isStopping() || segmented)
-                                    leaveLatch.countDown();
-                                else {
-                                    assert reconnector == null;
-
-                                    final Reconnector reconnector = new Reconnector();
-                                    this.reconnector = reconnector;
-                                    reconnector.start();
-
-                                    timer.schedule(new TimerTask() {
-                                        @Override public void run() {
-                                            if (reconnector.isAlive())
-                                                reconnector.cancel();
-                                        }
-                                    }, netTimeout);
-                                }
-                            }
-                        }
-                    }
-                    else if (msg == SPI_RECONNECT_FAILED) {
-                        if (!segmented) {
-                            segmented = true;
-
-                            reconnector.cancel();
-                            reconnector.join();
-
-                            notifyDiscovery(EVT_NODE_SEGMENTED, topVer, locNode, allVisibleNodes());
-                        }
-                    }
-                    else {
-                        TcpDiscoveryAbstractMessage discoMsg = (TcpDiscoveryAbstractMessage)msg;
-
-                        if (joinLatch.getCount() > 0) {
-                            IgniteSpiException err = null;
-
-                            if (discoMsg instanceof TcpDiscoveryDuplicateIdMessage)
-                                err = duplicateIdError((TcpDiscoveryDuplicateIdMessage)msg);
-                            else if (discoMsg instanceof TcpDiscoveryAuthFailedMessage)
-                                err = authenticationFailedError((TcpDiscoveryAuthFailedMessage)msg);
-                            else if (discoMsg instanceof TcpDiscoveryCheckFailedMessage)
-                                err = checkFailedError((TcpDiscoveryCheckFailedMessage)msg);
-
-                            if (err != null) {
-                                joinErr = err;
-
-                                joinLatch.countDown();
-
-                                break;
-                            }
-                        }
-
-                        processDiscoveryMessage((TcpDiscoveryAbstractMessage)msg);
-                    }
-                }
-            }
-            finally {
-                U.closeQuiet(currSock);
-
-                if (joinLatch.getCount() > 0) {
-                    // This should not occurs.
-                    joinErr = new IgniteSpiException("Some error occurs in joinig process");
-
-                    joinLatch.countDown();
-                }
-
-                if (reconnector != null) {
-                    reconnector.cancel();
-
-                    reconnector.join();
-                }
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        protected void processDiscoveryMessage(TcpDiscoveryAbstractMessage msg) {
-            assert msg != null;
-            assert msg.verified() || msg.senderNodeId() == null;
-
-            stats.onMessageProcessingStarted(msg);
-
-            if (msg instanceof TcpDiscoveryNodeAddedMessage)
-                processNodeAddedMessage((TcpDiscoveryNodeAddedMessage)msg);
-            else if (msg instanceof TcpDiscoveryNodeAddFinishedMessage)
-                processNodeAddFinishedMessage((TcpDiscoveryNodeAddFinishedMessage)msg);
-            else if (msg instanceof TcpDiscoveryNodeLeftMessage)
-                processNodeLeftMessage((TcpDiscoveryNodeLeftMessage)msg);
-            else if (msg instanceof TcpDiscoveryNodeFailedMessage)
-                processNodeFailedMessage((TcpDiscoveryNodeFailedMessage)msg);
-            else if (msg instanceof TcpDiscoveryHeartbeatMessage)
-                processHeartbeatMessage((TcpDiscoveryHeartbeatMessage)msg);
-            else if (msg instanceof TcpDiscoveryClientReconnectMessage)
-                processClientReconnectMessage((TcpDiscoveryClientReconnectMessage)msg);
-            else if (msg instanceof TcpDiscoveryCustomEventMessage)
-                processCustomMessage((TcpDiscoveryCustomEventMessage)msg);
-            else if (msg instanceof TcpDiscoveryClientPingResponse)
-                processClientPingResponse((TcpDiscoveryClientPingResponse)msg);
-            else if (msg instanceof TcpDiscoveryPingRequest)
-                processPingRequest();
-
-            stats.onMessageProcessingFinished(msg);
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processNodeAddedMessage(TcpDiscoveryNodeAddedMessage msg) {
-            if (getSpiContext().isStopping())
-                return;
-
-            TcpDiscoveryNode node = msg.node();
-
-            UUID newNodeId = node.id();
-
-            if (getLocalNodeId().equals(newNodeId)) {
-                if (joinLatch.getCount() > 0) {
-                    Collection<TcpDiscoveryNode> top = msg.topology();
-
-                    if (top != null) {
-                        gridStartTime = msg.gridStartTime();
-
-                        for (TcpDiscoveryNode n : top) {
-                            if (n.order() > 0)
-                                n.visible(true);
-
-                            rmtNodes.put(n.id(), n);
-                        }
-
-                        topHist.clear();
-
-                        if (msg.topologyHistory() != null)
-                            topHist.putAll(msg.topologyHistory());
-                    }
-                    else if (log.isDebugEnabled())
-                        log.debug("Discarding node added message with empty topology: " + msg);
-                }
-                else if (log.isDebugEnabled())
-                    log.debug("Discarding node added message (this message has already been processed) " +
-                        "[msg=" + msg + ", locNode=" + locNode + ']');
-            }
-            else {
-                boolean topChanged = rmtNodes.putIfAbsent(newNodeId, node) == null;
-
-                if (topChanged) {
-                    if (log.isDebugEnabled())
-                        log.debug("Added new node to topology: " + node);
-
-                    Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
-
-                    if (data != null)
-                        onExchange(newNodeId, newNodeId, data, null);
-                }
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processNodeAddFinishedMessage(TcpDiscoveryNodeAddFinishedMessage msg) {
-            if (getSpiContext().isStopping())
-                return;
-
-            if (getLocalNodeId().equals(msg.nodeId())) {
-                if (joinLatch.getCount() > 0) {
-                    Map<UUID, Map<Integer, byte[]>> dataMap = msg.clientDiscoData();
-
-                    if (dataMap != null) {
-                        for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
-                            onExchange(getLocalNodeId(), entry.getKey(), entry.getValue(), null);
-                    }
-
-                    locNode.setAttributes(msg.clientNodeAttributes());
-                    locNode.visible(true);
-
-                    long topVer = msg.topologyVersion();
-
-                    locNode.order(topVer);
-
-                    notifyDiscovery(EVT_NODE_JOINED, topVer, locNode, updateTopologyHistory(topVer));
-
-                    joinErr = null;
-
-                    joinLatch.countDown();
-
-                    stats.onJoinFinished();
-                }
-                else if (log.isDebugEnabled())
-                    log.debug("Discarding node add finished message (this message has already been processed) " +
-                        "[msg=" + msg + ", locNode=" + locNode + ']');
-            }
-            else {
-                TcpDiscoveryNode node = rmtNodes.get(msg.nodeId());
-
-                if (node == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node add finished message since node is not found [msg=" + msg + ']');
-
-                    return;
-                }
-
-                long topVer = msg.topologyVersion();
-
-                node.order(topVer);
-                node.visible(true);
-
-                if (locNodeVer.equals(node.version()))
-                    node.version(locNodeVer);
-
-                NavigableSet<ClusterNode> top = updateTopologyHistory(topVer);
-
-                if (!pending && joinLatch.getCount() > 0) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node add finished message (join process is not finished): " + msg);
-
-                    return;
-                }
-
-                notifyDiscovery(EVT_NODE_JOINED, topVer, node, top);
-
-                stats.onNodeJoined();
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processNodeLeftMessage(TcpDiscoveryNodeLeftMessage msg) {
-            if (getLocalNodeId().equals(msg.creatorNodeId())) {
-                if (log.isDebugEnabled())
-                    log.debug("Received node left message for local node: " + msg);
-
-                leaveLatch.countDown();
-            }
-            else {
-                if (getSpiContext().isStopping())
-                    return;
-
-                TcpDiscoveryNode node = rmtNodes.remove(msg.creatorNodeId());
-
-                if (node == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node left message since node is not found [msg=" + msg + ']');
-
-                    return;
-                }
-
-                NavigableSet<ClusterNode> top = updateTopologyHistory(msg.topologyVersion());
-
-                if (!pending && joinLatch.getCount() > 0) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node left message (join process is not finished): " + msg);
-
-                    return;
-                }
-
-                notifyDiscovery(EVT_NODE_LEFT, msg.topologyVersion(), node, top);
-
-                stats.onNodeLeft();
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) {
-            if (getSpiContext().isStopping()) {
-                if (!getLocalNodeId().equals(msg.creatorNodeId()) && getLocalNodeId().equals(msg.failedNodeId())) {
-                    if (leaveLatch.getCount() > 0) {
-                        log.debug("Remote node fail this node while node is stopping [locNode=" + getLocalNodeId()
-                            + ", rmtNode=" + msg.creatorNodeId() + ']');
-
-                        leaveLatch.countDown();
-                    }
-                }
-
-                return;
-            }
-
-            if (!getLocalNodeId().equals(msg.creatorNodeId())) {
-                TcpDiscoveryNode node = rmtNodes.remove(msg.failedNodeId());
-
-                if (node == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node failed message since node is not found [msg=" + msg + ']');
-
-                    return;
-                }
-
-                NavigableSet<ClusterNode> top = updateTopologyHistory(msg.topologyVersion());
-
-                if (!pending && joinLatch.getCount() > 0) {
-                    if (log.isDebugEnabled())
-                        log.debug("Discarding node failed message (join process is not finished): " + msg);
-
-                    return;
-                }
-
-                notifyDiscovery(EVT_NODE_FAILED, msg.topologyVersion(), node, top);
-
-                stats.onNodeFailed();
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) {
-            if (getSpiContext().isStopping())
-                return;
-
-            if (getLocalNodeId().equals(msg.creatorNodeId())) {
-                assert msg.senderNodeId() != null;
-
-                if (log.isDebugEnabled())
-                    log.debug("Received heartbeat response: " + msg);
-            }
-            else {
-                long tstamp = U.currentTimeMillis();
-
-                if (msg.hasMetrics()) {
-                    for (Map.Entry<UUID, MetricsSet> e : msg.metrics().entrySet()) {
-                        UUID nodeId = e.getKey();
-
-                        MetricsSet metricsSet = e.getValue();
-
-                        Map<Integer, CacheMetrics> cacheMetrics = msg.hasCacheMetrics() ?
-                                msg.cacheMetrics().get(nodeId) : Collections.<Integer, CacheMetrics>emptyMap();
-
-                        updateMetrics(nodeId, metricsSet.metrics(), cacheMetrics, tstamp);
-
-                        for (T2<UUID, ClusterMetrics> t : metricsSet.clientMetrics())
-                            updateMetrics(t.get1(), t.get2(), cacheMetrics, tstamp);
-                    }
-                }
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processClientReconnectMessage(TcpDiscoveryClientReconnectMessage msg) {
-            if (getSpiContext().isStopping())
-                return;
-
-            if (getLocalNodeId().equals(msg.creatorNodeId())) {
-                assert msg.success();
-
-                currSock = reconnector.sock;
-
-                sockWriter.setSocket(currSock);
-                sockReader.setSocket(currSock, locNode.clientRouterNodeId());
-
-                reconnector = null;
-
-                pending = true;
-
-                try {
-                    for (TcpDiscoveryAbstractMessage pendingMsg : msg.pendingMessages())
-                        processDiscoveryMessage(pendingMsg);
-                }
-                finally {
-                    pending = false;
-                }
-            }
-            else if (log.isDebugEnabled())
-                log.debug("Discarding reconnect message for another client: " + msg);
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) {
-            if (msg.verified() && joinLatch.getCount() == 0) {
-                DiscoverySpiListener lsnr = TcpClientDiscoverySpi.this.lsnr;
-
-                if (lsnr != null) {
-                    UUID nodeId = msg.creatorNodeId();
-
-                    TcpDiscoveryNode node = nodeId.equals(getLocalNodeId()) ? locNode : rmtNodes.get(nodeId);
-
-                    if (node != null && node.visible()) {
-                        try {
-                            DiscoverySpiCustomMessage msgObj = msg.message(marsh);
-
-                            notifyDiscovery(EVT_DISCOVERY_CUSTOM_EVT, topVer, node, allVisibleNodes(), msgObj);
-                        }
-                        catch (Throwable e) {
-                            U.error(log, "Failed to unmarshal discovery custom message.", e);
-                        }
-                    }
-                    else if (log.isDebugEnabled())
-                        log.debug("Received metrics from unknown node: " + nodeId);
-                }
-            }
-        }
-
-        /**
-         * @param msg Message.
-         */
-        private void processClientPingResponse(TcpDiscoveryClientPingResponse msg) {
-            GridFutureAdapter<Boolean> fut = pingFuts.remove(msg.nodeToPing());
-
-            if (fut != null)
-                fut.onDone(msg.result());
-        }
-
-        /**
-         * Router want to ping this client.
-         */
-        private void processPingRequest() {
-            TcpDiscoveryPingResponse res = new TcpDiscoveryPingResponse(getLocalNodeId());
-
-            res.client(true);
-
-            sockWriter.sendMessage(res);
-        }
-
-        /**
-         * @param nodeId Node ID.
-         * @param metrics Metrics.
-         * @param cacheMetrics Cache metrics.
-         * @param tstamp Timestamp.
-         */
-        private void updateMetrics(UUID nodeId,
-            ClusterMetrics metrics,
-            Map<Integer, CacheMetrics> cacheMetrics,
-            long tstamp)
-        {
-            assert nodeId != null;
-            assert metrics != null;
-            assert cacheMetrics != null;
-
-            TcpDiscoveryNode node = nodeId.equals(getLocalNodeId()) ? locNode : rmtNodes.get(nodeId);
-
-            if (node != null && node.visible()) {
-                node.setMetrics(metrics);
-                node.setCacheMetrics(cacheMetrics);
-
-                node.lastUpdateTime(tstamp);
-
-                notifyDiscovery(EVT_NODE_METRICS_UPDATED, topVer, node, allVisibleNodes());
-            }
-            else if (log.isDebugEnabled())
-                log.debug("Received metrics from unknown node: " + nodeId);
-        }
-
-        /**
-         * @param type Event type.
-         * @param topVer Topology version.
-         * @param node Node.
-         * @param top Topology snapshot.
-         */
-        private void notifyDiscovery(int type, long topVer, ClusterNode node, NavigableSet<ClusterNode> top) {
-            notifyDiscovery(type, topVer, node, top, null);
-        }
-
-        /**
-         * @param type Event type.
-         * @param topVer Topology version.
-         * @param node Node.
-         * @param top Topology snapshot.
-         */
-        private void notifyDiscovery(int type, long topVer, ClusterNode node, NavigableSet<ClusterNode> top,
-            @Nullable DiscoverySpiCustomMessage data) {
-            DiscoverySpiListener lsnr = TcpClientDiscoverySpi.this.lsnr;
-
-            if (lsnr != null) {
-                if (log.isDebugEnabled())
-                    log.debug("Discovery notification [node=" + node + ", type=" + U.gridEventName(type) +
-                        ", topVer=" + topVer + ']');
-
-                lsnr.onDiscovery(type, topVer, node, top, new TreeMap<>(topHist), data);
-            }
-            else if (log.isDebugEnabled())
-                log.debug("Skipped discovery notification [node=" + node + ", type=" + U.gridEventName(type) +
-                    ", topVer=" + topVer + ']');
-        }
-
-        /**
-         * @param msg Message.
-         */
-        public void addMessage(Object msg) {
-            queue.add(msg);
-        }
-
-        /**
-         *
-         */
-        public int queueSize() {
-            return queue.size();
-        }
-    }
-
-    /**
-     *
-     */
-    private static class SocketClosedMessage {
-        /** */
-        private final Socket sock;
-
-        /**
-         * @param sock Socket.
-         */
-        private SocketClosedMessage(Socket sock) {
-            this.sock = sock;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMBean.java
deleted file mode 100644
index 3101da8..0000000
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiMBean.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * 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.mxbean.*;
-import org.apache.ignite.spi.*;
-
-import java.util.*;
-
-/**
- * Management bean for {@link TcpClientDiscoverySpi}.
- */
-public interface TcpClientDiscoverySpiMBean extends IgniteSpiManagementMBean {
-    /**
-     * Gets socket timeout.
-     *
-     * @return Socket timeout.
-     */
-    @MXBeanDescription("Socket timeout.")
-    public long getSocketTimeout();
-
-    /**
-     * Gets message acknowledgement timeout.
-     *
-     * @return Message acknowledgement timeout.
-     */
-    @MXBeanDescription("Message acknowledgement timeout.")
-    public long getAckTimeout();
-
-    /**
-     * Gets network timeout.
-     *
-     * @return Network timeout.
-     */
-    @MXBeanDescription("Network timeout.")
-    public long getNetworkTimeout();
-
-    /**
-     * Gets thread priority. All threads within SPI will be started with it.
-     *
-     * @return Thread priority.
-     */
-    @MXBeanDescription("Threads priority.")
-    public int getThreadPriority();
-
-    /**
-     * Gets delay between heartbeat messages sent by coordinator.
-     *
-     * @return Time period in milliseconds.
-     */
-    @MXBeanDescription("Heartbeat frequency.")
-    public long getHeartbeatFrequency();
-
-    /**
-     * Gets {@link org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder} (string representation).
-     *
-     * @return IPFinder (string representation).
-     */
-    @MXBeanDescription("IP Finder.")
-    public String getIpFinderFormatted();
-
-    /**
-     * Gets message worker queue current size.
-     *
-     * @return Message worker queue current size.
-     */
-    @MXBeanDescription("Message worker queue current size.")
-    public int getMessageWorkerQueueSize();
-
-    /**
-     * Gets joined nodes count.
-     *
-     * @return Nodes joined count.
-     */
-    @MXBeanDescription("Nodes joined count.")
-    public long getNodesJoined();
-
-    /**
-     * Gets left nodes count.
-     *
-     * @return Left nodes count.
-     */
-    @MXBeanDescription("Nodes left count.")
-    public long getNodesLeft();
-
-    /**
-     * Gets failed nodes count.
-     *
-     * @return Failed nodes count.
-     */
-    @MXBeanDescription("Nodes failed count.")
-    public long getNodesFailed();
-
-    /**
-     * Gets avg message processing time.
-     *
-     * @return Avg message processing time.
-     */
-    @MXBeanDescription("Avg message processing time.")
-    public long getAvgMessageProcessingTime();
-
-    /**
-     * Gets max message processing time.
-     *
-     * @return Max message processing time.
-     */
-    @MXBeanDescription("Max message processing time.")
-    public long getMaxMessageProcessingTime();
-
-    /**
-     * Gets total received messages count.
-     *
-     * @return Total received messages count.
-     */
-    @MXBeanDescription("Total received messages count.")
-    public int getTotalReceivedMessages();
-
-    /**
-     * Gets received messages counts (grouped by type).
-     *
-     * @return Map containing message types and respective counts.
-     */
-    @MXBeanDescription("Received messages by type.")
-    public Map<String, Integer> getReceivedMessages();
-
-    /**
-     * Gets total processed messages count.
-     *
-     * @return Total processed messages count.
-     */
-    @MXBeanDescription("Total processed messages count.")
-    public int getTotalProcessedMessages();
-
-    /**
-     * Gets processed messages counts (grouped by type).
-     *
-     * @return Map containing message types and respective counts.
-     */
-    @MXBeanDescription("Received messages by type.")
-    public Map<String, Integer> getProcessedMessages();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/838c0fd8/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
new file mode 100644
index 0000000..8dad92a
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -0,0 +1,175 @@
+/*
+ * 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.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ *
+ */
+abstract class TcpDiscoveryImpl {
+    /** Response OK. */
+    protected static final int RES_OK = 1;
+
+    /** Response CONTINUE JOIN. */
+    protected static final int RES_CONTINUE_JOIN = 100;
+
+    /** Response WAIT. */
+    protected static final int RES_WAIT = 200;
+
+    /** */
+    protected final TcpDiscoverySpi adapter;
+
+    /** */
+    protected final IgniteLogger log;
+
+    /** */
+    protected TcpDiscoveryNode locNode;
+
+    /**
+     * @param adapter Adapter.
+     */
+    TcpDiscoveryImpl(TcpDiscoverySpi adapter) {
+        this.adapter = adapter;
+
+        log = adapter.log;
+    }
+
+    /**
+     *
+     */
+    public UUID getLocalNodeId() {
+        return adapter.getLocalNodeId();
+    }
+
+    /**
+     * @param msg Error message.
+     * @param e Exception.
+     */
+    protected void onException(String msg, Exception e){
+        adapter.getExceptionRegistry().onException(msg, e);
+    }
+
+    /**
+     * @param log Logger.
+     */
+    public abstract void dumpDebugInfo(IgniteLogger log);
+
+    /**
+     *
+     */
+    public abstract String getSpiState();
+
+    /**
+     *
+     */
+    public abstract int getMessageWorkerQueueSize();
+
+    /**
+     *
+     */
+    public abstract UUID getCoordinator();
+
+    /**
+     *
+     */
+    public abstract Collection<ClusterNode> getRemoteNodes();
+
+    /**
+     * @param nodeId Node id.
+     */
+    @Nullable public abstract ClusterNode getNode(UUID nodeId);
+
+    /**
+     * @param nodeId Node id.
+     */
+    public abstract boolean pingNode(UUID nodeId);
+
+    /**
+     *
+     */
+    public abstract void disconnect() throws IgniteSpiException;
+
+    /**
+     * @param auth Auth.
+     */
+    public abstract void setAuthenticator(DiscoverySpiNodeAuthenticator auth);
+
+    /**
+     * @param msg Message.
+     */
+    public abstract void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException;
+
+    /**
+     * @param nodeId Node id.
+     */
+    public abstract void failNode(UUID nodeId);
+
+    /**
+     * @param gridName Grid name.
+     */
+    public abstract void spiStart(@Nullable String gridName) throws IgniteSpiException;
+
+    /**
+     *
+     */
+    public abstract void spiStop() throws IgniteSpiException;
+
+    /**
+     * @param spiCtx Spi context.
+     */
+    public abstract void onContextInitialized0(IgniteSpiContext spiCtx) throws IgniteSpiException;
+
+    /**
+     * @param t Thread.
+     * @return Status as string.
+     */
+    protected static String threadStatus(Thread t) {
+        if (t == null)
+            return "N/A";
+
+        return t.isAlive() ? "alive" : "dead";
+    }
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     * <p>
+     * Simulates this node failure by stopping service threads. So, node will become
+     * unresponsive.
+     * <p>
+     * This method is intended for test purposes only.
+     */
+    abstract void simulateNodeFailure();
+
+    /**
+     * FOR TEST PURPOSE ONLY!
+     */
+    public abstract void brakeConnection();
+
+    /**
+     * FOR TEST PURPOSE ONLY!
+     */
+    protected abstract IgniteSpiThread workerThread();
+}


[45/53] [abbrv] incubator-ignite git commit: # ignite-943 fix services on client

Posted by se...@apache.org.
# ignite-943 fix services on client


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

Branch: refs/heads/ignite-sprint-5
Commit: 99234d0ac5fd2e1d1033b1930cb4d7d6d49e0f28
Parents: 477a215
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 16:39:00 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 16:39:00 2015 +0300

----------------------------------------------------------------------
 .../service/GridServiceProcessor.java           | 104 ++++++++++++++++++-
 .../service/ClosureServiceClientsNodesTest.java |  10 ++
 2 files changed, 110 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/99234d0a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index a9a15e9..aedac75 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.timeout.*;
 import org.apache.ignite.internal.util.*;
@@ -59,6 +60,12 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     /** Time to wait before reassignment retries. */
     private static final long RETRY_TIMEOUT = 1000;
 
+    /** */
+    private static final ServiceDeploymentPredicate DEPLOYMENT_PREDICATE = new ServiceDeploymentPredicate();
+
+    /** */
+    private static final ServiceAssignmentsPredicate ASSIGNMENTS_PREDICATE = new ServiceAssignmentsPredicate();
+
     /** Local service instances. */
     private final Map<String, Collection<ServiceContextImpl>> locSvcs = new HashMap<>();
 
@@ -345,7 +352,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 "different configuration) [deployed=" + dep.configuration() + ", new=" + cfg + ']'));
                         }
                         else {
-                            for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+                            Iterator<Cache.Entry<Object, Object>> it = serviceEntries(ASSIGNMENTS_PREDICATE);
+
+                            while (it.hasNext()) {
+                                Cache.Entry<Object, Object> e = it.next();
+
                                 if (e.getKey() instanceof GridServiceAssignmentsKey) {
                                     GridServiceAssignments assigns = (GridServiceAssignments)e.getValue();
 
@@ -437,7 +448,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public IgniteInternalFuture<?> cancelAll() {
         Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
-        for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+
+        while (it.hasNext()) {
+            Cache.Entry<Object, Object> e = it.next();
+
             if (!(e.getKey() instanceof GridServiceDeploymentKey))
                 continue;
 
@@ -456,7 +471,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     public Collection<ServiceDescriptor> serviceDescriptors() {
         Collection<ServiceDescriptor> descs = new ArrayList<>();
 
-        for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+        Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+
+        while (it.hasNext()) {
+            Cache.Entry<Object, Object> e = it.next();
+
             if (!(e.getKey() instanceof GridServiceDeploymentKey))
                 continue;
 
@@ -904,6 +923,43 @@ public class GridServiceProcessor extends GridProcessorAdapter {
     }
 
     /**
+     * @param p Entry predicate used to execute query from client node.
+     * @return Service deployment entries.
+     */
+    @SuppressWarnings("unchecked")
+    private Iterator<Cache.Entry<Object, Object>> serviceEntries(IgniteBiPredicate<Object, Object> p) {
+        if (!cache.context().affinityNode()) {
+            ClusterNode oldestSrvNode =
+                CU.oldestAliveCacheServerNode(cache.context().shared(), AffinityTopologyVersion.NONE);
+
+            if (oldestSrvNode == null)
+                return F.emptyIterator();
+
+            GridCacheQueryManager qryMgr = cache.context().queries();
+
+            CacheQuery<Map.Entry<Object, Object>> qry = qryMgr.createScanQuery(p, false);
+
+            qry.keepAll(false);
+
+            qry.projection(ctx.cluster().get().forNode(oldestSrvNode));
+
+            return cache.context().itHolder().iterator(qry.execute(),
+                new CacheIteratorConverter<Object, Map.Entry<Object,Object>>() {
+                    @Override protected Object convert(Map.Entry<Object, Object> e) {
+                        return new CacheEntryImpl<>(e.getKey(), e.getValue());
+                    }
+
+                    @Override protected void remove(Object item) {
+                        throw new UnsupportedOperationException();
+                    }
+                }
+            );
+        }
+        else
+            return cache.entrySetx().iterator();
+    }
+
+    /**
      * Service deployment listener.
      */
     private class DeploymentListener implements CacheEntryUpdatedListener<Object, Object> {
@@ -1056,7 +1112,11 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                                 ctx.cache().context().deploy().ignoreOwnership(true);
 
                             try {
-                                for (Cache.Entry<Object, Object> e : cache.entrySetx()) {
+                                Iterator<Cache.Entry<Object, Object>> it = serviceEntries(DEPLOYMENT_PREDICATE);
+
+                                while (it.hasNext()) {
+                                    Cache.Entry<Object, Object> e = it.next();
+
                                     if (!(e.getKey() instanceof GridServiceDeploymentKey))
                                         continue;
 
@@ -1265,4 +1325,40 @@ public class GridServiceProcessor extends GridProcessorAdapter {
          */
         public abstract void run0();
     }
+
+    /**
+     *
+     */
+    static class ServiceDeploymentPredicate implements IgniteBiPredicate<Object, Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(Object key, Object val) {
+            return key instanceof GridServiceDeploymentKey;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ServiceDeploymentPredicate.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    static class ServiceAssignmentsPredicate implements IgniteBiPredicate<Object, Object> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public boolean apply(Object key, Object val) {
+            return key instanceof GridServiceAssignmentsKey;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ServiceAssignmentsPredicate.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/99234d0a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
index 59594be..faccc9a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
@@ -84,6 +84,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
             srvNames.add(getTestGridName(i));
 
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             Collection<String> res = ignite.compute().broadcast(new IgniteCallable<String>() {
@@ -109,6 +111,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
      */
     public void testClientClosure() throws Exception {
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             Collection<String> res = ignite.compute(ignite.cluster().forClients()).
@@ -134,6 +138,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
      */
     public void testCustomClosure() throws Exception {
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             Collection<String> res = ignite.compute(ignite.cluster().forPredicate(F.<ClusterNode>alwaysTrue())).
@@ -157,6 +163,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
         UUID clientNodeId = grid(0).cluster().localNode().id();
 
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             ignite.services().deployNodeSingleton(SINGLETON_NAME, new TestService());
@@ -190,6 +198,8 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
         UUID clientNodeId = grid(0).cluster().localNode().id();
 
         for (int i = 0 ; i < NODES_CNT; i++) {
+            log.info("Iteration: " + i);
+
             Ignite ignite = grid(i);
 
             ignite.services(ignite.cluster().forClients()).deployNodeSingleton(SINGLETON_NAME, new TestService());


[48/53] [abbrv] incubator-ignite git commit: # ignite-943 fixed client futures remap

Posted by se...@apache.org.
# ignite-943 fixed client futures remap


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

Branch: refs/heads/ignite-sprint-5
Commit: d10fe3e9013bc9ecd27823ff00443983b7c066c2
Parents: b3d8e1e
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 17:29:28 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 17:29:28 2015 +0300

----------------------------------------------------------------------
 .../dht/colocated/GridDhtColocatedLockFuture.java | 18 ++++++++++++------
 .../near/GridNearOptimisticTxPrepareFuture.java   | 18 ++++++++++++------
 2 files changed, 24 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d10fe3e9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index 5a4c91d..c784948 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -549,7 +549,7 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
         }
 
         // Must get topology snapshot and map on that version.
-        mapOnTopology(false);
+        mapOnTopology(false, null);
     }
 
     /**
@@ -557,8 +557,9 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
      * will asynchronously wait for it's completeness and then try again.
      *
      * @param remap Remap flag.
+     * @param c Optional closure to run after map.
      */
-    private void mapOnTopology(final boolean remap) {
+    private void mapOnTopology(final boolean remap, @Nullable final Runnable c) {
         // We must acquire topology snapshot from the topology version future.
         cctx.topology().readLock();
 
@@ -597,12 +598,15 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
 
                 map(keys, remap);
 
+                if (c != null)
+                    c.run();
+
                 markInitialized();
             }
             else {
                 fut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                        mapOnTopology(remap);
+                        mapOnTopology(remap, c);
                     }
                 });
             }
@@ -1366,9 +1370,11 @@ public final class GridDhtColocatedLockFuture extends GridCompoundIdentityFuture
             for (KeyCacheObject key : GridDhtColocatedLockFuture.this.keys)
                 cctx.mvcc().removeExplicitLock(threadId, key, lockVer);
 
-            mapOnTopology(true);
-
-            onDone(true);
+            mapOnTopology(true, new Runnable() {
+                @Override public void run() {
+                    onDone(true);
+                }
+            });
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d10fe3e9/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
index 0a92112..44b7997 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearOptimisticTxPrepareFuture.java
@@ -226,13 +226,14 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
             return;
         }
 
-        prepareOnTopology(false);
+        prepareOnTopology(false, null);
     }
 
     /**
      * @param remap Remap flag.
+     * @param c Optional closure to run after map.
      */
-    private void prepareOnTopology(final boolean remap) {
+    private void prepareOnTopology(final boolean remap, @Nullable final Runnable c) {
         GridDhtTopologyFuture topFut = topologyReadLock();
 
         try {
@@ -271,13 +272,16 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
                     tx.topologyVersion(topFut.topologyVersion());
 
                 prepare0(remap);
+
+                if (c != null)
+                    c.run();
             }
             else {
                 topFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                     @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
                         cctx.kernalContext().closure().runLocalSafe(new GridPlainRunnable() {
                             @Override public void run() {
-                                prepareOnTopology(remap);
+                                prepareOnTopology(remap, c);
                             }
                         });
                     }
@@ -796,9 +800,11 @@ public class GridNearOptimisticTxPrepareFuture extends GridNearTxPrepareFutureAd
          *
          */
         private void remap() {
-            prepareOnTopology(true);
-
-            onDone(tx);
+            prepareOnTopology(true, new Runnable() {
+                @Override public void run() {
+                    onDone(tx);
+                }
+            });
         }
 
         /** {@inheritDoc} */


[32/53] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-943

Posted by se...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-709_2' into ignite-943


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

Branch: refs/heads/ignite-sprint-5
Commit: 3dea5f148e3fc017bf9643e69f2661a132cfc96c
Parents: 110573b 581f4d9
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 17:19:27 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 17:19:27 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  1 -
 .../processors/cache/GridCacheAdapter.java      | 15 ++++++++-
 .../cache/GridCacheConcurrentMap.java           | 21 ++++++++++---
 .../processors/cache/GridCacheProcessor.java    | 32 +++++++++++---------
 .../processors/cache/GridCacheProxyImpl.java    | 12 ++++++++
 .../processors/cache/IgniteInternalCache.java   |  5 +++
 .../cache/query/GridCacheQueryAdapter.java      |  2 ++
 .../cache/query/GridCacheQueryErrorFuture.java  |  2 ++
 .../cache/query/GridCacheQueryManager.java      |  2 +-
 .../continuous/CacheContinuousQueryManager.java | 24 ++++++++++++---
 .../cacheobject/IgniteCacheObjectProcessor.java |  5 ++-
 .../IgniteCacheObjectProcessorImpl.java         |  2 +-
 12 files changed, 93 insertions(+), 30 deletions(-)
----------------------------------------------------------------------



[43/53] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-943

Posted by se...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-sprint-5' into ignite-943


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

Branch: refs/heads/ignite-sprint-5
Commit: eb81019604bedd80bf8101f2d65e278b0de88cc0
Parents: ac258a5 7ec4c82
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 14:32:27 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 14:32:27 2015 +0300

----------------------------------------------------------------------
 dev-tools/src/main/groovy/jiraslurp.groovy      |   3 +
 .../shmem/IpcSharedMemoryServerEndpoint.java    |   2 +-
 modules/mesos/README.txt                        |  28 +
 modules/mesos/licenses/apache-2.0.txt           | 202 ++++++++
 modules/mesos/licenses/jetty-epl-license.txt    |  69 +++
 modules/mesos/pom.xml                           | 101 ++++
 .../apache/ignite/mesos/ClusterProperties.java  | 519 +++++++++++++++++++
 .../apache/ignite/mesos/IgniteFramework.java    | 119 +++++
 .../apache/ignite/mesos/IgniteScheduler.java    | 361 +++++++++++++
 .../org/apache/ignite/mesos/IgniteTask.java     |  86 +++
 .../org/apache/ignite/mesos/package-info.java   |  22 +
 .../ignite/mesos/resource/IgniteProvider.java   | 234 +++++++++
 .../ignite/mesos/resource/JettyServer.java      |  61 +++
 .../ignite/mesos/resource/ResourceHandler.java  | 142 +++++
 .../ignite/mesos/resource/ResourceProvider.java | 120 +++++
 .../ignite/mesos/resource/package-info.java     |  22 +
 .../main/resources/ignite-default-config.xml    |  35 ++
 .../org/apache/ignite/IgniteMesosTestSuite.java |  38 ++
 .../ignite/mesos/IgniteSchedulerSelfTest.java   | 464 +++++++++++++++++
 parent/pom.xml                                  |   4 +
 pom.xml                                         |   1 +
 scripts/git-patch-prop.sh                       |   2 +-
 22 files changed, 2633 insertions(+), 2 deletions(-)
----------------------------------------------------------------------



[21/53] [abbrv] incubator-ignite git commit: # ignite-709 include internal keys for scan query

Posted by se...@apache.org.
# ignite-709 include internal keys for scan query


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

Branch: refs/heads/ignite-sprint-5
Commit: 581f4d99fc1dd31ca83631f2ceabd7187f1572fe
Parents: 9d3ab16
Author: sboikov <sb...@gridgain.com>
Authored: Thu May 28 12:01:31 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu May 28 14:51:16 2015 +0300

----------------------------------------------------------------------
 .../configuration/CacheConfiguration.java       |  1 -
 .../processors/cache/GridCacheAdapter.java      | 15 ++++++++-
 .../cache/GridCacheConcurrentMap.java           | 21 ++++++++++---
 .../processors/cache/GridCacheProcessor.java    | 32 +++++++++++---------
 .../processors/cache/GridCacheProxyImpl.java    | 12 ++++++++
 .../processors/cache/IgniteInternalCache.java   |  5 +++
 .../cache/query/GridCacheQueryAdapter.java      |  2 ++
 .../cache/query/GridCacheQueryErrorFuture.java  |  2 ++
 .../cache/query/GridCacheQueryManager.java      |  2 +-
 .../continuous/CacheContinuousQueryManager.java | 24 ++++++++++++---
 .../cacheobject/IgniteCacheObjectProcessor.java |  5 ++-
 .../IgniteCacheObjectProcessorImpl.java         |  2 +-
 12 files changed, 93 insertions(+), 30 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
index 5a03ed8..3f5d7b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/CacheConfiguration.java
@@ -25,7 +25,6 @@ import org.apache.ignite.cache.eviction.*;
 import org.apache.ignite.cache.query.annotations.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.plugin.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index bbd13f1..a8bf1f7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -902,7 +902,12 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public Set<K> keySet() {
-        return keySet((CacheEntryPredicate[]) null);
+        return keySet((CacheEntryPredicate[])null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public Set<K> keySetx() {
+        return keySetx((CacheEntryPredicate[])null);
     }
 
     /** {@inheritDoc} */
@@ -4299,6 +4304,14 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * @param filter Filters to evaluate.
+     * @return Key set including internal keys.
+     */
+    public Set<K> keySetx(@Nullable CacheEntryPredicate... filter) {
+        return map.keySetx(filter);
+    }
+
+    /**
      * @param filter Primary key set.
      * @return Primary key set.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
index bd3e0f2..db5eed1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMap.java
@@ -626,7 +626,19 @@ public class GridCacheConcurrentMap {
     public <K, V> Set<K> keySet(CacheEntryPredicate... filter) {
         checkWeakQueue();
 
-        return new KeySet<>(this, filter);
+        return new KeySet<>(this, filter, false);
+    }
+
+    /**
+     * Key set including internal keys.
+     *
+     * @param filter Filter.
+     * @return Set of the keys contained in this map.
+     */
+    public <K, V> Set<K> keySetx(CacheEntryPredicate... filter) {
+        checkWeakQueue();
+
+        return new KeySet<>(this, filter, true);
     }
 
     /**
@@ -1921,7 +1933,7 @@ public class GridCacheConcurrentMap {
 
         /** {@inheritDoc} */
         @Override public void clear() {
-            ctx.cache().clearLocally0(new KeySet<K, V>(map, filter));
+            ctx.cache().clearLocally0(new KeySet<K, V>(map, filter, false));
         }
 
         /** {@inheritDoc} */
@@ -2171,11 +2183,12 @@ public class GridCacheConcurrentMap {
         /**
          * @param map Base map.
          * @param filter Key filter.
+         * @param internal Whether to allow internal keys.
          */
-        private KeySet(GridCacheConcurrentMap map, CacheEntryPredicate[] filter) {
+        private KeySet(GridCacheConcurrentMap map, CacheEntryPredicate[] filter, boolean internal) {
             assert map != null;
 
-            set = new Set0<>(map, nonInternal(filter));
+            set = new Set0<>(map, internal ? filter : nonInternal(filter));
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index d9a7755..3065a2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -48,7 +48,6 @@ import org.apache.ignite.internal.processors.plugin.*;
 import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -666,8 +665,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void onKernalStart() throws IgniteCheckedException {
-        List<GridCacheAdapter<?, ?>> locCaches = new ArrayList<>(registeredCaches.size());
-
         try {
             if (ctx.config().isDaemon())
                 return;
@@ -735,9 +732,6 @@ public class GridCacheProcessor extends GridProcessorAdapter {
                     startCache(cache);
 
                     jCacheProxies.put(maskNull(name), new IgniteCacheProxy(ctx, cache, null, false));
-
-                    if (loc)
-                        locCaches.add(cache);
                 }
             }
         }
@@ -785,21 +779,31 @@ public class GridCacheProcessor extends GridProcessorAdapter {
         for (GridCacheAdapter<?, ?> cache : caches.values())
             onKernalStart(cache);
 
+        boolean utilityCacheStarted = false;
+
         // Wait for caches in SYNC preload mode.
-        for (GridCacheAdapter<?, ?> cache : locCaches) {
-            CacheConfiguration cfg = cache.configuration();
+        for (CacheConfiguration cfg : ctx.config().getCacheConfiguration()) {
+            GridCacheAdapter cache = caches.get(maskNull(cfg.getName()));
+
+            if (cache != null) {
+                if (cfg.getRebalanceMode() == SYNC) {
+                    if (cfg.getCacheMode() == REPLICATED ||
+                        (cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0)) {
+                        cache.preloader().syncFuture().get();
+
+                        if (CU.isUtilityCache(cache.name())) {
+                            ctx.cacheObjects().onUtilityCacheStarted();
 
-            if (cfg.getRebalanceMode() == SYNC) {
-                if (cfg.getCacheMode() == REPLICATED ||
-                    (cfg.getCacheMode() == PARTITIONED && cfg.getRebalanceDelay() >= 0))
-                    cache.preloader().syncFuture().get();
+                            utilityCacheStarted = true;
+                        }
+                    }
+                }
             }
         }
 
-        ctx.cacheObjects().onCacheProcessorStarted();
-
         assert caches.containsKey(CU.MARSH_CACHE_NAME) : "Marshaller cache should be started";
         assert caches.containsKey(CU.UTILITY_CACHE_NAME) : "Utility cache should be started";
+        assert utilityCacheStarted;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index 55d2f84..9a6d08a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -741,6 +741,18 @@ public class GridCacheProxyImpl<K, V> implements IgniteInternalCache<K, V>, Exte
     }
 
     /** {@inheritDoc} */
+    @Override public Set<K> keySetx() {
+        CacheOperationContext prev = gate.enter(opCtx);
+
+        try {
+            return delegate.keySetx();
+        }
+        finally {
+            gate.leave(prev);
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public Set<K> primaryKeySet() {
         CacheOperationContext prev = gate.enter(opCtx);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
index 5184115..ccce1b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteInternalCache.java
@@ -775,6 +775,11 @@ public interface IgniteInternalCache<K, V> extends Iterable<Cache.Entry<K, V>> {
     public Set<K> keySet();
 
     /**
+     * @return Set of keys including internal keys.
+     */
+    public Set<K> keySetx();
+
+    /**
      * Set of keys for which this node is primary.
      * This set is dynamic and may change with grid topology changes.
      * Note that this set will contain mappings for all keys, even if their values are

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
index 7e3fb26..fab490f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryAdapter.java
@@ -376,10 +376,12 @@ public class GridCacheQueryAdapter<T> implements CacheQuery<T> {
         return execute(null, rmtTransform, args);
     }
 
+    /** {@inheritDoc} */
     @Override public QueryMetrics metrics() {
         return metrics.copy();
     }
 
+    /** {@inheritDoc} */
     @Override public void resetMetrics() {
         metrics = new GridCacheQueryMetricsAdapter();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
index 2999e7b..15eb368 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
@@ -43,6 +43,8 @@ public class GridCacheQueryErrorFuture<T> extends GridFinishedFuture<Collection<
 
     /** {@inheritDoc} */
     @Nullable @Override public T next() throws IgniteCheckedException {
+        get();
+
         return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 16a8028..32e9d63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -773,7 +773,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
             private IgniteCacheExpiryPolicy expiryPlc = cctx.cache().expiryPolicy(plc);
 
-            private Iterator<K> iter = backups ? prj.keySet().iterator() : prj.primaryKeySet().iterator();
+            private Iterator<K> iter = backups ? prj.keySetx().iterator() : prj.primaryKeySet().iterator();
 
             {
                 advance();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
index 97fd7f3..6277c5d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/CacheContinuousQueryManager.java
@@ -250,8 +250,13 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
      * @return Continuous routine ID.
      * @throws IgniteCheckedException In case of error.
      */
-    public UUID executeQuery(CacheEntryUpdatedListener locLsnr, CacheEntryEventSerializableFilter rmtFilter,
-        int bufSize, long timeInterval, boolean autoUnsubscribe, ClusterGroup grp) throws IgniteCheckedException {
+    public UUID executeQuery(CacheEntryUpdatedListener locLsnr,
+        CacheEntryEventSerializableFilter rmtFilter,
+        int bufSize,
+        long timeInterval,
+        boolean autoUnsubscribe,
+        ClusterGroup grp) throws IgniteCheckedException
+    {
         return executeQuery0(
             locLsnr,
             rmtFilter,
@@ -357,9 +362,18 @@ public class CacheContinuousQueryManager extends GridCacheManagerAdapter {
      * @return Continuous routine ID.
      * @throws IgniteCheckedException In case of error.
      */
-    private UUID executeQuery0(CacheEntryUpdatedListener locLsnr, final CacheEntryEventSerializableFilter rmtFilter,
-        int bufSize, long timeInterval, boolean autoUnsubscribe, boolean internal, boolean notifyExisting,
-        boolean oldValRequired, boolean sync, boolean ignoreExpired, ClusterGroup grp) throws IgniteCheckedException {
+    private UUID executeQuery0(CacheEntryUpdatedListener locLsnr,
+        final CacheEntryEventSerializableFilter rmtFilter,
+        int bufSize,
+        long timeInterval,
+        boolean autoUnsubscribe,
+        boolean internal,
+        boolean notifyExisting,
+        boolean oldValRequired,
+        boolean sync,
+        boolean ignoreExpired,
+        ClusterGroup grp) throws IgniteCheckedException
+    {
         cctx.checkSecurity(SecurityPermission.CACHE_READ);
 
         if (grp == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
index a04692d..f8e5a60 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessor.java
@@ -18,12 +18,10 @@
 package org.apache.ignite.internal.processors.cacheobject;
 
 import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -32,8 +30,9 @@ import org.jetbrains.annotations.*;
 public interface IgniteCacheObjectProcessor extends GridProcessor {
     /**
      * @see GridComponent#onKernalStart()
+     * @throws IgniteCheckedException If failed.
      */
-    public void onCacheProcessorStarted();
+    public void onUtilityCacheStarted() throws IgniteCheckedException;
 
     /**
      * @param typeName Type name.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/581f4d99/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
index fe5a356..45fc121 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cacheobject/IgniteCacheObjectProcessorImpl.java
@@ -208,7 +208,7 @@ public class IgniteCacheObjectProcessorImpl extends GridProcessorAdapter impleme
     }
 
     /** {@inheritDoc} */
-    @Override public void onCacheProcessorStarted() {
+    @Override public void onUtilityCacheStarted() throws IgniteCheckedException {
         // No-op.
     }
 


[37/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Bug fix: NPE in setAuthenticator

Posted by se...@apache.org.
# IGNITE-943 Bug fix: NPE in setAuthenticator


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

Branch: refs/heads/ignite-sprint-5
Commit: 6b1265c2b103b4131b3ffc6deecd55d440fbe9b0
Parents: 2bc0795
Author: sevdokimov <se...@jetbrains.com>
Authored: Thu May 28 22:06:29 2015 +0300
Committer: sevdokimov <se...@jetbrains.com>
Committed: Thu May 28 22:06:29 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  5 -----
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 20 ++++++--------------
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  5 -----
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  5 ++++-
 4 files changed, 10 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b1265c2/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 2171085..e672d64 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
@@ -319,11 +319,6 @@ class ClientImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
-        // No-op.
-    }
-
-    /** {@inheritDoc} */
     @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
         if (segmented)
             throw new IgniteException("Failed to send custom message: client is disconnected");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b1265c2/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 b5c9519..57c13d6 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
@@ -115,9 +115,6 @@ class ServerImpl extends TcpDiscoveryImpl {
     /** Response on join request from coordinator (in case of duplicate ID or auth failure). */
     private final GridTuple<TcpDiscoveryAbstractMessage> joinRes = F.t1();
 
-    /** Node authenticator. */
-    private DiscoverySpiNodeAuthenticator nodeAuth;
-
     /** Mutex. */
     private final Object mux = new Object();
 
@@ -631,11 +628,6 @@ class ServerImpl extends TcpDiscoveryImpl {
     }
 
     /** {@inheritDoc} */
-    @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator nodeAuth) {
-        this.nodeAuth = nodeAuth;
-    }
-
-    /** {@inheritDoc} */
     @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
         try {
             msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, spi.marsh.marshal(evt)));
@@ -680,10 +672,10 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if (log.isDebugEnabled())
                     log.debug("Join request message has not been sent (local node is the first in the topology).");
 
-                if (nodeAuth != null) {
+                if (spi.nodeAuth != null) {
                     // Authenticate local node.
                     try {
-                        SecurityContext subj = nodeAuth.authenticateNode(locNode, locCred);
+                        SecurityContext subj = spi.nodeAuth.authenticateNode(locNode, locCred);
 
                         if (subj == null)
                             throw new IgniteSpiException("Authentication failed for local node: " + locNode.id());
@@ -2469,12 +2461,12 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
-                if (nodeAuth != null) {
+                if (spi.nodeAuth != null) {
                     // Authenticate node first.
                     try {
                         SecurityCredentials cred = unmarshalCredentials(node);
 
-                        SecurityContext subj = nodeAuth.authenticateNode(node, cred);
+                        SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred);
 
                         if (subj == null) {
                             // Node has not pass authentication.
@@ -2843,7 +2835,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
-                if (!isLocalNodeCoordinator() && nodeAuth != null && nodeAuth.isGlobalNodeAuthentication()) {
+                if (!isLocalNodeCoordinator() && spi.nodeAuth != null && spi.nodeAuth.isGlobalNodeAuthentication()) {
                     boolean authFailed = true;
 
                     try {
@@ -2861,7 +2853,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             authFailed = false;
                         }
                         else {
-                            SecurityContext subj = nodeAuth.authenticateNode(node, cred);
+                            SecurityContext subj = spi.nodeAuth.authenticateNode(node, cred);
 
                             SecurityContext coordSubj = spi.ignite().configuration().getMarshaller().unmarshal(
                                 node.<byte[]>attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b1265c2/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index 4836911..f285279 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -113,11 +113,6 @@ abstract class TcpDiscoveryImpl {
     public abstract void disconnect() throws IgniteSpiException;
 
     /**
-     * @param auth Auth.
-     */
-    public abstract void setAuthenticator(DiscoverySpiNodeAuthenticator auth);
-
-    /**
      * @param msg Message.
      */
     public abstract void sendCustomEvent(DiscoverySpiCustomMessage msg) throws IgniteException;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/6b1265c2/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 64e6001..1b8424c 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
@@ -294,6 +294,9 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
     protected long ipFinderCleanFreq = DFLT_IP_FINDER_CLEAN_FREQ;
 
+    /** Node authenticator. */
+    protected DiscoverySpiNodeAuthenticator nodeAuth;
+
     /** Context initialization latch. */
     @GridToStringExclude
     private final CountDownLatch ctxInitLatch = new CountDownLatch(1);
@@ -353,7 +356,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
 
     /** {@inheritDoc} */
     @Override public void setAuthenticator(DiscoverySpiNodeAuthenticator auth) {
-        impl.setAuthenticator(auth);
+        nodeAuth = auth;
     }
 
     /** {@inheritDoc} */


[42/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix TcpClientDiscoverySpiSelfTest.testMetrics

Posted by se...@apache.org.
# IGNITE-943 Fix TcpClientDiscoverySpiSelfTest.testMetrics


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

Branch: refs/heads/ignite-sprint-5
Commit: ac258a5e06d3c8c9cca82b6883fc3af505a5e5dd
Parents: 2477c0e
Author: sevdokimov <se...@gridgain.com>
Authored: Fri May 29 13:47:00 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Fri May 29 13:47:00 2015 +0300

----------------------------------------------------------------------
 .../discovery/GridDiscoveryManagerAliveCacheSelfTest.java      | 2 +-
 .../processors/service/ClosureServiceClientsNodesTest.java     | 6 +-----
 2 files changed, 2 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac258a5e/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index af4dd0f..f2afb07 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -52,7 +52,7 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
     private static final int TMP_NODES_CNT = 3;
 
     /** */
-    private static final int ITERATIONS = 20;
+    private static final int ITERATIONS = 10;
 
     /** */
     private int gridCntr;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ac258a5e/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
index 761f00f..59594be 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/ClosureServiceClientsNodesTest.java
@@ -53,11 +53,7 @@ public class ClosureServiceClientsNodesTest extends GridCommonAbstractTest {
 
         cfg.setMarshaller(new OptimizedMarshaller(false));
 
-        TcpDiscoverySpi discoSpi = new TcpDiscoverySpi();
-
-        discoSpi.setIpFinder(ipFinder);
-
-        cfg.setDiscoverySpi(discoSpi);
+        cfg.setDiscoverySpi(new TcpDiscoverySpi().setIpFinder(ipFinder).setForceServerMode(true));
 
         cfg.setCacheConfiguration();
 


[39/53] [abbrv] incubator-ignite git commit: # ignite-709

Posted by se...@apache.org.
# ignite-709


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

Branch: refs/heads/ignite-sprint-5
Commit: 07b6cb598ddb8f6c8ca76245ffbe6c434e08f870
Parents: 2c3ce15
Author: sboikov <sb...@gridgain.com>
Authored: Fri May 29 11:26:45 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri May 29 11:34:51 2015 +0300

----------------------------------------------------------------------
 .../dht/preloader/GridDhtPartitionMap.java      |  2 +-
 ...niteCacheClientNodeChangingTopologyTest.java |  6 ++
 .../junits/common/GridCommonAbstractTest.java   | 61 +++++++++++++++++---
 3 files changed, 60 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07b6cb59/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
index facf7e3..faa6cf6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionMap.java
@@ -237,7 +237,7 @@ public class GridDhtPartitionMap implements Comparable<GridDhtPartitionMap>, Ext
      * @return Full string representation.
      */
     public String toFullString() {
-        return S.toString(GridDhtPartitionMap.class, this, "size", size(), "map", super.toString());
+        return S.toString(GridDhtPartitionMap.class, this, "size", size(), "map", map.toString());
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07b6cb59/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
index 47c1d7e..922e618 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheClientNodeChangingTopologyTest.java
@@ -1556,6 +1556,9 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
                 catch (TimeoutException e) {
                     log.error("Failed to wait for update.");
 
+                    for (Ignite ignite : G.allGrids())
+                        dumpCacheDebugInfo(ignite);
+
                     U.dumpThreads(log);
 
                     CyclicBarrier barrier0 = updateBarrier;
@@ -1593,6 +1596,9 @@ public class IgniteCacheClientNodeChangingTopologyTest extends GridCommonAbstrac
                 catch (TimeoutException e) {
                     log.error("Failed to wait for update.");
 
+                    for (Ignite ignite : G.allGrids())
+                        dumpCacheDebugInfo(ignite);
+
                     U.dumpThreads(log);
 
                     CyclicBarrier barrier0 = updateBarrier;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/07b6cb59/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index 1f4c7b6..cded7fa 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.colocated.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.processors.cache.local.*;
+import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
@@ -383,19 +384,31 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                             int actual = owners.size();
 
                             if (affNodes.size() != owners.size() || !affNodes.containsAll(owners)) {
-                                LT.warn(log(), null, "Waiting for topology map update [grid=" + g.name() +
-                                    ", cache=" + cfg.getName() + ", p=" + p + ", nodes=" + exp + ", owners=" + actual +
-                                    ", affNodes=" + affNodes + ", owners=" + owners +
-                                    ", locNode=" + g.cluster().localNode().id() + ']');
+                                LT.warn(log(), null, "Waiting for topology map update [" +
+                                    "grid=" + g.name() +
+                                    ", cache=" + cfg.getName() +
+                                    ", cacheId=" + dht.context().cacheId() +
+                                    ", p=" + p +
+                                    ", affNodesCnt=" + exp +
+                                    ", ownersCnt=" + actual +
+                                    ", affNodes=" + affNodes +
+                                    ", owners=" + owners +
+                                    ", locNode=" + g.cluster().localNode() + ']');
 
                                 if (i == 0)
                                     start = System.currentTimeMillis();
 
                                 if (System.currentTimeMillis() - start > 30_000)
-                                    throw new IgniteException("Timeout of waiting for topology map update [grid="
-                                        + g.name() + ", p=" + p + ", nodes=" + exp + ", owners=" + actual +
-                                            ", affNodes=" + affNodes + ", owners=" + owners + ", locNode="
-                                        + g.cluster().localNode().id() + ']');
+                                    throw new IgniteException("Timeout of waiting for topology map update [" +
+                                        "grid=" + g.name() +
+                                        ", cache=" + cfg.getName() +
+                                        ", cacheId=" + dht.context().cacheId() +
+                                        ", p=" + p +
+                                        ", affNodesCnt=" + exp +
+                                        ", ownersCnt=" + actual +
+                                        ", affNodes=" + affNodes +
+                                        ", owners=" + owners +
+                                        ", locNode=" + g.cluster().localNode() + ']');
 
                                 Thread.sleep(200); // Busy wait.
 
@@ -415,6 +428,38 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     }
 
     /**
+     * @param ignite Node.
+     */
+    public void dumpCacheDebugInfo(Ignite ignite) {
+        GridKernalContext ctx = ((IgniteKernal)ignite).context();
+
+        log.error("Cache information update [node=" + ignite.name() +
+            ", client=" + ignite.configuration().isClientMode() + ']');
+
+        GridCacheSharedContext cctx = ctx.cache().context();
+
+        log.error("Pending transactions:");
+
+        for (IgniteInternalTx tx : cctx.tm().activeTransactions())
+            log.error(">>> " + tx);
+
+        log.error("Pending explicit locks:");
+
+        for (GridCacheExplicitLockSpan lockSpan : cctx.mvcc().activeExplicitLocks())
+            log.error(">>> " + lockSpan);
+
+        log.error("Pending cache futures:");
+
+        for (GridCacheFuture<?> fut : cctx.mvcc().activeFutures())
+            log.error(">>> " + fut);
+
+        log.error("Pending atomic cache futures:");
+
+        for (GridCacheFuture<?> fut : cctx.mvcc().atomicFutures())
+            log.error(">>> " + fut);
+    }
+
+    /**
      * @param cache Cache.
      * @return Affinity.
      */


[33/53] [abbrv] incubator-ignite git commit: # ignite-857 review

Posted by se...@apache.org.
# ignite-857 review


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

Branch: refs/heads/ignite-sprint-5
Commit: 9cddb6fe669ef9f4ac6792a306a665fa2037f773
Parents: 3dea5f1
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu May 28 17:58:49 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu May 28 17:58:49 2015 +0300

----------------------------------------------------------------------
 .../managers/discovery/GridDiscoveryManager.java  |  3 ++-
 .../apache/ignite/spi/discovery/DiscoverySpi.java |  3 ++-
 .../ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 18 ++++++++++++------
 .../spi/discovery/tcp/TcpDiscoverySpiMBean.java   |  2 +-
 ...itionedClientOnlyNoPrimaryFullApiSelfTest.java |  3 ++-
 5 files changed, 19 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index a910950..4ef602e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -285,7 +285,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /** {@inheritDoc} */
     @Override protected void onKernalStart0() throws IgniteCheckedException {
         if (Boolean.TRUE.equals(ctx.config().isClientMode()) && !getSpi().isClientMode())
-            ctx.performance().add("Enable client mode for TcpDiscoverySpi (set TcpDiscoverySpi.forceServerMode to true)");
+            ctx.performance().add("Enable client mode for TcpDiscoverySpi " +
+                    "(set TcpDiscoverySpi.forceServerMode to false)");
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
index e7fbadc..b952087 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/DiscoverySpi.java
@@ -78,7 +78,8 @@ public interface DiscoverySpi extends IgniteSpi {
     /**
      * Sets node attributes and node version which will be distributed in grid during
      * join process. Note that these attributes cannot be changed and set only once.
-     *  @param attrs Map of node attributes.
+     *
+     * @param attrs Map of node attributes.
      * @param ver Product version.
      */
     public void setNodeAttributes(Map<String, Object> attrs, IgniteProductVersion ver);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/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 52ea78c..64e6001 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
@@ -54,10 +54,12 @@ import java.util.concurrent.atomic.*;
  * done across it.
  * <p>
  * If node is configured as client node (see {@link IgniteConfiguration#clientMode})
- * TcpDiscoverySpi starts in client mode too. In this case node does not insert to the ring,
- * it connects to any node in the ring router and communicated with that node only.
- * Thereby slowing or shutdown of client node will not affect whole cluster. If you want to start TcpDiscoverySpi in
- * server mode regardless {@link IgniteConfiguration#clientMode} you can set {@link #forceSrvMode} to true.
+ * TcpDiscoverySpi starts in client mode as well. In this case node does not take its place in the ring,
+ * but it connects to random node in the ring (IP taken from IP finder configured) and
+ * use it as a router for discovery traffic.
+ * Therefore slow client node or its shutdown will not affect whole cluster. If TcpDiscoverySpi
+ * needs to be started in server mode regardless of {@link IgniteConfiguration#clientMode},
+ * {@link #forceSrvMode} should be set to true.
  * <p>
  * At startup SPI tries to send messages to random IP taken from
  * {@link TcpDiscoveryIpFinder} about self start (stops when send succeeds)
@@ -388,11 +390,15 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     }
 
     /**
-     * If {@code true} TcpDiscoverySpi will started in server mode regardless
-     * of {@link IgniteConfiguration#isClientMode()}
+     * Sets force server mode flag.
+     * <p>
+     * If {@code true} TcpDiscoverySpi is started in server mode regardless
+     * of {@link IgniteConfiguration#isClientMode()}.
      *
      * @param forceSrvMode forceServerMode flag.
+     * @return {@code this} for chaining.
      */
+    @IgniteSpiConfiguration(optional = true)
     public TcpDiscoverySpi setForceServerMode(boolean forceSrvMode) {
         this.forceSrvMode = forceSrvMode;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
index 6f2ea6f..f338fab 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiMBean.java
@@ -277,7 +277,7 @@ public interface TcpDiscoverySpiMBean extends IgniteSpiManagementMBean {
      * Whether or not discovery is started in client mode.
      *
      * @return {@code true} if node is in client mode.
-     * @throws IllegalStateException If discovery SPI has not started.
+     * @throws IllegalStateException If discovery SPI is not started.
      */
     @MXBeanDescription("Client mode.")
     public boolean isClientMode() throws IllegalStateException;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/9cddb6fe/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
index 931e347..6b7d1ad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
@@ -65,7 +65,8 @@ public class GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest extends Grid
     @Override protected IgniteClosure<Throwable, Throwable> errorHandler() {
         return new IgniteClosure<Throwable, Throwable>() {
             @Override public Throwable apply(Throwable e) {
-                if (e instanceof IgniteException || e instanceof IgniteCheckedException || X.hasCause(e, ClusterTopologyCheckedException.class)) {
+                if (e instanceof IgniteException || e instanceof IgniteCheckedException ||
+                    X.hasCause(e, ClusterTopologyCheckedException.class)) {
                     info("Discarding exception: " + e);
 
                     return null;


[31/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix tests

Posted by se...@apache.org.
# IGNITE-943 Fix  tests


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

Branch: refs/heads/ignite-sprint-5
Commit: 110573bcbd6e4bee94c35cc912685e9225b602d4
Parents: 7f98e29
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 17:19:01 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 17:19:01 2015 +0300

----------------------------------------------------------------------
 .../GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java    | 2 ++
 1 file changed, 2 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/110573bc/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
index c79c32a..931e347 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.cluster.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.discovery.tcp.*;
 
 import java.util.*;
 
@@ -38,6 +39,7 @@ public class GridCachePartitionedClientOnlyNoPrimaryFullApiSelfTest extends Grid
     @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
         IgniteConfiguration cfg = super.getConfiguration(gridName);
 
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setForceServerMode(true);
         cfg.setClientMode(true);
 
         return cfg;


[08/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix tests.

Posted by se...@apache.org.
# IGNITE-943 Fix tests.


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

Branch: refs/heads/ignite-sprint-5
Commit: 4117f686300ca138fcda4a5adbe6ee0c2fc48ff4
Parents: bf0e157
Author: sevdokimov <se...@gridgain.com>
Authored: Wed May 27 17:17:53 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Wed May 27 17:17:53 2015 +0300

----------------------------------------------------------------------
 .../java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4117f686/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 922c496..90ab05a 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
@@ -1582,7 +1582,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
 
         unregisterMBean();
 
-        impl.spiStop();
+        if (impl != null)
+            impl.spiStop();
     }
 
     /**


[16/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Rename TcpDiscoveryImpl.adapter to spi

Posted by se...@apache.org.
# IGNITE-943 Rename TcpDiscoveryImpl.adapter to spi


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

Branch: refs/heads/ignite-sprint-5
Commit: 0e192ef84def1cdfe121e1e132c0b9740341fb9c
Parents: 92b2a57
Author: sevdokimov <se...@gridgain.com>
Authored: Thu May 28 12:11:22 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Thu May 28 14:46:07 2015 +0300

----------------------------------------------------------------------
 .../ignite/spi/discovery/tcp/ClientImpl.java    | 138 ++++----
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 348 +++++++++----------
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  14 +-
 3 files changed, 250 insertions(+), 250 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0e192ef8/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 aa254ec..2171085 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
@@ -116,7 +116,7 @@ class ClientImpl extends TcpDiscoveryImpl {
         b.append("    Message worker: ").append(threadStatus(msgWorker)).append(U.nl());
         b.append("    Socket reader: ").append(threadStatus(sockReader)).append(U.nl());
         b.append("    Socket writer: ").append(threadStatus(sockWriter)).append(U.nl());
-        b.append("    Socket timeout worker: ").append(threadStatus(adapter.sockTimeoutWorker)).append(U.nl());
+        b.append("    Socket timeout worker: ").append(threadStatus(spi.sockTimeoutWorker)).append(U.nl());
 
         b.append(U.nl());
 
@@ -127,7 +127,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         b.append(U.nl());
 
-        b.append("Stats: ").append(adapter.stats).append(U.nl());
+        b.append("Stats: ").append(spi.stats).append(U.nl());
 
         U.quietAndInfo(log, b.toString());
     }
@@ -153,9 +153,9 @@ class ClientImpl extends TcpDiscoveryImpl {
 
     /** {@inheritDoc} */
     @Override public void spiStart(@Nullable String gridName) throws IgniteSpiException {
-        adapter.initLocalNode(0, true);
+        spi.initLocalNode(0, true);
 
-        locNode = adapter.locNode;
+        locNode = spi.locNode;
 
         sockWriter = new SocketWriter();
         sockWriter.start();
@@ -176,9 +176,9 @@ class ClientImpl extends TcpDiscoveryImpl {
             throw new IgniteSpiException("Thread has been interrupted.", e);
         }
 
-        timer.schedule(new HeartbeatSender(), adapter.hbFreq, adapter.hbFreq);
+        timer.schedule(new HeartbeatSender(), spi.hbFreq, spi.hbFreq);
 
-        adapter.printStartInfo();
+        spi.printStartInfo();
     }
 
     /** {@inheritDoc} */
@@ -189,7 +189,7 @@ class ClientImpl extends TcpDiscoveryImpl {
             msgWorker.addMessage(SPI_STOP);
 
             try {
-                if (!leaveLatch.await(adapter.netTimeout, MILLISECONDS))
+                if (!leaveLatch.await(spi.netTimeout, MILLISECONDS))
                     U.warn(log, "Failed to left node: timeout [nodeId=" + locNode + ']');
             }
             catch (InterruptedException ignored) {
@@ -210,7 +210,7 @@ class ClientImpl extends TcpDiscoveryImpl {
         U.join(sockWriter, log);
         U.join(sockReader, log);
 
-        adapter.printStopInfo();
+        spi.printStopInfo();
     }
 
     /** {@inheritDoc} */
@@ -253,7 +253,7 @@ class ClientImpl extends TcpDiscoveryImpl {
             if (oldFut != null)
                 fut = oldFut;
             else {
-                if (adapter.getSpiContext().isStopping()) {
+                if (spi.getSpiContext().isStopping()) {
                     if (pingFuts.remove(nodeId, fut))
                         fut.onDone(false);
 
@@ -267,7 +267,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                         if (pingFuts.remove(nodeId, finalFut))
                             finalFut.onDone(false);
                     }
-                }, adapter.netTimeout);
+                }, spi.netTimeout);
 
                 sockWriter.sendMessage(new TcpDiscoveryClientPingRequest(getLocalNodeId(), nodeId));
             }
@@ -297,13 +297,13 @@ class ClientImpl extends TcpDiscoveryImpl {
         leaveLatch.countDown();
         joinLatch.countDown();
 
-        adapter.getSpiContext().deregisterPorts();
+        spi.getSpiContext().deregisterPorts();
 
         Collection<ClusterNode> rmts = getRemoteNodes();
 
         // This is restart/disconnection and remote nodes are not empty.
         // We need to fire FAIL event for each.
-        DiscoverySpiListener lsnr = adapter.lsnr;
+        DiscoverySpiListener lsnr = spi.lsnr;
 
         if (lsnr != null) {
             for (ClusterNode n : rmts) {
@@ -330,7 +330,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         try {
             sockWriter.sendMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt,
-                adapter.marsh.marshal(evt)));
+                spi.marsh.marshal(evt)));
         }
         catch (IgniteCheckedException e) {
             throw new IgniteSpiException("Failed to marshal custom event: " + evt, e);
@@ -364,16 +364,16 @@ class ClientImpl extends TcpDiscoveryImpl {
                 throw new InterruptedException();
 
             while (addrs == null || addrs.isEmpty()) {
-                addrs = adapter.resolvedAddresses();
+                addrs = spi.resolvedAddresses();
 
                 if (!F.isEmpty(addrs)) {
                     if (log.isDebugEnabled())
                         log.debug("Resolved addresses from IP finder: " + addrs);
                 }
                 else {
-                    U.warn(log, "No addresses registered in the IP finder (will retry in 2000ms): " + adapter.ipFinder);
+                    U.warn(log, "No addresses registered in the IP finder (will retry in 2000ms): " + spi.ipFinder);
 
-                    if (adapter.joinTimeout > 0 && (U.currentTimeMillis() - startTime) > adapter.joinTimeout)
+                    if (spi.joinTimeout > 0 && (U.currentTimeMillis() - startTime) > spi.joinTimeout)
                         return null;
 
                     Thread.sleep(2000);
@@ -401,20 +401,20 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     UUID rmtNodeId = t.get2();
 
-                    adapter.stats.onClientSocketInitialized(U.currentTimeMillis() - ts);
+                    spi.stats.onClientSocketInitialized(U.currentTimeMillis() - ts);
 
                     locNode.clientRouterNodeId(rmtNodeId);
 
                     TcpDiscoveryAbstractMessage msg = recon ?
                         new TcpDiscoveryClientReconnectMessage(getLocalNodeId(), rmtNodeId,
                             lastMsgId) :
-                        new TcpDiscoveryJoinRequestMessage(locNode, adapter.collectExchangeData(getLocalNodeId()));
+                        new TcpDiscoveryJoinRequestMessage(locNode, spi.collectExchangeData(getLocalNodeId()));
 
                     msg.client(true);
 
-                    adapter.writeToSocket(sock, msg);
+                    spi.writeToSocket(sock, msg);
 
-                    int res = adapter.readReceipt(sock, adapter.ackTimeout);
+                    int res = spi.readReceipt(sock, spi.ackTimeout);
 
                     switch (res) {
                         case RES_OK:
@@ -447,7 +447,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                 U.warn(log, "Failed to connect to any address from IP finder (will retry to join topology " +
                     "in 2000ms): " + addrs0);
 
-                if (adapter.joinTimeout > 0 && (U.currentTimeMillis() - startTime) > adapter.joinTimeout)
+                if (spi.joinTimeout > 0 && (U.currentTimeMillis() - startTime) > spi.joinTimeout)
                     return null;
 
                 Thread.sleep(2000);
@@ -470,11 +470,11 @@ class ClientImpl extends TcpDiscoveryImpl {
 
             topHist.put(topVer, allNodes);
 
-            if (topHist.size() > adapter.topHistSize)
+            if (topHist.size() > spi.topHistSize)
                 topHist.pollFirstEntry();
 
             assert topHist.lastKey() == topVer;
-            assert topHist.size() <= adapter.topHistSize;
+            assert topHist.size() <= spi.topHistSize;
         }
 
         return allNodes;
@@ -505,15 +505,15 @@ class ClientImpl extends TcpDiscoveryImpl {
     private IgniteBiTuple<Socket, UUID> initConnection(InetSocketAddress addr) throws IOException, IgniteCheckedException {
         assert addr != null;
 
-        Socket sock = adapter.openSocket(addr);
+        Socket sock = spi.openSocket(addr);
 
         TcpDiscoveryHandshakeRequest req = new TcpDiscoveryHandshakeRequest(getLocalNodeId());
 
         req.client(true);
 
-        adapter.writeToSocket(sock, req);
+        spi.writeToSocket(sock, req);
 
-        TcpDiscoveryHandshakeResponse res = adapter.readMessage(sock, null, adapter.ackTimeout);
+        TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, spi.ackTimeout);
 
         UUID nodeId = res.creatorNodeId();
 
@@ -529,11 +529,11 @@ class ClientImpl extends TcpDiscoveryImpl {
 
         U.interrupt(sockWriter);
         U.interrupt(msgWorker);
-        U.interrupt(adapter.sockTimeoutWorker);
+        U.interrupt(spi.sockTimeoutWorker);
 
         U.join(sockWriter, log);
         U.join(msgWorker, log);
-        U.join(adapter.sockTimeoutWorker, log);
+        U.join(spi.sockTimeoutWorker, log);
     }
 
     /** {@inheritDoc} */
@@ -569,9 +569,9 @@ class ClientImpl extends TcpDiscoveryImpl {
     private class HeartbeatSender extends TimerTask {
         /** {@inheritDoc} */
         @Override public void run() {
-            if (!adapter.getSpiContext().isStopping() && sockWriter.isOnline()) {
+            if (!spi.getSpiContext().isStopping() && sockWriter.isOnline()) {
                 TcpDiscoveryClientHeartbeatMessage msg = new TcpDiscoveryClientHeartbeatMessage(getLocalNodeId(),
-                    adapter.metricsProvider.metrics());
+                    spi.metricsProvider.metrics());
 
                 msg.client(true);
 
@@ -596,7 +596,7 @@ class ClientImpl extends TcpDiscoveryImpl {
         /**
          */
         protected SocketReader() {
-            super(adapter.ignite().name(), "tcp-client-disco-sock-reader", log);
+            super(spi.ignite().name(), "tcp-client-disco-sock-reader", log);
         }
 
         /**
@@ -640,7 +640,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                         TcpDiscoveryAbstractMessage msg;
 
                         try {
-                            msg = adapter.marsh.unmarshal(in, U.gridClassLoader());
+                            msg = spi.marsh.unmarshal(in, U.gridClassLoader());
                         }
                         catch (IgniteCheckedException e) {
                             if (log.isDebugEnabled())
@@ -670,9 +670,9 @@ class ClientImpl extends TcpDiscoveryImpl {
                         if (log.isDebugEnabled())
                             log.debug("Message has been received: " + msg);
 
-                        adapter.stats.onMessageReceived(msg);
+                        spi.stats.onMessageReceived(msg);
 
-                        if (adapter.ensured(msg))
+                        if (spi.ensured(msg))
                             lastMsgId = msg.id();
 
                         msgWorker.addMessage(msg);
@@ -715,7 +715,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          *
          */
         protected SocketWriter() {
-            super(adapter.ignite().name(), "tcp-client-disco-sock-writer", log);
+            super(spi.ignite().name(), "tcp-client-disco-sock-writer", log);
         }
 
         /**
@@ -775,11 +775,11 @@ class ClientImpl extends TcpDiscoveryImpl {
                     }
                 }
 
-                for (IgniteInClosure<TcpDiscoveryAbstractMessage> msgLsnr : adapter.sendMsgLsnrs)
+                for (IgniteInClosure<TcpDiscoveryAbstractMessage> msgLsnr : spi.sendMsgLsnrs)
                     msgLsnr.apply(msg);
 
                 try {
-                    adapter.writeToSocket(sock, msg);
+                    spi.writeToSocket(sock, msg);
 
                     msg = null;
                 }
@@ -814,7 +814,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          *
          */
         protected Reconnector() {
-            super(adapter.ignite().name(), "tcp-client-disco-msg-worker", log);
+            super(spi.ignite().name(), "tcp-client-disco-msg-worker", log);
         }
 
         /**
@@ -851,7 +851,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                 // Wait for
                 while (!isInterrupted()) {
-                    TcpDiscoveryAbstractMessage msg = adapter.marsh.unmarshal(in, U.gridClassLoader());
+                    TcpDiscoveryAbstractMessage msg = spi.marsh.unmarshal(in, U.gridClassLoader());
 
                     if (msg instanceof TcpDiscoveryClientReconnectMessage) {
                         TcpDiscoveryClientReconnectMessage res = (TcpDiscoveryClientReconnectMessage)msg;
@@ -902,13 +902,13 @@ class ClientImpl extends TcpDiscoveryImpl {
          *
          */
         private MessageWorker() {
-            super(adapter.ignite().name(), "tcp-client-disco-msg-worker", log);
+            super(spi.ignite().name(), "tcp-client-disco-msg-worker", log);
         }
 
         /** {@inheritDoc} */
         @SuppressWarnings("InfiniteLoopStatement")
         @Override protected void body() throws InterruptedException {
-            adapter.stats.onJoinStarted();
+            spi.stats.onJoinStarted();
 
             try {
                 final Socket sock = joinTopology(false);
@@ -930,7 +930,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                         if (joinLatch.getCount() > 0)
                             queue.add(JOIN_TIMEOUT);
                     }
-                }, adapter.netTimeout);
+                }, spi.netTimeout);
 
                 sockReader.setSocket(sock, locNode.clientRouterNodeId());
 
@@ -940,7 +940,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                     if (msg == JOIN_TIMEOUT) {
                         if (joinLatch.getCount() > 0) {
                             joinErr = new IgniteSpiException("Join process timed out [sock=" + sock +
-                                ", timeout=" + adapter.netTimeout + ']');
+                                ", timeout=" + spi.netTimeout + ']');
 
                             joinLatch.countDown();
 
@@ -948,7 +948,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                         }
                     }
                     else if (msg == SPI_STOP) {
-                        assert adapter.getSpiContext().isStopping();
+                        assert spi.getSpiContext().isStopping();
 
                         if (currSock != null) {
                             TcpDiscoveryAbstractMessage leftMsg = new TcpDiscoveryNodeLeftMessage(getLocalNodeId());
@@ -972,7 +972,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                                 break;
                             }
                             else {
-                                if (adapter.getSpiContext().isStopping() || segmented)
+                                if (spi.getSpiContext().isStopping() || segmented)
                                     leaveLatch.countDown();
                                 else {
                                     assert reconnector == null;
@@ -986,7 +986,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                                             if (reconnector.isAlive())
                                                 reconnector.cancel();
                                         }
-                                    }, adapter.netTimeout);
+                                    }, spi.netTimeout);
                                 }
                             }
                         }
@@ -1008,11 +1008,11 @@ class ClientImpl extends TcpDiscoveryImpl {
                             IgniteSpiException err = null;
 
                             if (discoMsg instanceof TcpDiscoveryDuplicateIdMessage)
-                                err = adapter.duplicateIdError((TcpDiscoveryDuplicateIdMessage)msg);
+                                err = spi.duplicateIdError((TcpDiscoveryDuplicateIdMessage)msg);
                             else if (discoMsg instanceof TcpDiscoveryAuthFailedMessage)
-                                err = adapter.authenticationFailedError((TcpDiscoveryAuthFailedMessage)msg);
+                                err = spi.authenticationFailedError((TcpDiscoveryAuthFailedMessage)msg);
                             else if (discoMsg instanceof TcpDiscoveryCheckFailedMessage)
-                                err = adapter.checkFailedError((TcpDiscoveryCheckFailedMessage)msg);
+                                err = spi.checkFailedError((TcpDiscoveryCheckFailedMessage)msg);
 
                             if (err != null) {
                                 joinErr = err;
@@ -1052,7 +1052,7 @@ class ClientImpl extends TcpDiscoveryImpl {
             assert msg != null;
             assert msg.verified() || msg.senderNodeId() == null;
 
-            adapter.stats.onMessageProcessingStarted(msg);
+            spi.stats.onMessageProcessingStarted(msg);
 
             if (msg instanceof TcpDiscoveryNodeAddedMessage)
                 processNodeAddedMessage((TcpDiscoveryNodeAddedMessage)msg);
@@ -1073,14 +1073,14 @@ class ClientImpl extends TcpDiscoveryImpl {
             else if (msg instanceof TcpDiscoveryPingRequest)
                 processPingRequest();
 
-            adapter.stats.onMessageProcessingFinished(msg);
+            spi.stats.onMessageProcessingFinished(msg);
         }
 
         /**
          * @param msg Message.
          */
         private void processNodeAddedMessage(TcpDiscoveryNodeAddedMessage msg) {
-            if (adapter.getSpiContext().isStopping())
+            if (spi.getSpiContext().isStopping())
                 return;
 
             TcpDiscoveryNode node = msg.node();
@@ -1092,7 +1092,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                     Collection<TcpDiscoveryNode> top = msg.topology();
 
                     if (top != null) {
-                        adapter.gridStartTime = msg.gridStartTime();
+                        spi.gridStartTime = msg.gridStartTime();
 
                         for (TcpDiscoveryNode n : top) {
                             if (n.order() > 0)
@@ -1123,7 +1123,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                     Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
 
                     if (data != null)
-                        adapter.onExchange(newNodeId, newNodeId, data, null);
+                        spi.onExchange(newNodeId, newNodeId, data, null);
                 }
             }
         }
@@ -1132,7 +1132,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          * @param msg Message.
          */
         private void processNodeAddFinishedMessage(TcpDiscoveryNodeAddFinishedMessage msg) {
-            if (adapter.getSpiContext().isStopping())
+            if (spi.getSpiContext().isStopping())
                 return;
 
             if (getLocalNodeId().equals(msg.nodeId())) {
@@ -1141,7 +1141,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     if (dataMap != null) {
                         for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
-                            adapter.onExchange(getLocalNodeId(), entry.getKey(), entry.getValue(), null);
+                            spi.onExchange(getLocalNodeId(), entry.getKey(), entry.getValue(), null);
                     }
 
                     locNode.setAttributes(msg.clientNodeAttributes());
@@ -1157,7 +1157,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     joinLatch.countDown();
 
-                    adapter.stats.onJoinFinished();
+                    spi.stats.onJoinFinished();
                 }
                 else if (log.isDebugEnabled())
                     log.debug("Discarding node add finished message (this message has already been processed) " +
@@ -1178,8 +1178,8 @@ class ClientImpl extends TcpDiscoveryImpl {
                 node.order(topVer);
                 node.visible(true);
 
-                if (adapter.locNodeVer.equals(node.version()))
-                    node.version(adapter.locNodeVer);
+                if (spi.locNodeVer.equals(node.version()))
+                    node.version(spi.locNodeVer);
 
                 NavigableSet<ClusterNode> top = updateTopologyHistory(topVer);
 
@@ -1192,7 +1192,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                 notifyDiscovery(EVT_NODE_JOINED, topVer, node, top);
 
-                adapter.stats.onNodeJoined();
+                spi.stats.onNodeJoined();
             }
         }
 
@@ -1207,7 +1207,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                 leaveLatch.countDown();
             }
             else {
-                if (adapter.getSpiContext().isStopping())
+                if (spi.getSpiContext().isStopping())
                     return;
 
                 TcpDiscoveryNode node = rmtNodes.remove(msg.creatorNodeId());
@@ -1230,7 +1230,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                 notifyDiscovery(EVT_NODE_LEFT, msg.topologyVersion(), node, top);
 
-                adapter.stats.onNodeLeft();
+                spi.stats.onNodeLeft();
             }
         }
 
@@ -1238,7 +1238,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          * @param msg Message.
          */
         private void processNodeFailedMessage(TcpDiscoveryNodeFailedMessage msg) {
-            if (adapter.getSpiContext().isStopping()) {
+            if (spi.getSpiContext().isStopping()) {
                 if (!getLocalNodeId().equals(msg.creatorNodeId()) && getLocalNodeId().equals(msg.failedNodeId())) {
                     if (leaveLatch.getCount() > 0) {
                         log.debug("Remote node fail this node while node is stopping [locNode=" + getLocalNodeId()
@@ -1272,7 +1272,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                 notifyDiscovery(EVT_NODE_FAILED, msg.topologyVersion(), node, top);
 
-                adapter.stats.onNodeFailed();
+                spi.stats.onNodeFailed();
             }
         }
 
@@ -1280,7 +1280,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          * @param msg Message.
          */
         private void processHeartbeatMessage(TcpDiscoveryHeartbeatMessage msg) {
-            if (adapter.getSpiContext().isStopping())
+            if (spi.getSpiContext().isStopping())
                 return;
 
             if (getLocalNodeId().equals(msg.creatorNodeId())) {
@@ -1314,7 +1314,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          * @param msg Message.
          */
         private void processClientReconnectMessage(TcpDiscoveryClientReconnectMessage msg) {
-            if (adapter.getSpiContext().isStopping())
+            if (spi.getSpiContext().isStopping())
                 return;
 
             if (getLocalNodeId().equals(msg.creatorNodeId())) {
@@ -1346,7 +1346,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          */
         private void processCustomMessage(TcpDiscoveryCustomEventMessage msg) {
             if (msg.verified() && joinLatch.getCount() == 0) {
-                DiscoverySpiListener lsnr = adapter.lsnr;
+                DiscoverySpiListener lsnr = spi.lsnr;
 
                 if (lsnr != null) {
                     UUID nodeId = msg.creatorNodeId();
@@ -1355,7 +1355,7 @@ class ClientImpl extends TcpDiscoveryImpl {
 
                     if (node != null && node.visible()) {
                         try {
-                            DiscoverySpiCustomMessage msgObj = msg.message(adapter.marsh);
+                            DiscoverySpiCustomMessage msgObj = msg.message(spi.marsh);
 
                             notifyDiscovery(EVT_DISCOVERY_CUSTOM_EVT, topVer, node, allVisibleNodes(), msgObj);
                         }
@@ -1437,7 +1437,7 @@ class ClientImpl extends TcpDiscoveryImpl {
          */
         private void notifyDiscovery(int type, long topVer, ClusterNode node, NavigableSet<ClusterNode> top,
             @Nullable DiscoverySpiCustomMessage data) {
-            DiscoverySpiListener lsnr = adapter.lsnr;
+            DiscoverySpiListener lsnr = spi.lsnr;
 
             if (lsnr != null) {
                 if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0e192ef8/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 184895b..a966363 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
@@ -230,36 +230,36 @@ class ServerImpl extends TcpDiscoveryImpl {
 
         tcpSrvr = new TcpServer();
 
-        adapter.initLocalNode(tcpSrvr.port, true);
+        spi.initLocalNode(tcpSrvr.port, true);
 
-        locNode = adapter.locNode;
+        locNode = spi.locNode;
 
         // Start TCP server thread after local node is initialized.
         tcpSrvr.start();
 
         ring.localNode(locNode);
 
-        if (adapter.ipFinder.isShared())
+        if (spi.ipFinder.isShared())
             registerLocalNodeAddress();
         else {
-            if (F.isEmpty(adapter.ipFinder.getRegisteredAddresses()))
+            if (F.isEmpty(spi.ipFinder.getRegisteredAddresses()))
                 throw new IgniteSpiException("Non-shared IP finder must have IP addresses specified in " +
                     "GridTcpDiscoveryIpFinder.getRegisteredAddresses() configuration property " +
                     "(specify list of IP addresses in configuration).");
 
-            ipFinderHasLocAddr = adapter.ipFinderHasLocalAddress();
+            ipFinderHasLocAddr = spi.ipFinderHasLocalAddress();
         }
 
-        if (adapter.getStatisticsPrintFrequency() > 0 && log.isInfoEnabled()) {
+        if (spi.getStatisticsPrintFrequency() > 0 && log.isInfoEnabled()) {
             statsPrinter = new StatisticsPrinter();
             statsPrinter.start();
         }
 
-        adapter.stats.onJoinStarted();
+        spi.stats.onJoinStarted();
 
         joinTopology();
 
-        adapter.stats.onJoinFinished();
+        spi.stats.onJoinFinished();
 
         hbsSnd = new HeartbeatsSender();
         hbsSnd.start();
@@ -267,12 +267,12 @@ class ServerImpl extends TcpDiscoveryImpl {
         chkStatusSnd = new CheckStatusSender();
         chkStatusSnd.start();
 
-        if (adapter.ipFinder.isShared()) {
+        if (spi.ipFinder.isShared()) {
             ipFinderCleaner = new IpFinderCleaner();
             ipFinderCleaner.start();
         }
 
-        adapter.printStartInfo();
+        spi.printStartInfo();
     }
 
     /**
@@ -283,7 +283,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         // Make sure address registration succeeded.
         while (true) {
             try {
-                adapter.ipFinder.initializeLocalAddresses(locNode.socketAddresses());
+                spi.ipFinder.initializeLocalAddresses(locNode.socketAddresses());
 
                 // Success.
                 break;
@@ -341,9 +341,9 @@ class ServerImpl extends TcpDiscoveryImpl {
             msgWorker.addMessage(new TcpDiscoveryNodeLeftMessage(locNode.id()));
 
             synchronized (mux) {
-                long threshold = U.currentTimeMillis() + adapter.netTimeout;
+                long threshold = U.currentTimeMillis() + spi.netTimeout;
 
-                long timeout = adapter.netTimeout;
+                long timeout = spi.netTimeout;
 
                 while (spiState != LEFT && timeout > 0) {
                     try {
@@ -400,9 +400,9 @@ class ServerImpl extends TcpDiscoveryImpl {
         Collection<TcpDiscoveryNode> rmts = null;
 
         if (!disconnect)
-            adapter.printStopInfo();
+            spi.printStopInfo();
         else {
-            adapter.getSpiContext().deregisterPorts();
+            spi.getSpiContext().deregisterPorts();
 
             rmts = ring.visibleRemoteNodes();
         }
@@ -414,7 +414,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         if (rmts != null && !rmts.isEmpty()) {
             // This is restart/disconnection and remote nodes are not empty.
             // We need to fire FAIL event for each.
-            DiscoverySpiListener lsnr = adapter.lsnr;
+            DiscoverySpiListener lsnr = spi.lsnr;
 
             if (lsnr != null) {
                 Collection<ClusterNode> processed = new HashSet<>();
@@ -438,7 +438,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
         printStatistics();
 
-        adapter.stats.clear();
+        spi.stats.clear();
 
         synchronized (mux) {
             // Clear stored data.
@@ -498,7 +498,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 return false;
         }
 
-        for (InetSocketAddress addr : adapter.getNodeAddresses(node, U.sameMacs(locNode, node))) {
+        for (InetSocketAddress addr : spi.getNodeAddresses(node, U.sameMacs(locNode, node))) {
             try {
                 // ID returned by the node should be the same as ID of the parameter for ping to succeed.
                 IgniteBiTuple<UUID, Boolean> t = pingNode(addr, clientNodeId);
@@ -530,7 +530,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
         UUID locNodeId = getLocalNodeId();
 
-        if (F.contains(adapter.locNodeAddrs, addr)) {
+        if (F.contains(spi.locNodeAddrs, addr)) {
             if (clientNodeId == null)
                 return F.t(getLocalNodeId(), false);
 
@@ -565,18 +565,18 @@ class ServerImpl extends TcpDiscoveryImpl {
             try {
                 Socket sock = null;
 
-                for (int i = 0; i < adapter.reconCnt; i++) {
+                for (int i = 0; i < spi.reconCnt; i++) {
                     try {
                         if (addr.isUnresolved())
                             addr = new InetSocketAddress(InetAddress.getByName(addr.getHostName()), addr.getPort());
 
                         long tstamp = U.currentTimeMillis();
 
-                        sock = adapter.openSocket(addr);
+                        sock = spi.openSocket(addr);
 
-                        adapter.writeToSocket(sock, new TcpDiscoveryPingRequest(locNodeId, clientNodeId));
+                        spi.writeToSocket(sock, new TcpDiscoveryPingRequest(locNodeId, clientNodeId));
 
-                        TcpDiscoveryPingResponse res = adapter.readMessage(sock, null, adapter.netTimeout);
+                        TcpDiscoveryPingResponse res = spi.readMessage(sock, null, spi.netTimeout);
 
                         if (locNodeId.equals(res.creatorNodeId())) {
                             if (log.isDebugEnabled())
@@ -585,7 +585,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             break;
                         }
 
-                        adapter.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+                        spi.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
 
                         IgniteBiTuple<UUID, Boolean> t = F.t(res.creatorNodeId(), res.clientExists());
 
@@ -638,7 +638,7 @@ class ServerImpl extends TcpDiscoveryImpl {
     /** {@inheritDoc} */
     @Override public void sendCustomEvent(DiscoverySpiCustomMessage evt) {
         try {
-            msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, adapter.marsh.marshal(evt)));
+            msgWorker.addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), evt, spi.marsh.marshal(evt)));
         }
         catch (IgniteCheckedException e) {
             throw new IgniteSpiException("Failed to marshal custom event: " + evt, e);
@@ -691,7 +691,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         Map<String, Object> attrs = new HashMap<>(locNode.attributes());
 
                         attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT,
-                            adapter.ignite().configuration().getMarshaller().marshal(subj));
+                            spi.ignite().configuration().getMarshaller().marshal(subj));
                         attrs.remove(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS);
 
                         locNode.setAttributes(attrs);
@@ -704,7 +704,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 locNode.order(1);
                 locNode.internalOrder(1);
 
-                adapter.gridStartTime = U.currentTimeMillis();
+                spi.gridStartTime = U.currentTimeMillis();
 
                 locNode.visible(true);
 
@@ -729,9 +729,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                 log.debug("Join request message has been sent (waiting for coordinator response).");
 
             synchronized (mux) {
-                long threshold = U.currentTimeMillis() + adapter.netTimeout;
+                long threshold = U.currentTimeMillis() + spi.netTimeout;
 
-                long timeout = adapter.netTimeout;
+                long timeout = spi.netTimeout;
 
                 while (spiState == CONNECTING && timeout > 0) {
                     try {
@@ -749,15 +749,15 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if (spiState == CONNECTED)
                     break;
                 else if (spiState == DUPLICATE_ID)
-                    throw adapter.duplicateIdError((TcpDiscoveryDuplicateIdMessage)joinRes.get());
+                    throw spi.duplicateIdError((TcpDiscoveryDuplicateIdMessage)joinRes.get());
                 else if (spiState == AUTH_FAILED)
-                    throw adapter.authenticationFailedError((TcpDiscoveryAuthFailedMessage)joinRes.get());
+                    throw spi.authenticationFailedError((TcpDiscoveryAuthFailedMessage)joinRes.get());
                 else if (spiState == CHECK_FAILED)
-                    throw adapter.checkFailedError((TcpDiscoveryCheckFailedMessage)joinRes.get());
+                    throw spi.checkFailedError((TcpDiscoveryCheckFailedMessage)joinRes.get());
                 else if (spiState == LOOPBACK_PROBLEM) {
                     TcpDiscoveryLoopbackProblemMessage msg = (TcpDiscoveryLoopbackProblemMessage)joinRes.get();
 
-                    boolean locHostLoopback = adapter.locHost.isLoopbackAddress();
+                    boolean locHostLoopback = spi.locHost.isLoopbackAddress();
 
                     String firstNode = locHostLoopback ? "local" : "remote";
 
@@ -774,7 +774,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         "Check remote nodes logs for possible error messages. " +
                         "Note that large topology may require significant time to start. " +
                         "Increase 'TcpDiscoverySpi.networkTimeout' configuration property " +
-                        "if getting this message on the starting nodes [networkTimeout=" + adapter.netTimeout + ']');
+                        "if getting this message on the starting nodes [networkTimeout=" + spi.netTimeout + ']');
             }
         }
 
@@ -796,13 +796,13 @@ class ServerImpl extends TcpDiscoveryImpl {
     @SuppressWarnings({"BusyWait"})
     private boolean sendJoinRequestMessage() throws IgniteSpiException {
         TcpDiscoveryAbstractMessage joinReq = new TcpDiscoveryJoinRequestMessage(locNode,
-            adapter.collectExchangeData(getLocalNodeId()));
+            spi.collectExchangeData(getLocalNodeId()));
 
         // Time when it has been detected, that addresses from IP finder do not respond.
         long noResStart = 0;
 
         while (true) {
-            Collection<InetSocketAddress> addrs = adapter.resolvedAddresses();
+            Collection<InetSocketAddress> addrs = spi.resolvedAddresses();
 
             if (F.isEmpty(addrs))
                 return false;
@@ -810,7 +810,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             boolean retry = false;
             Collection<Exception> errs = new ArrayList<>();
 
-            try (SocketMultiConnector multiConnector = new SocketMultiConnector(adapter, addrs, 2)) {
+            try (SocketMultiConnector multiConnector = new SocketMultiConnector(spi, addrs, 2)) {
                 GridTuple3<InetSocketAddress, Socket, Exception> tuple;
 
                 while ((tuple = multiConnector.next()) != null) {
@@ -897,7 +897,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     throw new IgniteSpiException("Thread has been interrupted.", e);
                 }
             }
-            else if (!adapter.ipFinder.isShared() && !ipFinderHasLocAddr) {
+            else if (!spi.ipFinder.isShared() && !ipFinderHasLocAddr) {
                 IgniteCheckedException e = null;
 
                 if (!errs.isEmpty()) {
@@ -912,10 +912,10 @@ class ServerImpl extends TcpDiscoveryImpl {
                         "(make sure IP finder addresses are correct and firewalls are disabled on all host machines): " +
                         addrs);
 
-                if (adapter.joinTimeout > 0) {
+                if (spi.joinTimeout > 0) {
                     if (noResStart == 0)
                         noResStart = U.currentTimeMillis();
-                    else if (U.currentTimeMillis() - noResStart > adapter.joinTimeout)
+                    else if (U.currentTimeMillis() - noResStart > spi.joinTimeout)
                         throw new IgniteSpiException(
                             "Failed to connect to any address from IP finder within join timeout " +
                                 "(make sure IP finder addresses are correct, and operating system firewalls are disabled " +
@@ -952,7 +952,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
         Collection<Throwable> errs = null;
 
-        long ackTimeout0 = adapter.ackTimeout;
+        long ackTimeout0 = spi.ackTimeout;
 
         int connectAttempts = 1;
 
@@ -960,7 +960,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
         UUID locNodeId = getLocalNodeId();
 
-        for (int i = 0; i < adapter.reconCnt; i++) {
+        for (int i = 0; i < spi.reconCnt; i++) {
             // Need to set to false on each new iteration,
             // since remote node may leave in the middle of the first iteration.
             joinReqSent = false;
@@ -971,14 +971,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                 long tstamp = U.currentTimeMillis();
 
                 if (sock == null)
-                    sock = adapter.openSocket(addr);
+                    sock = spi.openSocket(addr);
 
                 openSock = true;
 
                 // Handshake.
-                adapter.writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId));
+                spi.writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId));
 
-                TcpDiscoveryHandshakeResponse res = adapter.readMessage(sock, null, ackTimeout0);
+                TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, ackTimeout0);
 
                 if (locNodeId.equals(res.creatorNodeId())) {
                     if (log.isDebugEnabled())
@@ -987,14 +987,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                     break;
                 }
 
-                adapter.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+                spi.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
 
                 // Send message.
                 tstamp = U.currentTimeMillis();
 
-                adapter.writeToSocket(sock, msg);
+                spi.writeToSocket(sock, msg);
 
-                adapter.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
+                spi.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
 
                 if (debugMode)
                     debugLog("Message has been sent directly to address [msg=" + msg + ", addr=" + addr +
@@ -1009,7 +1009,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 // E.g. due to class not found issue.
                 joinReqSent = msg instanceof TcpDiscoveryJoinRequestMessage;
 
-                return adapter.readReceipt(sock, ackTimeout0);
+                return spi.readReceipt(sock, ackTimeout0);
             }
             catch (ClassCastException e) {
                 // This issue is rarely reproducible on AmazonEC2, but never
@@ -1087,7 +1087,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             Map<String, Object> attrs = new HashMap<>(node.getAttributes());
 
             attrs.put(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS,
-                adapter.marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS)));
+                spi.marsh.marshal(attrs.get(IgniteNodeAttributes.ATTR_SECURITY_CREDENTIALS)));
 
             node.setAttributes(attrs);
         }
@@ -1110,7 +1110,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             if (credBytes == null)
                 return null;
 
-            return adapter.marsh.unmarshal(credBytes, null);
+            return spi.marsh.unmarshal(credBytes, null);
         }
         catch (IgniteCheckedException e) {
             throw new IgniteSpiException("Failed to unmarshal node security credentials: " + node.id(), e);
@@ -1123,10 +1123,10 @@ class ServerImpl extends TcpDiscoveryImpl {
      * maximum acknowledgement timeout, {@code false} otherwise.
      */
     private boolean checkAckTimeout(long ackTimeout) {
-        if (ackTimeout > adapter.maxAckTimeout) {
+        if (ackTimeout > spi.maxAckTimeout) {
             LT.warn(log, null, "Acknowledgement timeout is greater than maximum acknowledgement timeout " +
                 "(consider increasing 'maxAckTimeout' configuration property) " +
-                "[ackTimeout=" + ackTimeout + ", maxAckTimeout=" + adapter.maxAckTimeout + ']');
+                "[ackTimeout=" + ackTimeout + ", maxAckTimeout=" + spi.maxAckTimeout + ']');
 
             return false;
         }
@@ -1145,7 +1145,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         assert type > 0;
         assert node != null;
 
-        DiscoverySpiListener lsnr = adapter.lsnr;
+        DiscoverySpiListener lsnr = spi.lsnr;
 
         TcpDiscoverySpiState spiState = spiStateCopy();
 
@@ -1179,7 +1179,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             topHist.put(topVer, top);
 
-            while (topHist.size() > adapter.topHistSize)
+            while (topHist.size() > spi.topHistSize)
                 topHist.remove(topHist.firstKey());
 
             if (log.isDebugEnabled())
@@ -1200,7 +1200,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             boolean crd = spiState == CONNECTED && locNode.equals(resolveCoordinator());
 
             if (crd)
-                adapter.stats.onBecomingCoordinator();
+                spi.stats.onBecomingCoordinator();
 
             return crd;
         }
@@ -1254,7 +1254,7 @@ class ServerImpl extends TcpDiscoveryImpl {
      * Prints SPI statistics.
      */
     private void printStatistics() {
-        if (log.isInfoEnabled() && adapter.statsPrintFreq > 0) {
+        if (log.isInfoEnabled() && spi.statsPrintFreq > 0) {
             int failedNodesSize;
             int leavingNodesSize;
 
@@ -1267,7 +1267,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             TcpDiscoveryNode coord = resolveCoordinator();
 
-            log.info("Discovery SPI statistics [statistics=" + adapter.stats + ", spiState=" + spiStateCopy() +
+            log.info("Discovery SPI statistics [statistics=" + spi.stats + ", spiState=" + spiStateCopy() +
                 ", coord=" + coord +
                 ", topSize=" + ring.allNodes().size() +
                 ", leavingNodesSize=" + leavingNodesSize + ", failedNodesSize=" + failedNodesSize +
@@ -1439,7 +1439,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             b.append("    Message worker: ").append(threadStatus(msgWorker)).append(U.nl());
             b.append("    Check status sender: ").append(threadStatus(chkStatusSnd)).append(U.nl());
             b.append("    HB sender: ").append(threadStatus(hbsSnd)).append(U.nl());
-            b.append("    Socket timeout worker: ").append(threadStatus(adapter.sockTimeoutWorker)).append(U.nl());
+            b.append("    Socket timeout worker: ").append(threadStatus(spi.sockTimeoutWorker)).append(U.nl());
             b.append("    IP finder cleaner: ").append(threadStatus(ipFinderCleaner)).append(U.nl());
             b.append("    Stats printer: ").append(threadStatus(statsPrinter)).append(U.nl());
 
@@ -1473,7 +1473,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             b.append(U.nl());
 
-            b.append("Stats: ").append(adapter.stats).append(U.nl());
+            b.append("Stats: ").append(spi.stats).append(U.nl());
 
             U.quietAndInfo(log, b.toString());
         }
@@ -1548,9 +1548,9 @@ class ServerImpl extends TcpDiscoveryImpl {
          * Constructor.
          */
         private HeartbeatsSender() {
-            super(adapter.ignite().name(), "tcp-disco-hb-sender", log);
+            super(spi.ignite().name(), "tcp-disco-hb-sender", log);
 
-            setPriority(adapter.threadPri);
+            setPriority(spi.threadPri);
         }
 
         /** {@inheritDoc} */
@@ -1576,7 +1576,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 msgWorker.addMessage(msg);
 
-                Thread.sleep(adapter.hbFreq);
+                Thread.sleep(spi.hbFreq);
             }
         }
     }
@@ -1592,9 +1592,9 @@ class ServerImpl extends TcpDiscoveryImpl {
          * Constructor.
          */
         private CheckStatusSender() {
-            super(adapter.ignite().name(), "tcp-disco-status-check-sender", log);
+            super(spi.ignite().name(), "tcp-disco-status-check-sender", log);
 
-            setPriority(adapter.threadPri);
+            setPriority(spi.threadPri);
         }
 
         /** {@inheritDoc} */
@@ -1604,7 +1604,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 log.debug("Status check sender has been started.");
 
             // Only 1 heartbeat missing is acceptable. Add 50 ms to avoid false alarm.
-            long checkTimeout = (long)adapter.maxMissedHbs * adapter.hbFreq + 50;
+            long checkTimeout = (long)spi.maxMissedHbs * spi.hbFreq + 50;
 
             long lastSent = 0;
 
@@ -1656,9 +1656,9 @@ class ServerImpl extends TcpDiscoveryImpl {
          * Constructor.
          */
         private IpFinderCleaner() {
-            super(adapter.ignite().name(), "tcp-disco-ip-finder-cleaner", log);
+            super(spi.ignite().name(), "tcp-disco-ip-finder-cleaner", log);
 
-            setPriority(adapter.threadPri);
+            setPriority(spi.threadPri);
         }
 
         /** {@inheritDoc} */
@@ -1668,7 +1668,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 log.debug("IP finder cleaner has been started.");
 
             while (!isInterrupted()) {
-                Thread.sleep(adapter.ipFinderCleanFreq);
+                Thread.sleep(spi.ipFinderCleanFreq);
 
                 if (!isLocalNodeCoordinator())
                     continue;
@@ -1680,7 +1680,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     return;
                 }
 
-                if (adapter.ipFinder.isShared())
+                if (spi.ipFinder.isShared())
                     cleanIpFinder();
             }
         }
@@ -1689,7 +1689,7 @@ class ServerImpl extends TcpDiscoveryImpl {
          * Cleans IP finder.
          */
         private void cleanIpFinder() {
-            assert adapter.ipFinder.isShared();
+            assert spi.ipFinder.isShared();
 
             try {
                 // Addresses that belongs to nodes in topology.
@@ -1698,7 +1698,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         ring.allNodes(),
                         new C1<TcpDiscoveryNode, Collection<InetSocketAddress>>() {
                             @Override public Collection<InetSocketAddress> apply(TcpDiscoveryNode node) {
-                                return !node.isClient() ? adapter.getNodeAddresses(node) :
+                                return !node.isClient() ? spi.getNodeAddresses(node) :
                                     Collections.<InetSocketAddress>emptyList();
                             }
                         }
@@ -1706,7 +1706,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 );
 
                 // Addresses registered in IP finder.
-                Collection<InetSocketAddress> regAddrs = adapter.registeredAddresses();
+                Collection<InetSocketAddress> regAddrs = spi.registeredAddresses();
 
                 // Remove all addresses that belong to alive nodes, leave dead-node addresses.
                 Collection<InetSocketAddress> rmvAddrs = F.view(
@@ -1742,7 +1742,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 // Unregister dead-nodes addresses.
                 if (!rmvAddrs.isEmpty()) {
-                    adapter.ipFinder.unregisterAddresses(rmvAddrs);
+                    spi.ipFinder.unregisterAddresses(rmvAddrs);
 
                     if (log.isDebugEnabled())
                         log.debug("Unregistered addresses from IP finder: " + rmvAddrs);
@@ -1756,7 +1756,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 // Re-register missing addresses.
                 if (!missingAddrs.isEmpty()) {
-                    adapter.ipFinder.registerAddresses(missingAddrs);
+                    spi.ipFinder.registerAddresses(missingAddrs);
 
                     if (log.isDebugEnabled())
                         log.debug("Registered missing addresses in IP finder: " + missingAddrs);
@@ -1897,7 +1897,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             if (debugMode)
                 debugLog("Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
 
-            adapter.stats.onMessageProcessingStarted(msg);
+            spi.stats.onMessageProcessingStarted(msg);
 
             if (msg instanceof TcpDiscoveryJoinRequestMessage)
                 processJoinRequestMessage((TcpDiscoveryJoinRequestMessage)msg);
@@ -1938,7 +1938,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             else
                 assert false : "Unknown message type: " + msg.getClass().getSimpleName();
 
-            adapter.stats.onMessageProcessingFinished(msg);
+            spi.stats.onMessageProcessingFinished(msg);
         }
 
         /**
@@ -1952,7 +1952,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             assert ring.hasRemoteNodes();
 
-            for (IgniteInClosure<TcpDiscoveryAbstractMessage> msgLsnr : adapter.sendMsgLsnrs)
+            for (IgniteInClosure<TcpDiscoveryAbstractMessage> msgLsnr : spi.sendMsgLsnrs)
                 msgLsnr.apply(msg);
 
             if (redirectToClients(msg)) {
@@ -1964,9 +1964,9 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     try {
                         if (marshalledMsg == null)
-                            marshalledMsg = adapter.marsh.marshal(msg);
+                            marshalledMsg = spi.marsh.marshal(msg);
 
-                        msgClone = adapter.marsh.unmarshal(marshalledMsg, null);
+                        msgClone = spi.marsh.unmarshal(marshalledMsg, null);
                     }
                     catch (IgniteCheckedException e) {
                         U.error(log, "Failed to marshal message: " + msg, e);
@@ -2043,8 +2043,8 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 List<InetSocketAddress> locNodeAddrs = U.arrayList(locNode.socketAddresses());
 
-                addr: for (InetSocketAddress addr : adapter.getNodeAddresses(next, sameHost)) {
-                    long ackTimeout0 = adapter.ackTimeout;
+                addr: for (InetSocketAddress addr : spi.getNodeAddresses(next, sameHost)) {
+                    long ackTimeout0 = spi.ackTimeout;
 
                     if (locNodeAddrs.contains(addr)){
                         if (log.isDebugEnabled())
@@ -2054,7 +2054,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         continue;
                     }
 
-                    for (int i = 0; i < adapter.reconCnt; i++) {
+                    for (int i = 0; i < spi.reconCnt; i++) {
                         if (sock == null) {
                             nextNodeExists = false;
 
@@ -2066,14 +2066,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                             try {
                                 long tstamp = U.currentTimeMillis();
 
-                                sock = adapter.openSocket(addr);
+                                sock = spi.openSocket(addr);
 
                                 openSock = true;
 
                                 // Handshake.
                                 writeToSocket(sock, new TcpDiscoveryHandshakeRequest(locNodeId));
 
-                                TcpDiscoveryHandshakeResponse res = adapter.readMessage(sock, null, ackTimeout0);
+                                TcpDiscoveryHandshakeResponse res = spi.readMessage(sock, null, ackTimeout0);
 
                                 if (locNodeId.equals(res.creatorNodeId())) {
                                     if (log.isDebugEnabled())
@@ -2086,7 +2086,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     break;
                                 }
 
-                                adapter.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
+                                spi.stats.onClientSocketInitialized(U.currentTimeMillis() - tstamp);
 
                                 UUID nextId = res.creatorNodeId();
 
@@ -2214,9 +2214,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                                         clearNodeAddedMessage(pendingMsg);
                                     }
 
-                                    adapter.stats.onMessageSent(pendingMsg, U.currentTimeMillis() - tstamp);
+                                    spi.stats.onMessageSent(pendingMsg, U.currentTimeMillis() - tstamp);
 
-                                    int res = adapter.readReceipt(sock, ackTimeout0);
+                                    int res = spi.readReceipt(sock, ackTimeout0);
 
                                     if (log.isDebugEnabled())
                                         log.debug("Pending message has been sent to next node [msg=" + msg.id() +
@@ -2237,9 +2237,9 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                                 writeToSocket(sock, msg);
 
-                                adapter.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
+                                spi.stats.onMessageSent(msg, U.currentTimeMillis() - tstamp);
 
-                                int res = adapter.readReceipt(sock, ackTimeout0);
+                                int res = spi.readReceipt(sock, ackTimeout0);
 
                                 if (log.isDebugEnabled())
                                     log.debug("Message has been sent to next node [msg=" + msg +
@@ -2382,10 +2382,10 @@ class ServerImpl extends TcpDiscoveryImpl {
         private void registerPendingMessage(TcpDiscoveryAbstractMessage msg) {
             assert msg != null;
 
-            if (adapter.ensured(msg)) {
+            if (spi.ensured(msg)) {
                 pendingMsgs.add(msg);
 
-                adapter.stats.onPendingMessageRegistered();
+                spi.stats.onPendingMessageRegistered();
 
                 if (log.isDebugEnabled())
                     log.debug("Pending message has been registered: " + msg.id());
@@ -2411,7 +2411,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 // This check is performed by the node joining node is connected to, but not by coordinator
                 // because loopback problem message is sent directly to the joining node which may be unavailable
                 // if coordinator resides on another host.
-                if (adapter.locHost.isLoopbackAddress() != rmtHostLoopback) {
+                if (spi.locHost.isLoopbackAddress() != rmtHostLoopback) {
                     String firstNode = rmtHostLoopback ? "remote" : "local";
 
                     String secondNode = rmtHostLoopback ? "local" : "remote";
@@ -2510,7 +2510,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                             try {
                                 trySendMessageDirectly(node, new TcpDiscoveryAuthFailedMessage(locNodeId,
-                                    adapter.locHost));
+                                    spi.locHost));
                             }
                             catch (IgniteSpiException e) {
                                 if (log.isDebugEnabled())
@@ -2541,7 +2541,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                                 try {
                                     trySendMessageDirectly(node, new TcpDiscoveryAuthFailedMessage(locNodeId,
-                                        adapter.locHost));
+                                        spi.locHost));
                                 }
                                 catch (IgniteSpiException e) {
                                     if (log.isDebugEnabled())
@@ -2557,7 +2557,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             Map<String, Object> attrs = new HashMap<>(node.getAttributes());
 
                             attrs.put(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT,
-                                adapter.ignite().configuration().getMarshaller().marshal(subj));
+                                spi.ignite().configuration().getMarshaller().marshal(subj));
 
                             node.setAttributes(attrs);
                         }
@@ -2578,7 +2578,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     }
                 }
 
-                IgniteNodeValidationResult err = adapter.getSpiContext().validateNode(node);
+                IgniteNodeValidationResult err = spi.getSpiContext().validateNode(node);
 
                 if (err != null) {
                     boolean ping = node.id().equals(err.nodeId()) ? pingNode(node) : pingNode(err.nodeId());
@@ -2666,7 +2666,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     log.debug("Internal order has been assigned to node: " + node);
 
                 TcpDiscoveryNodeAddedMessage nodeAddedMsg = new TcpDiscoveryNodeAddedMessage(locNodeId,
-                    node, msg.discoveryData(), adapter.gridStartTime);
+                    node, msg.discoveryData(), spi.gridStartTime);
 
                 nodeAddedMsg.client(msg.client());
 
@@ -2711,7 +2711,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             IgniteSpiException ex = null;
 
-            for (InetSocketAddress addr : adapter.getNodeAddresses(node, U.sameMacs(locNode, node))) {
+            for (InetSocketAddress addr : spi.getNodeAddresses(node, U.sameMacs(locNode, node))) {
                 try {
                     sendMessageDirectly(msg, addr, null);
 
@@ -2760,7 +2760,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 assert node.isClient();
 
                 node.clientRouterNodeId(msg.routerNodeId());
-                node.aliveCheck(adapter.maxMissedClientHbs);
+                node.aliveCheck(spi.maxMissedClientHbs);
 
                 if (isLocalNodeCoordinator()) {
                     Collection<TcpDiscoveryAbstractMessage> pending =
@@ -2826,7 +2826,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             if (isLocalNodeCoordinator()) {
                 if (msg.verified()) {
-                    adapter.stats.onRingMessageReceived(msg);
+                    spi.stats.onRingMessageReceived(msg);
 
                     TcpDiscoveryNodeAddFinishedMessage addFinishMsg = new TcpDiscoveryNodeAddFinishedMessage(locNodeId,
                         node.id());
@@ -2882,7 +2882,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         else {
                             SecurityContext subj = nodeAuth.authenticateNode(node, cred);
 
-                            SecurityContext coordSubj = adapter.ignite().configuration().getMarshaller().unmarshal(
+                            SecurityContext coordSubj = spi.ignite().configuration().getMarshaller().unmarshal(
                                 node.<byte[]>attribute(IgniteNodeAttributes.ATTR_SECURITY_SUBJECT),
                                 U.gridClassLoader());
 
@@ -2911,7 +2911,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         if (authFailed) {
                             try {
                                 trySendMessageDirectly(node, new TcpDiscoveryAuthFailedMessage(locNodeId,
-                                    adapter.locHost));
+                                    spi.locHost));
                             }
                             catch (IgniteSpiException e) {
                                 if (log.isDebugEnabled())
@@ -2929,7 +2929,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
 
                 if (msg.client())
-                    node.aliveCheck(adapter.maxMissedClientHbs);
+                    node.aliveCheck(spi.maxMissedClientHbs);
 
                 boolean topChanged = ring.add(node);
 
@@ -2939,9 +2939,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                     Map<Integer, byte[]> data = msg.newNodeDiscoveryData();
 
                     if (data != null)
-                        adapter.onExchange(node.id(), node.id(), data, U.gridClassLoader());
+                        spi.onExchange(node.id(), node.id(), data, U.gridClassLoader());
 
-                    msg.addDiscoveryData(locNodeId, adapter.collectExchangeData(node.id()));
+                    msg.addDiscoveryData(locNodeId, spi.collectExchangeData(node.id()));
                 }
 
                 if (log.isDebugEnabled())
@@ -2959,7 +2959,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         Collection<TcpDiscoveryNode> top = msg.topology();
 
                         if (top != null && !top.isEmpty()) {
-                            adapter.gridStartTime = msg.gridStartTime();
+                            spi.gridStartTime = msg.gridStartTime();
 
                             for (TcpDiscoveryNode n : top) {
                                 // Make all preceding nodes and local node visible.
@@ -3011,7 +3011,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 // Notify outside of synchronized block.
                 if (dataMap != null) {
                     for (Map.Entry<UUID, Map<Integer, byte[]>> entry : dataMap.entrySet())
-                        adapter.onExchange(node.id(), entry.getKey(), entry.getValue(), U.gridClassLoader());
+                        spi.onExchange(node.id(), entry.getKey(), entry.getValue(), U.gridClassLoader());
                 }
             }
 
@@ -3050,7 +3050,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             if (locNodeCoord) {
                 if (msg.verified()) {
-                    adapter.stats.onRingMessageReceived(msg);
+                    spi.stats.onRingMessageReceived(msg);
 
                     addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id()));
 
@@ -3088,15 +3088,15 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
 
             if (msg.verified() && !locNodeId.equals(nodeId) && spiStateCopy() == CONNECTED && fireEvt) {
-                adapter.stats.onNodeJoined();
+                spi.stats.onNodeJoined();
 
                 // Make sure that node with greater order will never get EVT_NODE_JOINED
                 // on node with less order.
                 assert node.internalOrder() > locNode.internalOrder() : "Invalid order [node=" + node +
                     ", locNode=" + locNode + ", msg=" + msg + ", ring=" + ring + ']';
 
-                if (adapter.locNodeVer.equals(node.version()))
-                    node.version(adapter.locNodeVer);
+                if (spi.locNodeVer.equals(node.version()))
+                    node.version(spi.locNodeVer);
 
                 if (!locNodeCoord) {
                     boolean b = ring.topologyVersion(topVer);
@@ -3113,8 +3113,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                 notifyDiscovery(EVT_NODE_JOINED, topVer, node);
 
                 try {
-                    if (adapter.ipFinder.isShared() && locNodeCoord)
-                        adapter.ipFinder.registerAddresses(node.socketAddresses());
+                    if (spi.ipFinder.isShared() && locNodeCoord)
+                        spi.ipFinder.registerAddresses(node.socketAddresses());
                 }
                 catch (IgniteSpiException e) {
                     if (log.isDebugEnabled())
@@ -3172,9 +3172,9 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
 
                 if (msg.verified() || !ring.hasRemoteNodes() || msg.senderNodeId() != null) {
-                    if (adapter.ipFinder.isShared() && !ring.hasRemoteNodes()) {
+                    if (spi.ipFinder.isShared() && !ring.hasRemoteNodes()) {
                         try {
-                            adapter.ipFinder.unregisterAddresses(locNode.socketAddresses());
+                            spi.ipFinder.unregisterAddresses(locNode.socketAddresses());
                         }
                         catch (IgniteSpiException e) {
                             U.error(log, "Failed to unregister local node address from IP finder.", e);
@@ -3222,7 +3222,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             if (locNodeCoord) {
                 if (msg.verified()) {
-                    adapter.stats.onRingMessageReceived(msg);
+                    spi.stats.onRingMessageReceived(msg);
 
                     addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id()));
 
@@ -3297,7 +3297,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     }
                 }
 
-                adapter.stats.onNodeLeft();
+                spi.stats.onNodeLeft();
 
                 notifyDiscovery(EVT_NODE_LEFT, topVer, leftNode);
 
@@ -3392,7 +3392,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             if (locNodeCoord) {
                 if (msg.verified()) {
-                    adapter.stats.onRingMessageReceived(msg);
+                    spi.stats.onRingMessageReceived(msg);
 
                     addMessage(new TcpDiscoveryDiscardMessage(locNodeId, msg.id()));
 
@@ -3442,7 +3442,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 notifyDiscovery(EVT_NODE_FAILED, topVer, node);
 
-                adapter.stats.onNodeFailed();
+                spi.stats.onNodeFailed();
             }
 
             if (ring.hasRemoteNodes())
@@ -3535,7 +3535,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
 
                 if (locNodeId.equals(msg.creatorNodeId()) && msg.senderNodeId() == null &&
-                    U.currentTimeMillis() - locNode.lastUpdateTime() < adapter.hbFreq) {
+                    U.currentTimeMillis() - locNode.lastUpdateTime() < spi.hbFreq) {
                     if (log.isDebugEnabled())
                         log.debug("Status check message discarded (local node receives updates).");
 
@@ -3643,8 +3643,8 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if ((locNodeId.equals(msg.creatorNodeId()) && msg.senderNodeId() == null ||
                     !hasMetrics(msg, locNodeId)) && spiStateCopy() == CONNECTED) {
                     // Message is on its first ring or just created on coordinator.
-                    msg.setMetrics(locNodeId, adapter.metricsProvider.metrics());
-                    msg.setCacheMetrics(locNodeId, adapter.metricsProvider.cacheMetrics());
+                    msg.setMetrics(locNodeId, spi.metricsProvider.metrics());
+                    msg.setCacheMetrics(locNodeId, spi.metricsProvider.cacheMetrics());
 
                     for (Map.Entry<UUID, ClientMessageWorker> e : clientMsgWorkers.entrySet()) {
                         UUID nodeId = e.getKey();
@@ -3665,7 +3665,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     for (TcpDiscoveryNode clientNode : ring.clientNodes()) {
                         if (clientNode.visible()) {
                             if (clientNodeIds.contains(clientNode.id()))
-                                clientNode.aliveCheck(adapter.maxMissedClientHbs);
+                                clientNode.aliveCheck(spi.maxMissedClientHbs);
                             else {
                                 int aliveCheck = clientNode.decrementAliveCheck();
 
@@ -3815,12 +3815,12 @@ class ServerImpl extends TcpDiscoveryImpl {
                 if (sndNext && ring.hasRemoteNodes())
                     sendMessageAcrossRing(msg);
                 else {
-                    adapter.stats.onRingMessageReceived(msg);
+                    spi.stats.onRingMessageReceived(msg);
 
                     DiscoverySpiCustomMessage msgObj = null;
 
                     try {
-                        msgObj = msg.message(adapter.marsh);
+                        msgObj = msg.message(spi.marsh);
                     }
                     catch (Throwable e) {
                         U.error(log, "Failed to unmarshal discovery custom message.", e);
@@ -3832,7 +3832,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         if (nextMsg != null) {
                             try {
                                 addMessage(new TcpDiscoveryCustomEventMessage(getLocalNodeId(), nextMsg,
-                                    adapter.marsh.marshal(nextMsg)));
+                                    spi.marsh.marshal(nextMsg)));
                             }
                             catch (IgniteCheckedException e) {
                                 U.error(log, "Failed to marshal discovery custom message.", e);
@@ -3856,7 +3856,7 @@ class ServerImpl extends TcpDiscoveryImpl {
          * @param msg Custom message.
          */
         private void notifyDiscoveryListener(TcpDiscoveryCustomEventMessage msg) {
-            DiscoverySpiListener lsnr = adapter.lsnr;
+            DiscoverySpiListener lsnr = spi.lsnr;
 
             TcpDiscoverySpiState spiState = spiStateCopy();
 
@@ -3873,7 +3873,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 if (node != null) {
                     try {
-                        DiscoverySpiCustomMessage msgObj = msg.message(adapter.marsh);
+                        DiscoverySpiCustomMessage msgObj = msg.message(spi.marsh);
 
                         lsnr.onDiscovery(DiscoveryCustomEvent.EVT_DISCOVERY_CUSTOM_EVT,
                             msg.topologyVersion(),
@@ -3883,7 +3883,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             msgObj);
 
                         if (msgObj.isMutable())
-                            msg.message(msgObj, adapter.marsh.marshal(msgObj));
+                            msg.message(msgObj, spi.marsh.marshal(msgObj));
                     }
                     catch (Throwable e) {
                         U.error(log, "Failed to unmarshal discovery custom message.", e);
@@ -3912,35 +3912,35 @@ class ServerImpl extends TcpDiscoveryImpl {
          * @throws IgniteSpiException In case of error.
          */
         TcpServer() throws IgniteSpiException {
-            super(adapter.ignite().name(), "tcp-disco-srvr", log);
+            super(spi.ignite().name(), "tcp-disco-srvr", log);
 
-            setPriority(adapter.threadPri);
+            setPriority(spi.threadPri);
 
-            for (port = adapter.locPort; port < adapter.locPort + adapter.locPortRange; port++) {
+            for (port = spi.locPort; port < spi.locPort + spi.locPortRange; port++) {
                 try {
-                    srvrSock = new ServerSocket(port, 0, adapter.locHost);
+                    srvrSock = new ServerSocket(port, 0, spi.locHost);
 
                     break;
                 }
                 catch (IOException e) {
-                    if (port < adapter.locPort + adapter.locPortRange - 1) {
+                    if (port < spi.locPort + spi.locPortRange - 1) {
                         if (log.isDebugEnabled())
                             log.debug("Failed to bind to local port (will try next port within range) " +
-                                "[port=" + port + ", localHost=" + adapter.locHost + ']');
+                                "[port=" + port + ", localHost=" + spi.locHost + ']');
 
                         onException("Failed to bind to local port. " +
-                            "[port=" + port + ", localHost=" + adapter.locHost + ']', e);
+                            "[port=" + port + ", localHost=" + spi.locHost + ']', e);
                     }
                     else {
                         throw new IgniteSpiException("Failed to bind TCP server socket (possibly all ports in range " +
-                            "are in use) [firstPort=" + adapter.locPort + ", lastPort=" + (adapter.locPort + adapter.locPortRange - 1) +
-                            ", addr=" + adapter.locHost + ']', e);
+                            "are in use) [firstPort=" + spi.locPort + ", lastPort=" + (spi.locPort + spi.locPortRange - 1) +
+                            ", addr=" + spi.locHost + ']', e);
                     }
                 }
             }
 
             if (log.isInfoEnabled())
-                log.info("Successfully bound to TCP port [port=" + port + ", localHost=" + adapter.locHost + ']');
+                log.info("Successfully bound to TCP port [port=" + port + ", localHost=" + spi.locHost + ']');
         }
 
         /** {@inheritDoc} */
@@ -3962,7 +3962,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         reader.start();
                     }
 
-                    adapter.stats.onServerSocketInitialized(U.currentTimeMillis() - tstamp);
+                    spi.stats.onServerSocketInitialized(U.currentTimeMillis() - tstamp);
                 }
             }
             catch (IOException e) {
@@ -4007,13 +4007,13 @@ class ServerImpl extends TcpDiscoveryImpl {
          * @param sock Socket to read data from.
          */
         SocketReader(Socket sock) {
-            super(adapter.ignite().name(), "tcp-disco-sock-reader", log);
+            super(spi.ignite().name(), "tcp-disco-sock-reader", log);
 
             this.sock = sock;
 
-            setPriority(adapter.threadPri);
+            setPriority(spi.threadPri);
 
-            adapter.stats.onSocketReaderCreated();
+            spi.stats.onSocketReaderCreated();
         }
 
         /** {@inheritDoc} */
@@ -4032,9 +4032,9 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                     int timeout = sock.getSoTimeout();
 
-                    sock.setSoTimeout((int)adapter.netTimeout);
+                    sock.setSoTimeout((int)spi.netTimeout);
 
-                    for (IgniteInClosure<Socket> connLsnr : adapter.incomeConnLsnrs)
+                    for (IgniteInClosure<Socket> connLsnr : spi.incomeConnLsnrs)
                         connLsnr.apply(sock);
 
                     in = new BufferedInputStream(sock.getInputStream());
@@ -4077,11 +4077,11 @@ class ServerImpl extends TcpDiscoveryImpl {
                     // Restore timeout.
                     sock.setSoTimeout(timeout);
 
-                    TcpDiscoveryAbstractMessage msg = adapter.readMessage(sock, in, adapter.netTimeout);
+                    TcpDiscoveryAbstractMessage msg = spi.readMessage(sock, in, spi.netTimeout);
 
                     // Ping.
                     if (msg instanceof TcpDiscoveryPingRequest) {
-                        if (!adapter.isNodeStopping0()) {
+                        if (!spi.isNodeStopping0()) {
                             TcpDiscoveryPingRequest req = (TcpDiscoveryPingRequest)msg;
 
                             TcpDiscoveryPingResponse res = new TcpDiscoveryPingResponse(locNodeId);
@@ -4093,7 +4093,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     res.clientExists(clientWorker.ping());
                             }
 
-                            adapter.writeToSocket(sock, res);
+                            spi.writeToSocket(sock, res);
                         }
                         else if (log.isDebugEnabled())
                             log.debug("Ignore ping request, node is stopping.");
@@ -4111,7 +4111,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     TcpDiscoveryHandshakeResponse res =
                         new TcpDiscoveryHandshakeResponse(locNodeId, locNode.internalOrder());
 
-                    adapter.writeToSocket(sock, res);
+                    spi.writeToSocket(sock, res);
 
                     // It can happen if a remote node is stopped and it has a loopback address in the list of addresses,
                     // the local node sends a handshake request message on the loopback address, so we get here.
@@ -4169,7 +4169,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     if (e.hasCause(SocketTimeoutException.class))
                         LT.warn(log, null, "Socket operation timed out on handshake " +
                             "(consider increasing 'networkTimeout' configuration property) " +
-                            "[netTimeout=" + adapter.netTimeout + ']');
+                            "[netTimeout=" + spi.netTimeout + ']');
 
                     else if (e.hasCause(ClassNotFoundException.class))
                         LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
@@ -4187,14 +4187,14 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                 while (!isInterrupted()) {
                     try {
-                        TcpDiscoveryAbstractMessage msg = adapter.marsh.unmarshal(in, U.gridClassLoader());
+                        TcpDiscoveryAbstractMessage msg = spi.marsh.unmarshal(in, U.gridClassLoader());
 
                         msg.senderNodeId(nodeId);
 
                         if (log.isDebugEnabled())
                             log.debug("Message has been received: " + msg);
 
-                        adapter.stats.onMessageReceived(msg);
+                        spi.stats.onMessageReceived(msg);
 
                         if (debugMode && recordable(msg))
                             debugLog("Message has been received: " + msg);
@@ -4217,14 +4217,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                                 TcpDiscoverySpiState state = spiStateCopy();
 
                                 if (state == CONNECTED) {
-                                    adapter.writeToSocket(sock, RES_OK);
+                                    spi.writeToSocket(sock, RES_OK);
 
                                     msgWorker.addMessage(msg);
 
                                     continue;
                                 }
                                 else {
-                                    adapter.writeToSocket(sock, RES_CONTINUE_JOIN);
+                                    spi.writeToSocket(sock, RES_CONTINUE_JOIN);
 
                                     break;
                                 }
@@ -4232,7 +4232,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         }
                         else if (msg instanceof TcpDiscoveryDuplicateIdMessage) {
                             // Send receipt back.
-                            adapter.writeToSocket(sock, RES_OK);
+                            spi.writeToSocket(sock, RES_OK);
 
                             boolean ignored = false;
 
@@ -4261,7 +4261,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         }
                         else if (msg instanceof TcpDiscoveryAuthFailedMessage) {
                             // Send receipt back.
-                            adapter.writeToSocket(sock, RES_OK);
+                            spi.writeToSocket(sock, RES_OK);
 
                             boolean ignored = false;
 
@@ -4290,7 +4290,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         }
                         else if (msg instanceof TcpDiscoveryCheckFailedMessage) {
                             // Send receipt back.
-                            adapter.writeToSocket(sock, RES_OK);
+                            spi.writeToSocket(sock, RES_OK);
 
                             boolean ignored = false;
 
@@ -4319,7 +4319,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         }
                         else if (msg instanceof TcpDiscoveryLoopbackProblemMessage) {
                             // Send receipt back.
-                            adapter.writeToSocket(sock, RES_OK);
+                            spi.writeToSocket(sock, RES_OK);
 
                             boolean ignored = false;
 
@@ -4361,7 +4361,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                         // Send receipt back.
                         if (clientMsgWrk == null)
-                            adapter.writeToSocket(sock, RES_OK);
+                            spi.writeToSocket(sock, RES_OK);
                     }
                     catch (IgniteCheckedException e) {
                         if (log.isDebugEnabled())
@@ -4461,7 +4461,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             TcpDiscoverySpiState state = spiStateCopy();
 
             if (state == CONNECTED) {
-                adapter.writeToSocket(sock, RES_OK);
+                spi.writeToSocket(sock, RES_OK);
 
                 if (log.isDebugEnabled())
                     log.debug("Responded to join request message [msg=" + msg + ", res=" + RES_OK + ']');
@@ -4473,7 +4473,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 return true;
             }
             else {
-                adapter.stats.onMessageProcessingStarted(msg);
+                spi.stats.onMessageProcessingStarted(msg);
 
                 Integer res;
 
@@ -4492,14 +4492,14 @@ class ServerImpl extends TcpDiscoveryImpl {
                     // Local node is stopping. Remote node should try next one.
                     res = RES_CONTINUE_JOIN;
 
-                adapter.writeToSocket(sock, res);
+                spi.writeToSocket(sock, res);
 
                 if (log.isDebugEnabled())
                     log.debug("Responded to join request message [msg=" + msg + ", res=" + res + ']');
 
                 fromAddrs.addAll(msg.node().socketAddresses());
 
-                adapter.stats.onMessageProcessingFinished(msg);
+                spi.stats.onMessageProcessingFinished(msg);
 
                 return false;
             }
@@ -4522,7 +4522,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 readers.remove(this);
             }
 
-            adapter.stats.onSocketReaderRemoved();
+            spi.stats.onSocketReaderRemoved();
         }
 
         /** {@inheritDoc} */
@@ -4539,13 +4539,13 @@ class ServerImpl extends TcpDiscoveryImpl {
          * Constructor.
          */
         StatisticsPrinter() {
-            super(adapter.ignite().name(), "tcp-disco-stats-printer", log);
+            super(spi.ignite().name(), "tcp-disco-stats-printer", log);
 
-            assert adapter.statsPrintFreq > 0;
+            assert spi.statsPrintFreq > 0;
 
             assert log.isInfoEnabled();
 
-            setPriority(adapter.threadPri);
+            setPriority(spi.threadPri);
         }
 
         /** {@inheritDoc} */
@@ -4555,7 +4555,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 log.debug("Statistics printer has been started.");
 
             while (!isInterrupted()) {
-                Thread.sleep(adapter.statsPrintFreq);
+                Thread.sleep(spi.statsPrintFreq);
 
                 printStatistics();
             }
@@ -4650,7 +4650,7 @@ class ServerImpl extends TcpDiscoveryImpl {
          *
          */
         public boolean ping() throws InterruptedException {
-            if (adapter.isNodeStopping0())
+            if (spi.isNodeStopping0())
                 return false;
 
             GridFutureAdapter<Boolean> fut;
@@ -4675,7 +4675,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             }
 
             try {
-                return fut.get(adapter.ackTimeout, TimeUnit.MILLISECONDS);
+                return fut.get(spi.ackTimeout, TimeUnit.MILLISECONDS);
             }
             catch (IgniteInterruptedCheckedException ignored) {
                 throw new InterruptedException();
@@ -4718,9 +4718,9 @@ class ServerImpl extends TcpDiscoveryImpl {
          * @param name Thread name.
          */
         protected MessageWorkerAdapter(String name) {
-            super(adapter.ignite().name(), name, log);
+            super(spi.ignite().name(), name, log);
 
-            setPriority(adapter.threadPri);
+            setPriority(spi.threadPri);
         }
 
         /** {@inheritDoc} */
@@ -4787,7 +4787,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             throws IOException, IgniteCheckedException {
             bout.reset();
 
-            adapter.writeToSocket(sock, msg, bout);
+            spi.writeToSocket(sock, msg, bout);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/0e192ef8/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index 8dad92a..4836911 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -40,7 +40,7 @@ abstract class TcpDiscoveryImpl {
     protected static final int RES_WAIT = 200;
 
     /** */
-    protected final TcpDiscoverySpi adapter;
+    protected final TcpDiscoverySpi spi;
 
     /** */
     protected final IgniteLogger log;
@@ -49,19 +49,19 @@ abstract class TcpDiscoveryImpl {
     protected TcpDiscoveryNode locNode;
 
     /**
-     * @param adapter Adapter.
+     * @param spi Adapter.
      */
-    TcpDiscoveryImpl(TcpDiscoverySpi adapter) {
-        this.adapter = adapter;
+    TcpDiscoveryImpl(TcpDiscoverySpi spi) {
+        this.spi = spi;
 
-        log = adapter.log;
+        log = spi.log;
     }
 
     /**
      *
      */
     public UUID getLocalNodeId() {
-        return adapter.getLocalNodeId();
+        return spi.getLocalNodeId();
     }
 
     /**
@@ -69,7 +69,7 @@ abstract class TcpDiscoveryImpl {
      * @param e Exception.
      */
     protected void onException(String msg, Exception e){
-        adapter.getExceptionRegistry().onException(msg, e);
+        spi.getExceptionRegistry().onException(msg, e);
     }
 
     /**


[06/53] [abbrv] incubator-ignite git commit: # IGNITE-943 Fix tests.

Posted by se...@apache.org.
# IGNITE-943 Fix tests.


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

Branch: refs/heads/ignite-sprint-5
Commit: bf0e15750d30a05a4371b86ad2135a9b5fd233cb
Parents: 838c0fd
Author: sevdokimov <se...@gridgain.com>
Authored: Wed May 27 17:01:02 2015 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Wed May 27 17:01:02 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/spi/discovery/tcp/ServerImpl.java  |  2 +-
 .../apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java | 11 +++++++++--
 .../spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java |  3 +--
 3 files changed, 11 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf0e1575/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 7e1f592..46a89aa 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
@@ -78,7 +78,7 @@ class ServerImpl extends TcpDiscoveryImpl {
     private RingMessageWorker msgWorker;
 
     /** Client message workers. */
-    private ConcurrentMap<UUID, ClientMessageWorker> clientMsgWorkers = new ConcurrentHashMap8<>();
+    protected ConcurrentMap<UUID, ClientMessageWorker> clientMsgWorkers = new ConcurrentHashMap8<>();
 
     /** Metrics sender. */
     @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf0e1575/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 2b2c691..922c496 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
@@ -1495,6 +1495,8 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
                 sockTimeout = DFLT_SOCK_TIMEOUT_CLIENT;
 
             impl = new ClientImpl(this);
+
+            ctxInitLatch.countDown();
         }
         else {
             if (ackTimeout == 0)
@@ -1506,8 +1508,6 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
             impl = new ServerImpl(this);
         }
 
-        startStopwatch();
-
         assertParameter(ipFinder != null, "ipFinder != null");
         assertParameter(hbFreq > 0, "heartbeatFreq > 0");
         assertParameter(netTimeout > 0, "networkTimeout > 0");
@@ -1636,6 +1636,13 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
     /**
      * <strong>FOR TEST ONLY!!!</strong>
      */
+    public int clientWorkerCount() {
+        return ((ServerImpl)impl).clientMsgWorkers.size();
+    }
+
+    /**
+     * <strong>FOR TEST ONLY!!!</strong>
+     */
     public void addSendMessageListener(IgniteInClosure<TcpDiscoveryAbstractMessage> lsnr) {
         sendMsgLsnrs.add(lsnr);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/bf0e1575/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
index 1655c33..f27ef33 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpiSelfTest.java
@@ -25,7 +25,6 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.io.*;
 import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
@@ -341,7 +340,7 @@ public class TcpClientDiscoverySpiSelfTest extends GridCommonAbstractTest {
 
         attachListeners(3, 3);
 
-        assert U.<Map>field(G.ignite("server-2").configuration().getDiscoverySpi(), "clientMsgWorkers").isEmpty();
+        assert ((TcpDiscoverySpi)G.ignite("server-2").configuration().getDiscoverySpi()).clientWorkerCount() == 0;
 
         failServer(2);