You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by sb...@apache.org on 2014/12/12 09:39:56 UTC

[1/7] incubator-ignite git commit: # ignite-21

Repository: incubator-ignite
Updated Branches:
  refs/heads/master 52f2b4002 -> c67dcde9f


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
new file mode 100644
index 0000000..fb1c29f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
@@ -0,0 +1,713 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.communication.*;
+import org.eclipse.jetty.util.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.util.direct.*;
+import org.gridgain.grid.util.lang.*;
+import org.gridgain.grid.util.nio.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.*;
+import org.gridgain.testframework.junits.spi.*;
+
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
+public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi> extends GridSpiAbstractTest<T> {
+    /** */
+    private static final Collection<GridTestResources> spiRsrcs = new ArrayList<>();
+
+    /** */
+    protected static final List<TcpCommunicationSpi> spis = new ArrayList<>();
+
+    /** */
+    protected static final List<ClusterNode> nodes = new ArrayList<>();
+
+    /** */
+    private static final int SPI_CNT = 2;
+
+    /** */
+    private static final int ITERS = 10;
+
+    /** */
+    private static int port = 30_000;
+
+    /**
+     *
+     */
+    static {
+        GridTcpCommunicationMessageFactory.registerCustom(new GridTcpCommunicationMessageProducer() {
+            @Override public GridTcpCommunicationMessageAdapter create(byte type) {
+                return new GridTestMessage();
+            }
+        }, GridTestMessage.DIRECT_TYPE);
+    }
+
+    /**
+     * Disable SPI auto-start.
+     */
+    public GridTcpCommunicationSpiRecoverySelfTest() {
+        super(false);
+    }
+
+    /** */
+    @SuppressWarnings({"deprecation"})
+    private class TestListener implements CommunicationListener<GridTcpCommunicationMessageAdapter> {
+        /** */
+        private boolean block;
+
+        /** */
+        private CountDownLatch blockLatch;
+
+        /** */
+        private ConcurrentHashSet<Long> msgIds = new ConcurrentHashSet<>();
+
+        /** */
+        private AtomicInteger rcvCnt = new AtomicInteger();
+
+        /** {@inheritDoc} */
+        @Override public void onMessage(UUID nodeId, GridTcpCommunicationMessageAdapter msg, IgniteRunnable msgC) {
+            // info("Test listener received message: " + msg);
+
+            assertTrue("Unexpected message: " + msg, msg instanceof GridTestMessage);
+
+            GridTestMessage msg0 = (GridTestMessage)msg;
+
+            assertTrue("Duplicated message received: " + msg0, msgIds.add(msg0.getMsgId()));
+
+            rcvCnt.incrementAndGet();
+
+            msgC.run();
+
+            try {
+                synchronized (this) {
+                    while (block) {
+                        info("Test listener blocks.");
+
+                        assert blockLatch != null;
+
+                        blockLatch.countDown();
+
+                        wait();
+
+                        if (block)
+                            continue;
+
+                        info("Test listener throws exception.");
+
+                        throw new RuntimeException("Test exception.");
+                    }
+                }
+            }
+            catch (InterruptedException e) {
+                fail("Unexpected error: " + e);
+            }
+        }
+
+        /**
+         *
+         */
+        void block() {
+            synchronized (this) {
+                block = true;
+
+                blockLatch = new CountDownLatch(1);
+            }
+        }
+
+        /**
+         *
+         */
+        void unblock() {
+            synchronized (this) {
+                block = false;
+
+                notifyAll();
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onDisconnected(UUID nodeId) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBlockListener() throws Exception {
+        // Test listener throws exception and stops selector thread, so must restart SPI.
+        for (int i = 0; i < ITERS; i++) {
+            log.info("Creating SPIs: " + i);
+
+            createSpis();
+
+            try {
+                checkBlockListener();
+            }
+            finally {
+                stopSpis();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("BusyWait")
+    private void checkBlockListener() throws Exception {
+        TcpCommunicationSpi spi0 = spis.get(0);
+        TcpCommunicationSpi spi1 = spis.get(1);
+
+        final TestListener lsnr0 = (TestListener)spi0.getListener();
+        final TestListener lsnr1 = (TestListener)spi1.getListener();
+
+        ClusterNode node0 = nodes.get(0);
+        ClusterNode node1 = nodes.get(1);
+
+        lsnr1.block();
+
+        int msgId = 0;
+
+        for (int j = 0; j < 10; j++) {
+            spi0.sendMessage(node1, new GridTestMessage(node0.id(), ++msgId, 0));
+
+            spi1.sendMessage(node0, new GridTestMessage(node1.id(), ++msgId, 0));
+        }
+
+        lsnr1.blockLatch.await();
+
+        lsnr1.unblock();
+
+        Thread.sleep(500);
+
+        int errCnt = 0;
+
+        int msgs = 0;
+
+        while (true) {
+            try {
+                int id = msgId + 1;
+
+                spi0.sendMessage(node1, new GridTestMessage(node0.id(), id, 0));
+
+                msgId++;
+
+                msgs++;
+
+                if (msgs == 10)
+                    break;
+            }
+            catch (IgniteSpiException e) {
+                errCnt++;
+
+                if (errCnt > 10)
+                    fail("Failed to send message: " + e);
+            }
+        }
+
+        for (int j = 0; j < 10; j++)
+            spi1.sendMessage(node0, new GridTestMessage(node1.id(), ++msgId, 0));
+
+        final int expMsgs = 20;
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return lsnr0.rcvCnt.get() >= expMsgs && lsnr1.rcvCnt.get() >= expMsgs;
+            }
+        }, 5000);
+
+        assertEquals(expMsgs, lsnr0.rcvCnt.get());
+        assertEquals(expMsgs, lsnr1.rcvCnt.get());
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBlockRead1() throws Exception {
+        createSpis();
+
+        try {
+            final TcpCommunicationSpi spi0 = spis.get(0);
+            final TcpCommunicationSpi spi1 = spis.get(1);
+
+            final TestListener lsnr1 = (TestListener)spi1.getListener();
+
+            final ClusterNode node0 = nodes.get(0);
+            final ClusterNode node1 = nodes.get(1);
+
+            final AtomicInteger msgId = new AtomicInteger();
+
+            // Send message to establish connection.
+            spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
+
+            final AtomicInteger sentCnt = new AtomicInteger(1);
+
+            int errCnt = 0;
+
+            for (int i = 0; i < ITERS; i++) {
+                log.info("Iteration: " + i);
+
+                try {
+                    final GridNioSession ses0 = communicationSession(spi0);
+                    final GridNioSession ses1 = communicationSession(spi1);
+
+                    ses1.pauseReads().get();
+
+                    IgniteFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            for (int i = 0; i < 5000; i++) {
+                                spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
+
+                                sentCnt.incrementAndGet();
+                            }
+
+                            return null;
+                        }
+                    });
+
+                    // Wait when session is closed because of write timeout.
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override public boolean apply() {
+                            return ses0.closeTime() != 0;
+                        }
+                    }, 5000);
+
+                    assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
+
+                    ses1.resumeReads().get();
+
+                    for (int j = 0; j < 100; j++) {
+                        spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
+
+                        sentCnt.incrementAndGet();
+                    }
+
+                    sndFut.get();
+
+                    final int expMsgs = sentCnt.get();
+
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override public boolean apply() {
+                            return lsnr1.rcvCnt.get() >= expMsgs;
+                        }
+                    }, 60_000);
+
+                    assertEquals(expMsgs, lsnr1.rcvCnt.get());
+                }
+                catch (GridException e) {
+                    if (e.hasCause(BindException.class)) {
+                        errCnt++;
+
+                        if (errCnt > 3) {
+                            log.warning("Got exception > 3 times, test fails.");
+
+                            throw e;
+                        }
+
+                        if (i < ITERS - 1) {
+                            info("Got exception caused by BindException, will retry after delay: " + e);
+
+                            U.sleep(10_000);
+                        }
+                        else
+                            info("Got exception caused by BindException, will ignore: " + e);
+                    }
+                    else {
+                        log.warning("Unexpected exception: " + e, e);
+
+                        throw e;
+                    }
+                }
+            }
+        }
+        finally {
+            stopSpis();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBlockRead2() throws Exception {
+        createSpis();
+
+        try {
+            final TcpCommunicationSpi spi0 = spis.get(0);
+            final TcpCommunicationSpi spi1 = spis.get(1);
+
+            final TestListener lsnr0 = (TestListener)spi0.getListener();
+            final TestListener lsnr1 = (TestListener)spi1.getListener();
+
+            final ClusterNode node0 = nodes.get(0);
+            final ClusterNode node1 = nodes.get(1);
+
+            final AtomicInteger msgId = new AtomicInteger();
+
+            final AtomicInteger expCnt0 = new AtomicInteger();
+
+            final AtomicInteger expCnt1 = new AtomicInteger();
+
+            // Send message to establish connection.
+            spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
+
+            expCnt1.incrementAndGet();
+
+            int errCnt = 0;
+
+            for (int i = 0; i < ITERS; i++) {
+                log.info("Iteration: " + i);
+
+                try {
+                    final GridNioSession ses0 = communicationSession(spi0);
+                    final GridNioSession ses1 = communicationSession(spi1);
+
+                    ses1.pauseReads().get();
+
+                    IgniteFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            for (int i = 0; i < 5000; i++) {
+                                spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
+
+                                expCnt1.incrementAndGet();
+                            }
+
+                            return null;
+                        }
+                    });
+
+                    // Wait when session is closed because of write timeout.
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override public boolean apply() {
+                            return ses0.closeTime() != 0;
+                        }
+                    }, 5000);
+
+                    assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
+
+                    ses1.resumeReads().get();
+
+                    // Wait when session is closed, then try to open new connection from node1.
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override
+                        public boolean apply() {
+                            return ses1.closeTime() != 0;
+                        }
+                    }, 5000);
+
+                    assertTrue("Failed to wait for session close", ses1.closeTime() != 0);
+
+                    for (int j = 0; j < 100; j++) {
+                        spi1.sendMessage(node0, new GridTestMessage(node1.id(), msgId.incrementAndGet(), 0));
+
+                        expCnt0.incrementAndGet();
+                    }
+
+                    sndFut.get();
+
+                    final int expMsgs0 = expCnt0.get();
+                    final int expMsgs1 = expCnt1.get();
+
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override public boolean apply() {
+                            return lsnr0.rcvCnt.get() >= expMsgs0 && lsnr1.rcvCnt.get() >= expMsgs1;
+                        }
+                    }, 60_000);
+
+                    assertEquals(expMsgs0, lsnr0.rcvCnt.get());
+                    assertEquals(expMsgs1, lsnr1.rcvCnt.get());
+                }
+                catch (GridException e) {
+                    if (e.hasCause(BindException.class)) {
+                        errCnt++;
+
+                        if (errCnt > 3) {
+                            log.warning("Got exception > 3 times, test fails.");
+
+                            throw e;
+                        }
+
+                        if (i < ITERS - 1) {
+                            info("Got exception caused by BindException, will retry after delay: " + e);
+
+                            U.sleep(10_000);
+                        }
+                        else
+                            info("Got exception caused by BindException, will ignore: " + e);
+                    }
+                    else {
+                        log.warning("Unexpected exception: " + e, e);
+
+                        throw e;
+                    }
+                }
+            }
+        }
+        finally {
+            stopSpis();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBlockRead3() throws Exception {
+        createSpis();
+
+        try {
+            final TcpCommunicationSpi spi0 = spis.get(0);
+            final TcpCommunicationSpi spi1 = spis.get(1);
+
+            final TestListener lsnr1 = (TestListener)spi1.getListener();
+
+            final ClusterNode node0 = nodes.get(0);
+            final ClusterNode node1 = nodes.get(1);
+
+            final AtomicInteger msgId = new AtomicInteger();
+
+            // Send message to establish connection.
+            spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
+
+            final AtomicInteger sentCnt = new AtomicInteger(1);
+
+            int errCnt = 0;
+
+            for (int i = 0; i < ITERS; i++) {
+                log.info("Iteration: " + i);
+
+                try {
+                    final GridNioSession ses0 = communicationSession(spi0);
+                    final GridNioSession ses1 = communicationSession(spi1);
+
+                    ses1.pauseReads().get();
+
+                    IgniteFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            for (int i = 0; i < 5000; i++) {
+                                spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
+
+                                sentCnt.incrementAndGet();
+                            }
+
+                            return null;
+                        }
+                    });
+
+                    // Wait when session is closed because of write timeout.
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override public boolean apply() {
+                            return ses0.closeTime() != 0;
+                        }
+                    }, 5000);
+
+                    assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
+
+                    ses1.resumeReads().get();
+
+                    sndFut.get();
+
+                    final int expMsgs = sentCnt.get();
+
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override public boolean apply() {
+                            return lsnr1.rcvCnt.get() >= expMsgs;
+                        }
+                    }, 60_000);
+
+                    assertEquals(expMsgs, lsnr1.rcvCnt.get());
+                }
+                catch (GridException e) {
+                    if (e.hasCause(BindException.class)) {
+                        errCnt++;
+
+                        if (errCnt > 3) {
+                            log.warning("Got exception > 3 times, test fails.");
+
+                            throw e;
+                        }
+
+                        if (i < ITERS - 1) {
+                            info("Got exception caused by BindException, will retry after delay: " + e);
+
+                            U.sleep(10_000);
+                        }
+                        else
+                            info("Got exception caused by BindException, will ignore: " + e);
+                    }
+                    else {
+                        log.warning("Unexpected exception: " + e, e);
+
+                        throw e;
+                    }
+                }
+            }
+        }
+        finally {
+            stopSpis();
+        }
+    }
+
+    /**
+     * @param spi SPI.
+     * @return Session.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    private GridNioSession communicationSession(TcpCommunicationSpi spi) throws Exception {
+        final GridNioServer srv = U.field(spi, "nioSrvr");
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
+
+                return !sessions.isEmpty();
+            }
+        }, 5000);
+
+        Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
+
+        assertEquals(1, sessions.size());
+
+        return sessions.iterator().next();
+    }
+
+    /**
+     * @param idx SPI index.
+     * @return SPI instance.
+     */
+    protected TcpCommunicationSpi getSpi(int idx) {
+        TcpCommunicationSpi spi = new TcpCommunicationSpi();
+
+        spi.setSharedMemoryPort(-1);
+        spi.setLocalPort(port++);
+        spi.setIdleConnectionTimeout(10_000);
+        spi.setConnectTimeout(10_000);
+        spi.setAckSendThreshold(5);
+        spi.setSocketWriteTimeout(1000);
+        spi.setSocketSendBuffer(512);
+        spi.setSocketReceiveBuffer(512);
+
+        return spi;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void startSpis() throws Exception {
+        spis.clear();
+        nodes.clear();
+        spiRsrcs.clear();
+
+        Map<ClusterNode, GridSpiTestContext> ctxs = new HashMap<>();
+
+        for (int i = 0; i < SPI_CNT; i++) {
+            TcpCommunicationSpi spi = getSpi(i);
+
+            GridTestUtils.setFieldValue(spi, "gridName", "grid-" + i);
+
+            GridTestResources rsrcs = new GridTestResources();
+
+            GridTestNode node = new GridTestNode(rsrcs.getNodeId());
+
+            node.order(i);
+
+            GridSpiTestContext ctx = initSpiContext();
+
+            ctx.setLocalNode(node);
+
+            spiRsrcs.add(rsrcs);
+
+            rsrcs.inject(spi);
+
+            spi.setListener(new TestListener());
+
+            node.setAttributes(spi.getNodeAttributes());
+
+            nodes.add(node);
+
+            spi.spiStart(getTestGridName() + (i + 1));
+
+            spis.add(spi);
+
+            spi.onContextInitialized(ctx);
+
+            ctxs.put(node, ctx);
+        }
+
+        // For each context set remote nodes.
+        for (Map.Entry<ClusterNode, GridSpiTestContext> e : ctxs.entrySet()) {
+            for (ClusterNode n : nodes) {
+                if (!n.equals(e.getKey()))
+                    e.getValue().remoteNodes().add(n);
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void createSpis() throws Exception {
+        for (int i = 0; i < 3; i++) {
+            try {
+                startSpis();
+
+                break;
+            }
+            catch (GridException e) {
+                if (e.hasCause(BindException.class)) {
+                    if (i < 2) {
+                        info("Failed to start SPIs because of BindException, will retry after delay.");
+
+                        stopSpis();
+
+                        U.sleep(10_000);
+                    }
+                    else
+                        throw e;
+                }
+                else
+                    throw e;
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void stopSpis() throws Exception {
+        for (CommunicationSpi<GridTcpCommunicationMessageAdapter> spi : spis) {
+            spi.onContextDestroyed();
+
+            spi.setListener(null);
+
+            spi.spiStop();
+        }
+
+        for (GridTestResources rsrcs : spiRsrcs) {
+            rsrcs.stopThreads();
+        }
+
+        spis.clear();
+        nodes.clear();
+        spiRsrcs.clear();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiShmemSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiShmemSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiShmemSelfTest.java
index ff093ce..85ae027 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiShmemSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiShmemSelfTest.java
@@ -18,6 +18,6 @@ import org.gridgain.testframework.junits.spi.*;
 public class GridTcpCommunicationSpiShmemSelfTest extends GridTcpCommunicationSpiAbstractTest {
     /** */
     public GridTcpCommunicationSpiShmemSelfTest() {
-        super(true);
+        super(false);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiTcpNoDelayOffSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiTcpNoDelayOffSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiTcpNoDelayOffSelfTest.java
new file mode 100644
index 0000000..e18ca96
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiTcpNoDelayOffSelfTest.java
@@ -0,0 +1,20 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+/**
+ *
+ */
+public class GridTcpCommunicationSpiTcpNoDelayOffSelfTest extends GridTcpCommunicationSpiTcpSelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean tcpNoDelay() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java
index 28863d5..15e4d39 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java
@@ -120,6 +120,8 @@ public class GridManagerStopSelfTest extends GridCommonAbstractTest {
 
         GridIoManager mgr = new GridIoManager(ctx);
 
+        mgr.onKernalStop(true);
+
         mgr.stop(false);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/gridgain/grid/spi/GridSpiStartStopAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/spi/GridSpiStartStopAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/spi/GridSpiStartStopAbstractTest.java
index 1c11d91..ebbfe16 100644
--- a/modules/core/src/test/java/org/gridgain/grid/spi/GridSpiStartStopAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/spi/GridSpiStartStopAbstractTest.java
@@ -65,6 +65,8 @@ public abstract class GridSpiStartStopAbstractTest<T extends IgniteSpi> extends
 
         getTestResources().inject(spi);
 
+        spi.onContextDestroyed();
+
         spi.spiStop();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/gridgain/grid/util/nio/impl/GridNioFilterChainSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/util/nio/impl/GridNioFilterChainSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/util/nio/impl/GridNioFilterChainSelfTest.java
index 47d5492..b62507e 100644
--- a/modules/core/src/test/java/org/gridgain/grid/util/nio/impl/GridNioFilterChainSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/util/nio/impl/GridNioFilterChainSelfTest.java
@@ -368,5 +368,15 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         @Override public boolean readsPaused() {
             return false;
         }
+
+        /** {@inheritDoc} */
+        @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+            return null;
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/gridgain/testframework/GridTestNode.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testframework/GridTestNode.java b/modules/core/src/test/java/org/gridgain/testframework/GridTestNode.java
index da867c7..01ed47e 100644
--- a/modules/core/src/test/java/org/gridgain/testframework/GridTestNode.java
+++ b/modules/core/src/test/java/org/gridgain/testframework/GridTestNode.java
@@ -48,6 +48,9 @@ public class GridTestNode extends GridMetadataAwareAdapter implements ClusterNod
     private ClusterNodeMetrics metrics;
 
     /** */
+    private long order;
+
+    /** */
     public GridTestNode() {
         // No-op.
 
@@ -177,7 +180,14 @@ public class GridTestNode extends GridMetadataAwareAdapter implements ClusterNod
 
     /** {@inheritDoc} */
     @Override public long order() {
-        return metrics == null ? -1 : metrics.getStartTime();
+        return order != 0 ? order : (metrics == null ? -1 : metrics.getStartTime());
+    }
+
+    /**
+     * @param order Order.
+     */
+    public void order(long order) {
+        this.order = order;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/gridgain/testsuites/GridSpiCommunicationSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/testsuites/GridSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/gridgain/testsuites/GridSpiCommunicationSelfTestSuite.java
index e7c0c0d..548f660 100644
--- a/modules/core/src/test/java/org/gridgain/testsuites/GridSpiCommunicationSelfTestSuite.java
+++ b/modules/core/src/test/java/org/gridgain/testsuites/GridSpiCommunicationSelfTestSuite.java
@@ -21,9 +21,15 @@ public class GridSpiCommunicationSelfTestSuite extends TestSuite {
      * @throws Exception If failed.
      */
     public static TestSuite suite() throws Exception {
-        TestSuite suite = new TestSuite("Gridgain Communication SPI Test Suite");
+        TestSuite suite = new TestSuite("Communication SPI Test Suite");
+
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoveryAckSelfTest.class));
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoverySelfTest.class));
+
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSelfTest.class));
 
         suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpSelfTest.class));
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiTcpNoDelayOffSelfTest.class));
         suite.addTest(new TestSuite(GridTcpCommunicationSpiShmemSelfTest.class));
 
         suite.addTest(new TestSuite(GridTcpCommunicationSpiStartStopSelfTest.class));


[2/7] incubator-ignite git commit: # ignite-21

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioServer.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioServer.java
index 1e61a17..ddd301b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioServer.java
@@ -11,6 +11,7 @@ package org.gridgain.grid.util.nio;
 
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.lang.*;
 import org.apache.ignite.thread.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.util.*;
@@ -329,7 +330,7 @@ public class GridNioServer<T> {
 
         NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
 
-        send0(impl, fut);
+        send0(impl, fut, false);
 
         return fut;
     }
@@ -346,7 +347,7 @@ public class GridNioServer<T> {
 
         NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
 
-        send0(impl, fut);
+        send0(impl, fut, false);
 
         return fut;
     }
@@ -354,19 +355,17 @@ public class GridNioServer<T> {
     /**
      * @param ses Session.
      * @param fut Future.
+     * @param sys System message flag.
      */
-    private void send0(GridSelectorNioSessionImpl ses, NioOperationFuture<?> fut) {
+    private void send0(GridSelectorNioSessionImpl ses, NioOperationFuture<?> fut, boolean sys) {
         assert ses != null;
         assert fut != null;
 
-        int msgCnt = ses.offerFuture(fut);
+        int msgCnt = sys ? ses.offerSystemFuture(fut) : ses.offerFuture(fut);
 
         if (ses.closed()) {
-            NioOperationFuture<?> fut0;
-
-            // Cleanup as session.close() may have been already finished.
-            while ((fut0 = (NioOperationFuture<?>)ses.pollFuture()) != null)
-                fut0.connectionClosed();
+            if (ses.removeFuture(fut))
+                fut.connectionClosed();
         }
         else if (msgCnt == 1)
             // Change from 0 to 1 means that worker thread should be waken up.
@@ -374,6 +373,76 @@ public class GridNioServer<T> {
     }
 
     /**
+     * Adds message at the front of the queue without acquiring back pressure semaphore.
+     *
+     * @param ses Session.
+     * @param msg Message.
+     * @return Future.
+     */
+    public GridNioFuture<?> sendSystem(GridNioSession ses, GridTcpCommunicationMessageAdapter msg) {
+        return sendSystem(ses, msg, null);
+    }
+
+    /**
+     * Adds message at the front of the queue without acquiring back pressure semaphore.
+     *
+     * @param ses Session.
+     * @param msg Message.
+     * @param lsnr Future listener notified from the session thread.
+     * @return Future.
+     */
+    public GridNioFuture<?> sendSystem(GridNioSession ses,
+        GridTcpCommunicationMessageAdapter msg,
+        @Nullable IgniteInClosure<? super GridNioFuture<?>> lsnr) {
+        assert ses instanceof GridSelectorNioSessionImpl;
+
+        GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
+
+        NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
+
+        if (lsnr != null) {
+            fut.listenAsync(lsnr);
+
+            assert !fut.isDone();
+        }
+
+        send0(impl, fut, true);
+
+        return fut;
+    }
+
+    /**
+     * @param ses Session.
+     */
+    public void resend(GridNioSession ses) {
+        assert ses instanceof GridSelectorNioSessionImpl;
+
+        GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+
+        if (recoveryDesc != null && !recoveryDesc.messagesFutures().isEmpty()) {
+            Deque<GridNioFuture<?>> futs = recoveryDesc.messagesFutures();
+
+            if (log.isDebugEnabled())
+                log.debug("Resend messages [rmtNode=" + recoveryDesc.node().id() + ", msgCnt=" + futs.size() + ']');
+
+            GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
+
+            GridNioFuture<?> fut0 = futs.iterator().next();
+
+            for (GridNioFuture<?> fut : futs) {
+                fut.messageThread(true);
+
+                ((NioOperationFuture)fut).resetMessage(ses0);
+            }
+
+            ses0.resend(futs);
+
+            // Wake up worker.
+            clientWorkers.get(ses0.selectorIndex()).offer(((NioOperationFuture)fut0));
+        }
+    }
+
+    /**
      * @param ses Session.
      * @param op Operation.
      * @return Future for operation.
@@ -385,7 +454,8 @@ public class GridNioServer<T> {
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
         if (impl.closed())
-            return new GridNioFinishedFuture(new IOException("Failed to send message (connection was closed): " + ses));
+            return new GridNioFinishedFuture(new IOException("Failed to pause/resume reads " +
+                "(connection was closed): " + ses));
 
         NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, op);
 
@@ -406,7 +476,7 @@ public class GridNioServer<T> {
         try {
             ch.configureBlocking(false);
 
-            NioOperationFuture<GridNioSession> req = new NioOperationFuture<>(ch, NioOperation.REGISTER, false, meta);
+            NioOperationFuture<GridNioSession> req = new NioOperationFuture<>(ch, false, meta);
 
             offerBalanced(req);
 
@@ -437,7 +507,7 @@ public class GridNioServer<T> {
 
     /**
      * Gets configurable idle timeout for this session. If not set, default value is
-     * {@link org.apache.ignite.configuration.IgniteConfiguration#DFLT_REST_IDLE_TIMEOUT}.
+     * {@link IgniteConfiguration#DFLT_REST_IDLE_TIMEOUT}.
      *
      * @return Idle timeout in milliseconds.
      */
@@ -1313,9 +1383,16 @@ public class GridNioServer<T> {
                     readBuf.order(order);
                 }
 
-                final GridSelectorNioSessionImpl ses = new GridSelectorNioSessionImpl(idx, filterChain,
-                    (InetSocketAddress)sockCh.getLocalAddress(), (InetSocketAddress)sockCh.getRemoteAddress(),
-                    req.accepted(), sndQueueLimit, writeBuf, readBuf);
+                final GridSelectorNioSessionImpl ses = new GridSelectorNioSessionImpl(
+                    log,
+                    idx,
+                    filterChain,
+                    (InetSocketAddress)sockCh.getLocalAddress(),
+                    (InetSocketAddress)sockCh.getRemoteAddress(),
+                    req.accepted(),
+                    sndQueueLimit,
+                    writeBuf,
+                    readBuf);
 
                 Map<Integer, ?> meta = req.meta();
 
@@ -1328,6 +1405,9 @@ public class GridNioServer<T> {
 
                 ses.key(key);
 
+                if (!ses.accepted())
+                    resend(ses);
+
                 sessions.add(ses);
 
                 try {
@@ -1418,11 +1498,27 @@ public class GridNioServer<T> {
                 // Since ses is in closed state, no write requests will be added.
                 NioOperationFuture<?> fut = ses.removeMeta(NIO_OPERATION.ordinal());
 
-                if (fut != null)
-                    fut.connectionClosed();
+                GridNioRecoveryDescriptor recovery = ses.recoveryDescriptor();
 
-                while ((fut = (NioOperationFuture<?>)ses.pollFuture()) != null)
-                    fut.connectionClosed();
+                if (recovery != null) {
+                    try {
+                        // Poll will update recovery data.
+                        while ((fut = (NioOperationFuture<?>)ses.pollFuture()) != null) {
+                            if (fut.skipRecovery())
+                                fut.connectionClosed();
+                        }
+                    }
+                    finally {
+                        recovery.release();
+                    }
+                }
+                else {
+                    if (fut != null)
+                        fut.connectionClosed();
+
+                    while ((fut = (NioOperationFuture<?>)ses.pollFuture()) != null)
+                        fut.connectionClosed();
+                }
 
                 return true;
             }
@@ -1669,19 +1765,22 @@ public class GridNioServer<T> {
          * @param sockCh Socket channel to register on selector.
          */
         NioOperationFuture(SocketChannel sockCh) {
-            this(sockCh, NioOperation.REGISTER, true, null);
+            this(sockCh, true, null);
         }
 
         /**
          * @param sockCh Socket channel.
-         * @param op Operation.
          * @param accepted {@code True} if socket has been accepted.
          * @param meta Optional meta.
          */
-        NioOperationFuture(SocketChannel sockCh, NioOperation op, boolean accepted,
-            @Nullable Map<Integer, ?> meta) {
+        NioOperationFuture(
+            SocketChannel sockCh,
+            boolean accepted,
+            @Nullable Map<Integer, ?> meta
+        ) {
+            op = NioOperation.REGISTER;
+
             this.sockCh = sockCh;
-            this.op = op;
             this.accepted = accepted;
             this.meta = meta;
         }
@@ -1761,6 +1860,17 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @param ses New session instance.
+         */
+        private void resetMessage(GridSelectorNioSessionImpl ses) {
+            assert commMsg != null;
+
+            commMsg = commMsg.clone();
+
+            this.ses = ses;
+        }
+
+        /**
          * @return Socket channel for register request.
          */
         private SocketChannel socketChannel() {
@@ -1799,6 +1909,11 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
+        @Override public boolean skipRecovery() {
+            return commMsg != null && commMsg.skipRecovery();
+        }
+
+        /** {@inheritDoc} */
         @Override public String toString() {
             return S.toString(NioOperationFuture.class, this);
         }
@@ -1836,9 +1951,9 @@ public class GridNioServer<T> {
         /** {@inheritDoc} */
         @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
             if (directMode) {
-                boolean sslSystem = sslFilter != null && msg instanceof ByteBuffer;
+                boolean sslSys = sslFilter != null && msg instanceof ByteBuffer;
 
-                if (sslSystem) {
+                if (sslSys) {
                     ConcurrentLinkedDeque8<ByteBuffer> queue = ses.meta(BUF_SSL_SYSTEM_META_KEY);
 
                     queue.offer((ByteBuffer)msg);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSession.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSession.java
index e1600ba..27acfcc 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSession.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSession.java
@@ -146,4 +146,14 @@ public interface GridNioSession {
      * @return {@code True} if reads are paused.
      */
     public boolean readsPaused();
+
+    /**
+     * @param recoveryDesc Recovery descriptor.
+     */
+    public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
+
+    /**
+     * @return Recovery descriptor if recovery is supported, {@code null otherwise.}
+     */
+    @Nullable public GridNioRecoveryDescriptor recoveryDescriptor();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSessionImpl.java
index d81f98f..347cb72 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSessionImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSessionImpl.java
@@ -288,6 +288,16 @@ public class GridNioSessionImpl implements GridNioSession {
     }
 
     /** {@inheritDoc} */
+    @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridNioSessionImpl.class, this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridSelectorNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridSelectorNioSessionImpl.java
index c44c6a9..db772dc 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridSelectorNioSessionImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridSelectorNioSessionImpl.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.util.nio;
 
+import org.apache.ignite.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.tostring.*;
 import org.jdk8.backport.*;
@@ -17,6 +18,7 @@ import org.jetbrains.annotations.*;
 import java.net.*;
 import java.nio.*;
 import java.nio.channels.*;
+import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
@@ -49,9 +51,16 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     /** Read buffer. */
     private ByteBuffer readBuf;
 
+    /** Recovery data. */
+    private GridNioRecoveryDescriptor recovery;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
     /**
      * Creates session instance.
      *
+     * @param log Logger.
      * @param selectorIdx Selector index for this session.
      * @param filterChain Filter chain that will handle requests.
      * @param locAddr Local address.
@@ -62,6 +71,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      * @param readBuf Read buffer.
      */
     GridSelectorNioSessionImpl(
+        IgniteLogger log,
         int selectorIdx,
         GridNioFilterChain filterChain,
         InetSocketAddress locAddr,
@@ -79,6 +89,10 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
         assert locAddr != null : "GridSelectorNioSessionImpl should have local socket address.";
         assert rmtAddr != null : "GridSelectorNioSessionImpl should have remote socket address.";
 
+        assert log != null;
+
+        this.log = log;
+
         this.selectorIdx = selectorIdx;
 
         sem = sndQueueLimit > 0 ? new Semaphore(sndQueueLimit) : null;
@@ -136,6 +150,22 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     }
 
     /**
+     * Adds write future at the front of the queue without acquiring back pressure semaphore.
+     *
+     * @param writeFut Write request.
+     * @return Updated size of the queue.
+     */
+    int offerSystemFuture(GridNioFuture<?> writeFut) {
+        writeFut.messageThread(true);
+
+        boolean res = queue.offerFirst(writeFut);
+
+        assert res : "Future was not added to queue";
+
+        return queueSize.incrementAndGet();
+    }
+
+    /**
      * Adds write future to the pending list and returns the size of the queue.
      * <p>
      * Note that separate counter for the queue size is needed because in case of concurrent
@@ -161,6 +191,21 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     }
 
     /**
+     * @param futs Futures to resend.
+     */
+    void resend(Collection<GridNioFuture<?>> futs) {
+        assert queue.isEmpty() : queue.size();
+
+        boolean add = queue.addAll(futs);
+
+        assert add;
+
+        boolean set = queueSize.compareAndSet(0, futs.size());
+
+        assert set;
+    }
+
+    /**
      * @return Message that is in the head of the queue, {@code null} if queue is empty.
      */
     @Nullable GridNioFuture<?> pollFuture() {
@@ -171,12 +216,38 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
 
             if (sem != null && !last.messageThread())
                 sem.release();
+
+            if (recovery != null) {
+                if (!recovery.add(last)) {
+                    LT.warn(log, null, "Unacknowledged messages queue size overflow, will attempt to reconnect " +
+                        "[remoteAddr=" + remoteAddress() +
+                        ", queueLimit=" + recovery.queueLimit() + ']');
+
+                    if (log.isDebugEnabled())
+                        log.debug("Unacknowledged messages queue size overflow, will attempt to reconnect " +
+                            "[remoteAddr=" + remoteAddress() +
+                            ", queueSize=" + recovery.messagesFutures().size() +
+                            ", queueLimit=" + recovery.queueLimit() + ']');
+
+                    close();
+                }
+            }
         }
 
         return last;
     }
 
     /**
+     * @param fut Future.
+     * @return {@code True} if future was removed from queue.
+     */
+    boolean removeFuture(GridNioFuture<?> fut) {
+        assert closed();
+
+        return queue.removeLastOccurrence(fut);
+    }
+
+    /**
      * Gets number of write requests in a queue that have not been processed yet.
      *
      * @return Number of write requests.
@@ -186,6 +257,32 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     }
 
     /** {@inheritDoc} */
+    @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        assert recoveryDesc != null;
+
+        recovery = recoveryDesc;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+        return recovery;
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> T addMeta(int key, @Nullable T val) {
+        if (val instanceof GridNioRecoveryDescriptor) {
+            recovery = (GridNioRecoveryDescriptor)val;
+
+            if (!accepted())
+                recovery.connected();
+
+            return null;
+        }
+        else
+            return super.addMeta(key, val);
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridSelectorNioSessionImpl.class, this, super.toString());
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridShmemCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridShmemCommunicationClient.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridShmemCommunicationClient.java
index d5dec86..505c788 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridShmemCommunicationClient.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridShmemCommunicationClient.java
@@ -103,7 +103,7 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
     }
 
     /** {@inheritDoc} */
-    @Override public synchronized void sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
+    @Override public synchronized boolean sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
         throws GridException {
         if (closed())
             throw new GridException("Communication client was closed: " + this);
@@ -120,6 +120,8 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
         }
 
         markUsed();
+
+        return false;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpCommunicationClient.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpCommunicationClient.java
index d1c8c5e..fbca363 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpCommunicationClient.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpCommunicationClient.java
@@ -182,7 +182,7 @@ public class GridTcpCommunicationClient extends GridAbstractCommunicationClient
     }
 
     /** {@inheritDoc} */
-    @Override public void sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
+    @Override public boolean sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
         throws GridException {
         if (closed())
             throw new GridException("Client was closed: " + this);
@@ -199,6 +199,8 @@ public class GridTcpCommunicationClient extends GridAbstractCommunicationClient
         }
 
         markUsed();
+
+        return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpNioCommunicationClient.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpNioCommunicationClient.java
index 6d2a8f7..55997d3 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpNioCommunicationClient.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpNioCommunicationClient.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.util.nio;
 
+import org.apache.ignite.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.util.direct.*;
 import org.jetbrains.annotations.*;
@@ -23,27 +24,24 @@ import java.util.*;
  * Grid client for NIO server.
  */
 public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClient {
-    /** Socket. */
+    /** Session. */
     private final GridNioSession ses;
 
-    /**
-     * Constructor for test purposes only.
-     */
-    public GridTcpNioCommunicationClient() {
-        super(null);
-
-        ses = null;
-    }
+    /** Logger. */
+    private final IgniteLogger log;
 
     /**
      * @param ses Session.
+     * @param log Logger.
      */
-    public GridTcpNioCommunicationClient(GridNioSession ses) {
+    public GridTcpNioCommunicationClient(GridNioSession ses, IgniteLogger log) {
         super(null);
 
         assert ses != null;
+        assert log != null;
 
         this.ses = ses;
+        this.log = log;
     }
 
     /**
@@ -98,14 +96,11 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
     }
 
     /** {@inheritDoc} */
-    @Override public void sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
+    @Override public boolean sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
         throws GridException {
         // Node ID is never provided in asynchronous send mode.
         assert nodeId == null;
 
-        if (closed())
-            throw new GridException("Client was closed: " + this);
-
         GridNioFuture<?> fut = ses.send(msg);
 
         if (fut.isDone()) {
@@ -113,9 +108,23 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
                 fut.get();
             }
             catch (IOException e) {
-                throw new GridException("Failed to send message [client=" + this + ']', e);
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send message [client=" + this + ", err=" +e + ']');
+
+                return true;
+            }
+            catch (GridException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to send message [client=" + this + ", err=" +e + ']');
+
+                if (e.getCause() instanceof IOException)
+                    return true;
+                else
+                    throw new GridException("Failed to send message [client=" + this + ']', e);
             }
         }
+
+        return false;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
index dc2ee2a..86d68a7 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
@@ -9,20 +9,22 @@
 
 package org.apache.ignite.spi.communication;
 
-import mx4j.tools.adaptor.http.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.util.direct.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.*;
-import org.gridgain.testframework.config.*;
 import org.gridgain.testframework.junits.*;
 import org.gridgain.testframework.junits.spi.*;
 
-import javax.management.*;
+import java.net.*;
 import java.util.*;
 import java.util.Map.*;
 
+import static org.gridgain.grid.kernal.GridNodeAttributes.*;
+
 /**
  * Super class for all communication self tests.
  * @param <T> Type of communication SPI.
@@ -47,22 +49,15 @@ public abstract class GridAbstractCommunicationSelfTest<T extends CommunicationS
     /** */
     private static final Object mux = new Object();
 
-    /** */
-    private static final ObjectName mBeanName;
-
+    /**
+     *
+     */
     static {
         GridTcpCommunicationMessageFactory.registerCustom(new GridTcpCommunicationMessageProducer() {
             @Override public GridTcpCommunicationMessageAdapter create(byte type) {
                 return new GridTestMessage();
             }
         }, GridTestMessage.DIRECT_TYPE);
-
-        try {
-            mBeanName = new ObjectName("mbeanAdaptor:protocol=HTTP");
-        }
-        catch (MalformedObjectNameException e) {
-            throw new GridRuntimeException(e);
-        }
     }
 
     /** */
@@ -237,6 +232,36 @@ public abstract class GridAbstractCommunicationSelfTest<T extends CommunicationS
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        for (int i = 0; i < 3; i++) {
+            try {
+                startSpis();
+
+                break;
+            }
+            catch (GridException e) {
+                if (e.hasCause(BindException.class)) {
+                    if (i < 2) {
+                        info("Failed to start SPIs because of BindException, will retry after delay.");
+
+                        afterTestsStopped();
+
+                        U.sleep(30_000);
+                    }
+                    else
+                        throw e;
+                }
+                else
+                    throw e;
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void startSpis() throws Exception {
+        U.setWorkDirectory(null, U.getGridGainHome());
+
         spis.clear();
         nodes.clear();
         spiRsrcs.clear();
@@ -246,10 +271,14 @@ public abstract class GridAbstractCommunicationSelfTest<T extends CommunicationS
         for (int i = 0; i < getSpiCount(); i++) {
             CommunicationSpi<GridTcpCommunicationMessageAdapter> spi = getSpi(i);
 
-            GridTestResources rsrcs = new GridTestResources(getMBeanServer(i));
+            GridTestUtils.setFieldValue(spi, "gridName", "grid-" + i);
+
+            GridTestResources rsrcs = new GridTestResources();
 
             GridTestNode node = new GridTestNode(rsrcs.getNodeId());
 
+            node.order(i);
+
             GridSpiTestContext ctx = initSpiContext();
 
             ctx.setLocalNode(node);
@@ -263,6 +292,7 @@ public abstract class GridAbstractCommunicationSelfTest<T extends CommunicationS
             spi.setListener(new MessageListener(rsrcs.getNodeId()));
 
             node.setAttributes(spi.getNodeAttributes());
+            node.setAttribute(ATTR_MACS, F.concat(U.allLocalMACs(), ", "));
 
             nodes.add(node);
 
@@ -284,38 +314,17 @@ public abstract class GridAbstractCommunicationSelfTest<T extends CommunicationS
         }
     }
 
-    /**
-     * @param idx Node index.
-     * @return Configured MBean server.
-     * @throws Exception If failed.
-     */
-    private MBeanServer getMBeanServer(int idx) throws Exception {
-        HttpAdaptor mbeanAdaptor = new HttpAdaptor();
-
-        MBeanServer mbeanSrv = MBeanServerFactory.createMBeanServer();
-
-        mbeanAdaptor.setPort(
-            Integer.valueOf(GridTestProperties.getProperty("comm.mbeanserver.selftest.baseport")) + idx);
-
-        mbeanSrv.registerMBean(mbeanAdaptor, mBeanName);
-
-        mbeanAdaptor.start();
-
-        return mbeanSrv;
-    }
-
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         for (CommunicationSpi<GridTcpCommunicationMessageAdapter> spi : spis.values()) {
+            spi.onContextDestroyed();
+
             spi.setListener(null);
 
             spi.spiStop();
         }
 
-        for (GridTestResources rsrcs : spiRsrcs) {
+        for (GridTestResources rsrcs : spiRsrcs)
             rsrcs.stopThreads();
-
-            rsrcs.getMBeanServer().unregisterMBean(mBeanName);
-        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java
index 8015176..58cb184 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridTestMessage.java
@@ -86,14 +86,23 @@ public class GridTestMessage extends GridTcpCommunicationMessageAdapter {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings("CloneDoesntCallSuperClone")
+    @SuppressWarnings({"CloneDoesntCallSuperClone", "CloneCallsConstructors"})
     @Override public GridTcpCommunicationMessageAdapter clone() {
-        throw new UnsupportedOperationException();
+        GridTestMessage msg = new GridTestMessage();
+
+        clone0(msg);
+
+        return msg;
     }
 
     /** {@inheritDoc} */
     @Override protected void clone0(GridTcpCommunicationMessageAdapter _msg) {
-        // No-op.
+        GridTestMessage _clone = (GridTestMessage)_msg;
+
+        _clone.srcNodeId = srcNodeId;
+        _clone.msgId = msgId;
+        _clone.resId = resId;
+        _clone.payload = payload;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
index 16c7ea0..1dabf5d 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiAbstractTest.java
@@ -29,9 +29,6 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
     public static final int IDLE_CONN_TIMEOUT = 2000;
 
     /** */
-    private boolean tcpNoDelay;
-
-    /** */
     private final boolean useShmem;
 
     /**
@@ -50,11 +47,18 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
 
         spi.setLocalPort(GridTestUtils.getNextCommPort(getClass()));
         spi.setIdleConnectionTimeout(IDLE_CONN_TIMEOUT);
-        spi.setTcpNoDelay(tcpNoDelay);
+        spi.setTcpNoDelay(tcpNoDelay());
 
         return spi;
     }
 
+    /**
+     * @return Value of property '{@link TcpCommunicationSpi#isTcpNoDelay()}'.
+     */
+    protected boolean tcpNoDelay() {
+        return true;
+    }
+
     /** {@inheritDoc} */
     @Override protected int getSpiCount() {
         return SPI_COUNT;
@@ -68,21 +72,12 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
         for (CommunicationSpi spi : spis.values()) {
             ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
 
-            assert clients.size() == 2;
+            assertEquals(2, clients.size());
 
             clients.put(UUID.randomUUID(), F.first(clients.values()));
         }
     }
 
-    /**
-     * @throws Exception If failed.
-     */
-    public void testTcpNoDelay() throws Exception {
-        tcpNoDelay = true;
-
-        super.testSendToManyNodes();
-    }
-
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         super.afterTest();
@@ -91,7 +86,8 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
             ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
 
             for (int i = 0; i < 10 && !clients.isEmpty(); i++) {
-                U.warn(log, "Check failed for SPI: " + spi);
+                info("Check failed for SPI [grid=" + GridTestUtils.getFieldValue(spi, "gridName") +
+                    ", spi=" + spi + ']');
 
                 U.sleep(1000);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
new file mode 100644
index 0000000..ffe5d57
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
@@ -0,0 +1,398 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.communication.*;
+import org.eclipse.jetty.util.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.util.direct.*;
+import org.gridgain.grid.util.lang.*;
+import org.gridgain.grid.util.nio.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.*;
+import org.gridgain.testframework.junits.spi.*;
+
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
+public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends CommunicationSpi>
+    extends GridSpiAbstractTest<T> {
+    /** */
+    private static final int SPI_CNT = 2;
+
+    /** */
+    private static final int ITERS = 50;
+
+    /** */
+    private static final Collection<GridTestResources> spiRsrcs = new ArrayList<>();
+
+    /** */
+    protected static final List<CommunicationSpi<GridTcpCommunicationMessageAdapter>> spis = new ArrayList<>();
+
+    /** */
+    protected static final List<ClusterNode> nodes = new ArrayList<>();
+
+    /** */
+    private static int port = 60_000;
+
+    /**
+     *
+     */
+    static {
+        GridTcpCommunicationMessageFactory.registerCustom(new GridTcpCommunicationMessageProducer() {
+            @Override
+            public GridTcpCommunicationMessageAdapter create(byte type) {
+                return new GridTestMessage();
+            }
+        }, GridTestMessage.DIRECT_TYPE);
+    }
+
+    /**
+     * Disable SPI auto-start.
+     */
+    public GridTcpCommunicationSpiConcurrentConnectSelfTest() {
+        super(false);
+    }
+
+    /**
+     *
+     */
+    private static class MessageListener implements CommunicationListener<GridTcpCommunicationMessageAdapter> {
+        /** */
+        private final CountDownLatch latch;
+
+        /** */
+        private final AtomicInteger cntr = new AtomicInteger();
+
+        /** */
+        private final ConcurrentHashSet<Long> msgIds = new ConcurrentHashSet<>();
+
+        /**
+         * @param latch Latch.
+         */
+        MessageListener(CountDownLatch latch) {
+            this.latch = latch;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessage(UUID nodeId, GridTcpCommunicationMessageAdapter msg, IgniteRunnable msgC) {
+            msgC.run();
+
+            assertTrue(msg instanceof GridTestMessage);
+
+            cntr.incrementAndGet();
+
+            GridTestMessage msg0 = (GridTestMessage)msg;
+
+            assertEquals(nodeId, msg0.getSourceNodeId());
+
+            assertTrue(msgIds.add(msg0.getMsgId()));
+
+            latch.countDown();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onDisconnected(UUID nodeId) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testTwoThreads() throws Exception {
+        concurrentConnect(2, 10, ITERS, false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultithreaded() throws Exception {
+        int threads = Runtime.getRuntime().availableProcessors() * 5;
+
+        concurrentConnect(threads, 10, ITERS, false, false);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testWithLoad() throws Exception {
+        int threads = Runtime.getRuntime().availableProcessors() * 5;
+
+        concurrentConnect(threads, 10, ITERS / 2, false, true);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomSleep() throws Exception {
+        concurrentConnect(4, 1, ITERS, true, false);
+    }
+
+    /**
+     * @param threads Number of threads.
+     * @param msgPerThread Messages per thread.
+     * @param iters Number of iterations.
+     * @param sleep If {@code true} sleeps random time before starts send messages.
+     * @param load Run load threads flag.
+     * @throws Exception If failed.
+     */
+    private void concurrentConnect(final int threads,
+        final int msgPerThread,
+        final int iters,
+        final boolean sleep,
+        boolean load) throws Exception {
+        log.info("Concurrent connect [threads=" + threads +
+            ", msgPerThread=" + msgPerThread +
+            ", iters=" + iters +
+            ", load=" + load +
+            ", sleep=" + sleep + ']');
+
+        final AtomicBoolean stop = new AtomicBoolean();
+
+        IgniteFuture<?> loadFut = null;
+
+        if (load) {
+            loadFut = GridTestUtils.runMultiThreadedAsync(new Callable<Long>() {
+                @Override
+                public Long call() throws Exception {
+                    long dummyRes = 0;
+
+                    List<String> list = new ArrayList<>();
+
+                    while (!stop.get()) {
+                        for (int i = 0; i < 100; i++) {
+                            String str = new String(new byte[i]);
+
+                            list.add(str);
+
+                            dummyRes += str.hashCode();
+                        }
+
+                        if (list.size() > 1000_000) {
+                            list = new ArrayList<>();
+
+                            System.gc();
+                        }
+                    }
+
+                    return dummyRes;
+                }
+            }, 2, "test-load");
+        }
+
+        try {
+            for (int i = 0; i < iters; i++) {
+                log.info("Iteration: " + i);
+
+                final AtomicInteger msgId = new AtomicInteger();
+
+                final int expMsgs = threads * msgPerThread;
+
+                CountDownLatch latch = new CountDownLatch(expMsgs);
+
+                MessageListener lsnr = new MessageListener(latch);
+
+                createSpis(lsnr);
+
+                final AtomicInteger idx = new AtomicInteger();
+
+                try {
+                    GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                        @Override public Void call() throws Exception {
+                            int idx0 = idx.getAndIncrement();
+
+                            Thread.currentThread().setName("Test thread [idx=" + idx0 + ", grid=" + (idx0 % 2) + ']');
+
+                            CommunicationSpi<GridTcpCommunicationMessageAdapter> spi = spis.get(idx0 % 2);
+
+                            ClusterNode srcNode = nodes.get(idx0 % 2);
+
+                            ClusterNode dstNode = nodes.get((idx0 + 1) % 2);
+
+                            if (sleep) {
+                                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                                long millis = rnd.nextLong(10);
+
+                                if (millis > 0)
+                                    Thread.sleep(millis);
+                            }
+
+                            for (int i = 0; i < msgPerThread; i++)
+                                spi.sendMessage(dstNode, new GridTestMessage(srcNode.id(), msgId.incrementAndGet(), 0));
+
+                            return null;
+                        }
+                    }, threads, "test");
+
+                    assertTrue(latch.await(10, TimeUnit.SECONDS));
+
+                    for (CommunicationSpi spi : spis) {
+                        ConcurrentMap<UUID, GridTcpCommunicationClient> clients = U.field(spi, "clients");
+
+                        assertEquals(1, clients.size());
+
+                        final GridNioServer srv = U.field(spi, "nioSrvr");
+
+                        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                            @Override public boolean apply() {
+                                Collection sessions = U.field(srv, "sessions");
+
+                                return sessions.size() == 1;
+                            }
+                        }, 5000);
+
+                        Collection sessions = U.field(srv, "sessions");
+
+                        assertEquals(1, sessions.size());
+                    }
+
+                    assertEquals(expMsgs, lsnr.cntr.get());
+                }
+                finally {
+                    stopSpis();
+                }
+            }
+        }
+        finally {
+            stop.set(true);
+
+            if (loadFut != null)
+                loadFut.get();
+        }
+    }
+
+    /**
+     * @return SPI.
+     */
+    private CommunicationSpi createSpi() {
+        TcpCommunicationSpi spi = new TcpCommunicationSpi();
+
+        spi.setLocalAddress("127.0.0.1");
+        spi.setSharedMemoryPort(-1);
+        spi.setLocalPort(port++);
+        spi.setIdleConnectionTimeout(60_000);
+        spi.setConnectTimeout(10_000);
+
+        return spi;
+    }
+
+    /**
+     * @param lsnr Message listener.
+     * @throws Exception If failed.
+     */
+    private void startSpis(MessageListener lsnr) throws Exception {
+        spis.clear();
+        nodes.clear();
+        spiRsrcs.clear();
+
+        Map<ClusterNode, GridSpiTestContext> ctxs = new HashMap<>();
+
+        for (int i = 0; i < SPI_CNT; i++) {
+            CommunicationSpi<GridTcpCommunicationMessageAdapter> spi = createSpi();
+
+            GridTestUtils.setFieldValue(spi, "gridName", "grid-" + i);
+
+            GridTestResources rsrcs = new GridTestResources();
+
+            GridTestNode node = new GridTestNode(rsrcs.getNodeId());
+
+            node.order(i + 1);
+
+            GridSpiTestContext ctx = initSpiContext();
+
+            ctx.setLocalNode(node);
+
+            info(">>> Initialized context: nodeId=" + ctx.localNode().id());
+
+            spiRsrcs.add(rsrcs);
+
+            rsrcs.inject(spi);
+
+            spi.setListener(lsnr);
+
+            node.setAttributes(spi.getNodeAttributes());
+
+            nodes.add(node);
+
+            spi.spiStart(getTestGridName() + (i + 1));
+
+            spis.add(spi);
+
+            spi.onContextInitialized(ctx);
+
+            ctxs.put(node, ctx);
+        }
+
+        // For each context set remote nodes.
+        for (Map.Entry<ClusterNode, GridSpiTestContext> e : ctxs.entrySet()) {
+            for (ClusterNode n : nodes) {
+                if (!n.equals(e.getKey()))
+                    e.getValue().remoteNodes().add(n);
+            }
+        }
+    }
+
+    /**
+     * @param lsnr Message listener.
+     * @throws Exception If failed.
+     */
+    private void createSpis(MessageListener lsnr) throws Exception {
+        for (int i = 0; i < 3; i++) {
+            try {
+                startSpis(lsnr);
+
+                break;
+            }
+            catch (GridException e) {
+                if (e.hasCause(BindException.class)) {
+                    if (i < 2) {
+                        info("Failed to start SPIs because of BindException, will retry after delay.");
+
+                        stopSpis();
+
+                        U.sleep(10_000);
+                    }
+                    else
+                        throw e;
+                }
+                else
+                    throw e;
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void stopSpis() throws Exception {
+        for (CommunicationSpi<GridTcpCommunicationMessageAdapter> spi : spis) {
+            spi.onContextDestroyed();
+
+            spi.setListener(null);
+
+            spi.spiStop();
+        }
+
+        for (GridTestResources rsrcs : spiRsrcs)
+            rsrcs.stopThreads();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
index 833c20d..ba34ddd 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConfigSelfTest.java
@@ -37,5 +37,9 @@ public class GridTcpCommunicationSpiConfigSelfTest extends GridSpiAbstractConfig
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "bufferSizeRatio", 0);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectTimeout", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "maxConnectTimeout", -1);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "socketWriteTimeout", -1);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "ackSendThreshold", 0);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "ackSendThreshold", -1);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "unacknowledgedMessagesBufferSize", -1);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index c49f104..0d650fb 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -9,27 +9,27 @@
 
 package org.apache.ignite.spi.communication.tcp;
 
-import mx4j.tools.adaptor.http.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
 import org.apache.ignite.spi.communication.*;
 import org.gridgain.grid.util.direct.*;
+import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.nio.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.*;
-import org.gridgain.testframework.config.*;
 import org.gridgain.testframework.junits.*;
 import org.gridgain.testframework.junits.spi.*;
 import org.jdk8.backport.*;
 
-import javax.management.*;
 import java.util.*;
 import java.util.Map.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 
+import static org.gridgain.grid.kernal.GridNodeAttributes.*;
+
 /**
  * Class for multithreaded {@link TcpCommunicationSpi} test.
  */
@@ -60,9 +60,6 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
     /** Initialized nodes */
     private static final List<ClusterNode> nodes = new ArrayList<>();
 
-    /** */
-    private static final ObjectName mBeanName;
-
     /** Flag indicating if listener should reject messages. */
     private static boolean reject;
 
@@ -72,13 +69,6 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
                 return new GridTestMessage();
             }
         }, GridTestMessage.DIRECT_TYPE);
-
-        try {
-            mBeanName = new ObjectName("mbeanAdaptor:protocol=HTTP");
-        }
-        catch (MalformedObjectNameException e) {
-            throw new GridRuntimeException(e);
-        }
     }
 
     /**
@@ -168,8 +158,7 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
 
         assertEquals("Invalid listener count", getSpiCount(), lsnrs.size());
 
-        final ConcurrentMap<UUID, ConcurrentLinkedDeque8<GridTestMessage>> msgs =
-            new ConcurrentHashMap<>();
+        final ConcurrentMap<UUID, ConcurrentLinkedDeque8<GridTestMessage>> msgs = new ConcurrentHashMap<>();
 
         final int iterationCnt = 5000;
 
@@ -331,6 +320,28 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
         run.set(false);
 
         fut2.get();
+
+        // Wait when all messages are acknowledged to do not break next tests' logic.
+        for (CommunicationSpi<GridTcpCommunicationMessageAdapter> spi : spis.values()) {
+            GridNioServer srv = U.field(spi, "nioSrvr");
+
+            Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
+
+            for (GridNioSession ses : sessions) {
+                final GridNioRecoveryDescriptor snd = ses.recoveryDescriptor();
+
+                if (snd != null) {
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override public boolean apply() {
+                            return snd.messagesFutures().isEmpty();
+                        }
+                    }, 10_000);
+
+                    assertEquals("Unexpected messages: " + snd.messagesFutures(), 0,
+                        snd.messagesFutures().size());
+                }
+            }
+        }
     }
 
     /**
@@ -415,6 +426,8 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
 
     /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
+        U.setWorkDirectory(null, U.getGridGainHome());
+
         spis.clear();
         nodes.clear();
         spiRsrcs.clear();
@@ -425,10 +438,14 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
         for (int i = 0; i < getSpiCount(); i++) {
             CommunicationSpi<GridTcpCommunicationMessageAdapter> spi = newCommunicationSpi();
 
-            GridTestResources rsrcs = new GridTestResources(getMBeanServer(i));
+            GridTestUtils.setFieldValue(spi, "gridName", "grid-" + i);
+
+            GridTestResources rsrcs = new GridTestResources();
 
             GridTestNode node = new GridTestNode(rsrcs.getNodeId());
 
+            node.order(i);
+
             GridSpiTestContext ctx = initSpiContext();
 
             ctx.setLocalNode(node);
@@ -448,6 +465,7 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
             info("Lsnrs: " + lsnrs);
 
             node.setAttributes(spi.getNodeAttributes());
+            node.setAttribute(ATTR_MACS, F.concat(U.allLocalMACs(), ", "));
 
             nodes.add(node);
 
@@ -491,40 +509,19 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
         }
     }
 
-    /**
-     * @param idx Node index.
-     * @return Configured MBean server.
-     * @throws Exception If failed.
-     */
-    private MBeanServer getMBeanServer(int idx) throws Exception {
-        HttpAdaptor mbeanAdaptor = new HttpAdaptor();
-
-        MBeanServer mbeanSrv = MBeanServerFactory.createMBeanServer();
-
-        mbeanAdaptor.setPort(
-            Integer.valueOf(GridTestProperties.getProperty("comm.mbeanserver.selftest.baseport")) + idx);
-
-        mbeanSrv.registerMBean(mbeanAdaptor, mBeanName);
-
-        mbeanAdaptor.start();
-
-        return mbeanSrv;
-    }
-
     /** {@inheritDoc} */
     @Override protected void afterTestsStopped() throws Exception {
         for (CommunicationSpi<GridTcpCommunicationMessageAdapter> spi : spis.values()) {
+            spi.onContextDestroyed();
+
             spi.setListener(null);
 
             spi.spiStop();
         }
 
-        for (GridTestResources rsrcs : spiRsrcs) {
+        for (GridTestResources rsrcs : spiRsrcs)
             rsrcs.stopThreads();
 
-            rsrcs.getMBeanServer().unregisterMBean(mBeanName);
-        }
-
         lsnrs.clear();
         spiRsrcs.clear();
         spis.clear();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedShmemTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedShmemTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedShmemTest.java
index 98e614e..70334ee 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedShmemTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedShmemTest.java
@@ -15,6 +15,6 @@ package org.apache.ignite.spi.communication.tcp;
 public class GridTcpCommunicationSpiMultithreadedShmemTest extends GridTcpCommunicationSpiMultithreadedSelfTest {
     /** */
     public GridTcpCommunicationSpiMultithreadedShmemTest() {
-        super(true);
+        super(false);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
new file mode 100644
index 0000000..b20e50e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
@@ -0,0 +1,426 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.spi.communication.*;
+import org.eclipse.jetty.util.*;
+import org.gridgain.grid.*;
+import org.gridgain.grid.util.direct.*;
+import org.gridgain.grid.util.lang.*;
+import org.gridgain.grid.util.nio.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.gridgain.testframework.*;
+import org.gridgain.testframework.junits.*;
+import org.gridgain.testframework.junits.spi.*;
+
+import java.net.*;
+import java.util.*;
+import java.util.concurrent.atomic.*;
+
+/**
+ *
+ */
+@GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
+public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationSpi> extends GridSpiAbstractTest<T> {
+    /** */
+    private static final Collection<GridTestResources> spiRsrcs = new ArrayList<>();
+
+    /** */
+    protected static final List<TcpCommunicationSpi> spis = new ArrayList<>();
+
+    /** */
+    protected static final List<ClusterNode> nodes = new ArrayList<>();
+
+    /** */
+    private static final int SPI_CNT = 2;
+
+    /**
+     *
+     */
+    static {
+        GridTcpCommunicationMessageFactory.registerCustom(new GridTcpCommunicationMessageProducer() {
+            @Override
+            public GridTcpCommunicationMessageAdapter create(byte type) {
+                return new GridTestMessage();
+            }
+        }, GridTestMessage.DIRECT_TYPE);
+    }
+
+    /**
+     * Disable SPI auto-start.
+     */
+    public GridTcpCommunicationSpiRecoveryAckSelfTest() {
+        super(false);
+    }
+
+    /** */
+    @SuppressWarnings({"deprecation"})
+    private class TestListener implements CommunicationListener<GridTcpCommunicationMessageAdapter> {
+        /** */
+        private ConcurrentHashSet<Long> msgIds = new ConcurrentHashSet<>();
+
+        /** */
+        private AtomicInteger rcvCnt = new AtomicInteger();
+
+        /** {@inheritDoc} */
+        @Override public void onMessage(UUID nodeId, GridTcpCommunicationMessageAdapter msg, IgniteRunnable msgC) {
+            info("Test listener received message: " + msg);
+
+            assertTrue("Unexpected message: " + msg, msg instanceof GridTestMessage);
+
+            GridTestMessage msg0 = (GridTestMessage)msg;
+
+            assertTrue("Duplicated message received: " + msg0, msgIds.add(msg0.getMsgId()));
+
+            rcvCnt.incrementAndGet();
+
+            msgC.run();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onDisconnected(UUID nodeId) {
+            // No-op.
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAckOnIdle() throws Exception {
+        checkAck(10, 2000, 9);
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testAckOnCount() throws Exception {
+        checkAck(10, 60_000, 10);
+    }
+
+    /**
+     * @param ackCnt Recovery acknowledgement count.
+     * @param idleTimeout Idle connection timeout.
+     * @param msgPerIter Messages per iteration.
+     * @throws Exception If failed.
+     */
+    private void checkAck(int ackCnt, int idleTimeout, int msgPerIter) throws Exception {
+        createSpis(ackCnt, idleTimeout, TcpCommunicationSpi.DFLT_MSG_QUEUE_LIMIT);
+
+        try {
+            TcpCommunicationSpi spi0 = spis.get(0);
+            TcpCommunicationSpi spi1 = spis.get(1);
+
+            ClusterNode node0 = nodes.get(0);
+            ClusterNode node1 = nodes.get(1);
+
+            int msgId = 0;
+
+            int expMsgs = 0;
+
+            for (int i = 0; i < 5; i++) {
+                info("Iteration: " + i);
+
+                for (int j = 0; j < msgPerIter; j++) {
+                    spi0.sendMessage(node1, new GridTestMessage(node0.id(), ++msgId, 0));
+
+                    spi1.sendMessage(node0, new GridTestMessage(node1.id(), ++msgId, 0));
+                }
+
+                expMsgs += msgPerIter;
+
+                for (TcpCommunicationSpi spi : spis) {
+                    GridNioServer srv = U.field(spi, "nioSrvr");
+
+                    Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
+
+                    assertFalse(sessions.isEmpty());
+
+                    boolean found = false;
+
+                    for (GridNioSession ses : sessions) {
+                        final GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+
+                        if (recoveryDesc != null) {
+                            found = true;
+
+                            GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                                @Override public boolean apply() {
+                                    return recoveryDesc.messagesFutures().isEmpty();
+                                }
+                            }, 10_000);
+
+                            assertEquals("Unexpected messages: " + recoveryDesc.messagesFutures(), 0,
+                                recoveryDesc.messagesFutures().size());
+
+                            break;
+                        }
+                    }
+
+                    assertTrue(found);
+                }
+
+                final int expMsgs0 = expMsgs;
+
+                for (TcpCommunicationSpi spi : spis) {
+                    final TestListener lsnr = (TestListener)spi.getListener();
+
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override
+                        public boolean apply() {
+                            return lsnr.rcvCnt.get() >= expMsgs0;
+                        }
+                    }, 5000);
+
+                    assertEquals(expMsgs, lsnr.rcvCnt.get());
+                }
+            }
+        }
+        finally {
+            stopSpis();
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testQueueOverflow() throws Exception {
+        for (int i = 0; i < 3; i++) {
+            try {
+                startSpis(5, 60_000, 10);
+
+                checkOverflow();
+
+                break;
+            }
+            catch (GridException e) {
+                if (e.hasCause(BindException.class)) {
+                    if (i < 2) {
+                        info("Got exception caused by BindException, will retry after delay: " + e);
+
+                        stopSpis();
+
+                        U.sleep(10_000);
+                    }
+                    else
+                        throw e;
+                }
+                else
+                    throw e;
+            }
+            finally {
+                stopSpis();
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void checkOverflow() throws Exception {
+        TcpCommunicationSpi spi0 = spis.get(0);
+        TcpCommunicationSpi spi1 = spis.get(1);
+
+        ClusterNode node0 = nodes.get(0);
+        ClusterNode node1 = nodes.get(1);
+
+        final GridNioServer srv1 = U.field(spi1, "nioSrvr");
+
+        int msgId = 0;
+
+        // Send message to establish connection.
+        spi0.sendMessage(node1, new GridTestMessage(node0.id(), ++msgId, 0));
+
+        // Prevent node1 from send
+        GridTestUtils.setFieldValue(srv1, "skipWrite", true);
+
+        final GridNioSession ses0 = communicationSession(spi0);
+
+        for (int i = 0; i < 150; i++)
+            spi0.sendMessage(node1, new GridTestMessage(node0.id(), ++msgId, 0));
+
+        // Wait when session is closed because of queue overflow.
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override public boolean apply() {
+                return ses0.closeTime() != 0;
+            }
+        }, 5000);
+
+        assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
+
+        GridTestUtils.setFieldValue(srv1, "skipWrite", false);
+
+        for (int i = 0; i < 100; i++)
+            spi0.sendMessage(node1, new GridTestMessage(node0.id(), ++msgId, 0));
+
+        final int expMsgs = 251;
+
+        final TestListener lsnr = (TestListener)spi1.getListener();
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override
+            public boolean apply() {
+                return lsnr.rcvCnt.get() >= expMsgs;
+            }
+        }, 5000);
+
+        assertEquals(expMsgs, lsnr.rcvCnt.get());
+    }
+
+    /**
+     * @param spi SPI.
+     * @return Session.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("unchecked")
+    private GridNioSession communicationSession(TcpCommunicationSpi spi) throws Exception {
+        final GridNioServer srv = U.field(spi, "nioSrvr");
+
+        GridTestUtils.waitForCondition(new GridAbsPredicate() {
+            @Override
+            public boolean apply() {
+                Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
+
+                return !sessions.isEmpty();
+            }
+        }, 5000);
+
+        Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
+
+        assertEquals(1, sessions.size());
+
+        return sessions.iterator().next();
+    }
+
+    /**
+     * @param ackCnt Recovery acknowledgement count.
+     * @param idleTimeout Idle connection timeout.
+     * @param queueLimit Message queue limit.
+     * @return SPI instance.
+     */
+    protected TcpCommunicationSpi getSpi(int ackCnt, int idleTimeout, int queueLimit) {
+        TcpCommunicationSpi spi = new TcpCommunicationSpi();
+
+        spi.setSharedMemoryPort(-1);
+        spi.setLocalPort(GridTestUtils.getNextCommPort(getClass()));
+        spi.setIdleConnectionTimeout(idleTimeout);
+        spi.setTcpNoDelay(true);
+        spi.setAckSendThreshold(ackCnt);
+        spi.setMessageQueueLimit(queueLimit);
+
+        return spi;
+    }
+
+    /**
+     * @param ackCnt Recovery acknowledgement count.
+     * @param idleTimeout Idle connection timeout.
+     * @param queueLimit Message queue limit.
+     * @throws Exception If failed.
+     */
+    private void startSpis(int ackCnt, int idleTimeout, int queueLimit) throws Exception {
+        spis.clear();
+        nodes.clear();
+        spiRsrcs.clear();
+
+        Map<ClusterNode, GridSpiTestContext> ctxs = new HashMap<>();
+
+        for (int i = 0; i < SPI_CNT; i++) {
+            TcpCommunicationSpi spi = getSpi(ackCnt, idleTimeout, queueLimit);
+
+            GridTestUtils.setFieldValue(spi, "gridName", "grid-" + i);
+
+            GridTestResources rsrcs = new GridTestResources();
+
+            GridTestNode node = new GridTestNode(rsrcs.getNodeId());
+
+            GridSpiTestContext ctx = initSpiContext();
+
+            ctx.setLocalNode(node);
+
+            spiRsrcs.add(rsrcs);
+
+            rsrcs.inject(spi);
+
+            spi.setListener(new TestListener());
+
+            node.setAttributes(spi.getNodeAttributes());
+
+            nodes.add(node);
+
+            spi.spiStart(getTestGridName() + (i + 1));
+
+            spis.add(spi);
+
+            spi.onContextInitialized(ctx);
+
+            ctxs.put(node, ctx);
+        }
+
+        // For each context set remote nodes.
+        for (Map.Entry<ClusterNode, GridSpiTestContext> e : ctxs.entrySet()) {
+            for (ClusterNode n : nodes) {
+                if (!n.equals(e.getKey()))
+                    e.getValue().remoteNodes().add(n);
+            }
+        }
+    }
+
+    /**
+     * @param ackCnt Recovery acknowledgement count.
+     * @param idleTimeout Idle connection timeout.
+     * @param queueLimit Message queue limit.
+     * @throws Exception If failed.
+     */
+    private void createSpis(int ackCnt, int idleTimeout, int queueLimit) throws Exception {
+        for (int i = 0; i < 3; i++) {
+            try {
+                startSpis(ackCnt, idleTimeout, queueLimit);
+
+                break;
+            }
+            catch (GridException e) {
+                if (e.hasCause(BindException.class)) {
+                    if (i < 2) {
+                        info("Failed to start SPIs because of BindException, will retry after delay.");
+
+                        stopSpis();
+
+                        U.sleep(10_000);
+                    }
+                    else
+                        throw e;
+                }
+                else
+                    throw e;
+            }
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    private void stopSpis() throws Exception {
+        for (CommunicationSpi<GridTcpCommunicationMessageAdapter> spi : spis) {
+            spi.onContextDestroyed();
+
+            spi.setListener(null);
+
+            spi.spiStop();
+        }
+
+        for (GridTestResources rsrcs : spiRsrcs)
+            rsrcs.stopThreads();
+
+        spis.clear();
+        nodes.clear();
+        spiRsrcs.clear();
+    }
+}


[7/7] incubator-ignite git commit: # Merge remote-tracking branch 'remotes/origin/master' into ignite-21

Posted by sb...@apache.org.
# Merge remote-tracking branch 'remotes/origin/master' into ignite-21


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

Branch: refs/heads/master
Commit: c67dcde9f967349dec79b66bc19babb50e2d04f0
Parents: 4a73539
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 12 11:14:01 2014 +0400
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 12 11:14:01 2014 +0400

----------------------------------------------------------------------
 .../spi/communication/tcp/TcpCommunicationSpi.java      | 12 ++++++------
 .../grid/util/nio/GridNioRecoveryDescriptor.java        |  2 +-
 .../GridAbstractCommunicationSelfTest.java              |  2 +-
 ...ridTcpCommunicationSpiConcurrentConnectSelfTest.java |  4 ++--
 .../tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java |  6 +++---
 .../tcp/GridTcpCommunicationSpiRecoverySelfTest.java    | 10 +++++-----
 6 files changed, 18 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c67dcde9/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 2c7f8be..9ba7c45 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -211,7 +211,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     public static final boolean DFLT_TCP_NODELAY = true;
 
     /** Default received messages threshold for sending ack. */
-    public static final int DFLT_ACK_SND_THRESHOLD = 512;
+    public static final int DFLT_ACK_SND_THRESHOLD = 16;
 
     /** Default socket write timeout. */
     public static final long DFLT_SOCK_WRITE_TIMEOUT = GridNioServer.DFLT_SES_WRITE_TIMEOUT;
@@ -586,7 +586,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                                     fut.onDone(client);
                                 }
-                                catch (GridException | IOException e) {
+                                catch (IgniteCheckedException | IOException e) {
                                     if (log.isDebugEnabled())
                                         log.debug("Failed to send recovery handshake " +
                                             "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
@@ -1780,7 +1780,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         ClusterNode node0 = getSpiContext().node(node.id());
 
                         if (node0 == null)
-                            throw new GridException("Failed to send message to remote node " +
+                            throw new IgniteCheckedException("Failed to send message to remote node " +
                                 "(node has left the grid): " + node.id());
                     }
                 }
@@ -2199,7 +2199,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * @param recovery Recovery descriptor if use recovery handshake, otherwise {@code null}.
      * @param rmtNodeId Remote node.
      * @param timeout Timeout for handshake.
-     * @throws GridException If handshake failed or wasn't completed withing timeout.
+     * @throws IgniteCheckedException If handshake failed or wasn't completed withing timeout.
      * @return Handshake response.
      */
     @SuppressWarnings("ThrowFromFinallyBlock")
@@ -2280,7 +2280,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             int read = ch.read(buf);
 
                             if (read == -1)
-                                throw new GridException("Failed to read remote node recovery handshake " +
+                                throw new IgniteCheckedException("Failed to read remote node recovery handshake " +
                                     "(connection closed).");
 
                             i += read;
@@ -2833,7 +2833,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                     client.release();
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     if (recoveryDesc.nodeAlive(getSpiContext().node(node.id()))) {
                         if (log.isDebugEnabled())
                             log.debug("Recovery reconnect failed, will retry " +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c67dcde9/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioRecoveryDescriptor.java
index 846a388..6ed15a9 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioRecoveryDescriptor.java
@@ -170,7 +170,7 @@ public class GridNioRecoveryDescriptor {
 
             assert fut != null;
 
-            ((GridNioFutureImpl)fut).onDone();
+            assert fut.isDone();
 
             acked++;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c67dcde9/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
index 07b5059..2e2355f 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
@@ -238,7 +238,7 @@ public abstract class GridAbstractCommunicationSelfTest<T extends CommunicationS
 
                 break;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (e.hasCause(BindException.class)) {
                     if (i < 2) {
                         info("Failed to start SPIs because of BindException, will retry after delay.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c67dcde9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
index ffe5d57..8a28058 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
@@ -9,11 +9,11 @@
 
 package org.apache.ignite.spi.communication.tcp;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.communication.*;
 import org.eclipse.jetty.util.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.nio.*;
@@ -361,7 +361,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
 
                 break;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (e.hasCause(BindException.class)) {
                     if (i < 2) {
                         info("Failed to start SPIs because of BindException, will retry after delay.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c67dcde9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
index b20e50e..e9eef6a 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryAckSelfTest.java
@@ -9,11 +9,11 @@
 
 package org.apache.ignite.spi.communication.tcp;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.communication.*;
 import org.eclipse.jetty.util.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.nio.*;
@@ -201,7 +201,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
 
                 break;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (e.hasCause(BindException.class)) {
                     if (i < 2) {
                         info("Got exception caused by BindException, will retry after delay: " + e);
@@ -386,7 +386,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
 
                 break;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (e.hasCause(BindException.class)) {
                     if (i < 2) {
                         info("Failed to start SPIs because of BindException, will retry after delay.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c67dcde9/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
index fb1c29f..fadac25 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
@@ -9,12 +9,12 @@
 
 package org.apache.ignite.spi.communication.tcp;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.communication.*;
 import org.eclipse.jetty.util.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.nio.*;
@@ -314,7 +314,7 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     assertEquals(expMsgs, lsnr1.rcvCnt.get());
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     if (e.hasCause(BindException.class)) {
                         errCnt++;
 
@@ -436,7 +436,7 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
                     assertEquals(expMsgs0, lsnr0.rcvCnt.get());
                     assertEquals(expMsgs1, lsnr1.rcvCnt.get());
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     if (e.hasCause(BindException.class)) {
                         errCnt++;
 
@@ -535,7 +535,7 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     assertEquals(expMsgs, lsnr1.rcvCnt.get());
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     if (e.hasCause(BindException.class)) {
                         errCnt++;
 
@@ -672,7 +672,7 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                 break;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (e.hasCause(BindException.class)) {
                     if (i < 2) {
                         info("Failed to start SPIs because of BindException, will retry after delay.");


[5/7] incubator-ignite git commit: # Merge remote-tracking branch 'remotes/origin/master' into ignite-21

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index 1911b5b,0b3330a..2c7f8be
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@@ -1758,35 -1428,22 +1758,35 @@@ public class TcpCommunicationSpi extend
              GridCommunicationClient client = null;
  
              try {
 -                client = reserveClient(node);
 +                boolean retry;
 +
 +                do {
 +                    client = reserveClient(node);
  
 -                UUID nodeId = null;
 +                    UUID nodeId = null;
  
 -                if (!client.async() && !getSpiContext().localNode().version().equals(node.version()))
 -                    nodeId = node.id();
 +                    if (!client.async() && !getSpiContext().localNode().version().equals(node.version()))
 +                        nodeId = node.id();
  
 -                client.sendMessage(nodeId, msg);
 +                    retry = client.sendMessage(nodeId, msg);
  
 -                client.release();
 +                    client.release();
  
 -                client = null;
 +                    client = null;
 +
 +                    if (!retry)
 +                        sentMsgsCnt.increment();
 +                    else {
 +                        ClusterNode node0 = getSpiContext().node(node.id());
  
 -                sentMsgsCnt.increment();
 +                        if (node0 == null)
 +                            throw new GridException("Failed to send message to remote node " +
 +                                "(node has left the grid): " + node.id());
 +                    }
 +                }
 +                while (retry);
              }
-             catch (GridException e) {
+             catch (IgniteCheckedException e) {
                  throw new IgniteSpiException("Failed to send message to remote node: " + node, e);
              }
              finally {
@@@ -2196,19 -1812,12 +2196,19 @@@
       * Performs handshake in timeout-safe way.
       *
       * @param client Client.
 +     * @param recovery Recovery descriptor if use recovery handshake, otherwise {@code null}.
       * @param rmtNodeId Remote node.
       * @param timeout Timeout for handshake.
 -     * @throws IgniteCheckedException If handshake failed or wasn't completed withing timeout.
 +     * @throws GridException If handshake failed or wasn't completed withing timeout.
 +     * @return Handshake response.
       */
      @SuppressWarnings("ThrowFromFinallyBlock")
 -    private <T> void safeHandshake(T client, UUID rmtNodeId, long timeout) throws IgniteCheckedException {
 +    private <T> long safeHandshake(
 +        T client,
 +        @Nullable GridNioRecoveryDescriptor recovery,
 +        UUID rmtNodeId,
 +        long timeout
-     ) throws GridException {
++    ) throws IgniteCheckedException {
          HandshakeTimeoutObject<T> obj = new HandshakeTimeoutObject<>(client, U.currentTimeMillis() + timeout);
  
          sockTimeoutWorker.addTimeoutObject(obj);
@@@ -2373,56 -1901,8 +2373,56 @@@
          return S.toString(TcpCommunicationSpi.class, this);
      }
  
 +    /**
 +     *
 +     */
 +    private static class ClientKey {
 +        /** */
 +        private UUID nodeId;
 +
 +        /** */
 +        private long order;
 +
 +        /**
 +         * @param nodeId Node ID.
 +         * @param order Node order.
 +         */
 +        private ClientKey(UUID nodeId, long order) {
 +            this.nodeId = nodeId;
 +            this.order = order;
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public boolean equals(Object obj) {
 +            if (this == obj)
 +                return true;
 +
 +            if (obj == null || getClass() != obj.getClass())
 +                return false;
 +
 +            ClientKey other = (ClientKey)obj;
 +
 +            return order == other.order && nodeId.equals(other.nodeId);
 +
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public int hashCode() {
 +            int res = nodeId.hashCode();
 +
 +            res = 31 * res + (int)(order ^ (order >>> 32));
 +
 +            return res;
 +        }
 +
 +        /** {@inheritDoc} */
 +        @Override public String toString() {
 +            return S.toString(ClientKey.class, this);
 +        }
 +    }
 +
      /** Internal exception class for proper timeout handling. */
-     private static class HandshakeTimeoutException extends GridException {
+     private static class HandshakeTimeoutException extends IgniteCheckedException {
          /** */
          private static final long serialVersionUID = 0L;
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpiAdapter.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/gridgain/grid/util/nio/GridCommunicationClient.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/gridgain/grid/util/nio/GridCommunicationClient.java
index 5b0db53,6404002..6c9631e
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridCommunicationClient.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridCommunicationClient.java
@@@ -86,10 -86,9 +86,10 @@@ public interface GridCommunicationClien
      /**
       * @param nodeId Node ID (provided only if versions of local and remote nodes are different).
       * @param msg Message to send.
-      * @throws GridException If failed.
+      * @throws IgniteCheckedException If failed.
 +     * @return {@code True} if should try to resend message.
       */
-     boolean sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg) throws GridException;
 -    void sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg) throws IgniteCheckedException;
++    boolean sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg) throws IgniteCheckedException;
  
      /**
       * @param timeout Timeout.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFinishedFuture.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFuture.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFuture.java
index a3ab1ef,1754db7..19de132
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFuture.java
@@@ -27,12 -28,12 +28,12 @@@ public interface GridNioFuture<R> 
       * returns operation result.
       *
       * @return Operation result.
 -     * @throws GridInterruptedException Subclass of {@link IgniteCheckedException} thrown if the wait was interrupted.
 -     * @throws org.apache.ignite.lang.IgniteFutureCancelledException Subclass of {@link IgniteCheckedException} throws if operation was cancelled.
 +     * @throws GridInterruptedException Subclass of {@link GridException} thrown if the wait was interrupted.
 +     * @throws IgniteFutureCancelledException Subclass of {@link GridException} throws if operation was cancelled.
-      * @throws GridException If operation failed.
+      * @throws IgniteCheckedException If operation failed.
       * @throws IOException If IOException occurred while performing operation.
       */
-     public R get() throws IOException, GridException;
+     public R get() throws IOException, IgniteCheckedException;
  
      /**
       * Synchronously waits for completion of the operation for
@@@ -41,13 -42,13 +42,13 @@@
       *
       * @param timeout The maximum time to wait in milliseconds.
       * @return Operation result.
 -     * @throws GridInterruptedException Subclass of {@link IgniteCheckedException} thrown if the wait was interrupted.
 -     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException Subclass of {@link IgniteCheckedException} thrown if the wait was timed out.
 -     * @throws org.apache.ignite.lang.IgniteFutureCancelledException Subclass of {@link IgniteCheckedException} throws if operation was cancelled.
 +     * @throws GridInterruptedException Subclass of {@link GridException} thrown if the wait was interrupted.
 +     * @throws IgniteFutureTimeoutException Subclass of {@link GridException} thrown if the wait was timed out.
 +     * @throws IgniteFutureCancelledException Subclass of {@link GridException} throws if operation was cancelled.
-      * @throws GridException If operation failed.
+      * @throws IgniteCheckedException If operation failed.
       * @throws IOException If IOException occurred while performing operation.
       */
-     public R get(long timeout) throws IOException, GridException;
+     public R get(long timeout) throws IOException, IgniteCheckedException;
  
      /**
       * Synchronously waits for completion of the operation for
@@@ -56,13 -57,13 +57,13 @@@
       * @param timeout The maximum time to wait.
       * @param unit The time unit of the {@code timeout} argument.
       * @return Operation result.
 -     * @throws GridInterruptedException Subclass of {@link IgniteCheckedException} thrown if the wait was interrupted.
 -     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException Subclass of {@link IgniteCheckedException} thrown if the wait was timed out.
 -     * @throws org.apache.ignite.lang.IgniteFutureCancelledException Subclass of {@link IgniteCheckedException} throws if operation was cancelled.
 +     * @throws GridInterruptedException Subclass of {@link GridException} thrown if the wait was interrupted.
 +     * @throws IgniteFutureTimeoutException Subclass of {@link GridException} thrown if the wait was timed out.
 +     * @throws IgniteFutureCancelledException Subclass of {@link GridException} throws if operation was cancelled.
-      * @throws GridException If operation failed.
+      * @throws IgniteCheckedException If operation failed.
       * @throws IOException If IOException occurred while performing operation.
       */
-     public R get(long timeout, TimeUnit unit) throws IOException, GridException;
+     public R get(long timeout, TimeUnit unit) throws IOException, IgniteCheckedException;
  
      /**
       * Cancels this future.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFutureImpl.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFutureImpl.java
index 7ab2e14,6c5a6bc..ee84796
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFutureImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFutureImpl.java
@@@ -118,10 -119,10 +119,10 @@@ public class GridNioFutureImpl<R> exten
       * @param nanosTimeout Timeout (nanoseconds).
       * @return Result.
       * @throws InterruptedException If interrupted.
 -     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException If timeout reached before computation completed.
 +     * @throws IgniteFutureTimeoutException If timeout reached before computation completed.
-      * @throws GridException If error occurred.
+      * @throws IgniteCheckedException If error occurred.
       */
-     @Nullable protected R get0(long nanosTimeout) throws InterruptedException, GridException {
+     @Nullable protected R get0(long nanosTimeout) throws InterruptedException, IgniteCheckedException {
          if (endTime == 0 && !tryAcquireSharedNanos(0, nanosTimeout))
              throw new IgniteFutureTimeoutException("Timeout was reached before computation completed.");
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioServer.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioSessionImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/gridgain/grid/util/nio/GridShmemCommunicationClient.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/gridgain/grid/util/nio/GridShmemCommunicationClient.java
index 505c788,8777405..501e7ee
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridShmemCommunicationClient.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridShmemCommunicationClient.java
@@@ -103,10 -103,10 +103,10 @@@ public class GridShmemCommunicationClie
      }
  
      /** {@inheritDoc} */
 -    @Override public synchronized void sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
 +    @Override public synchronized boolean sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
-         throws GridException {
+         throws IgniteCheckedException {
          if (closed())
-             throw new GridException("Communication client was closed: " + this);
+             throw new IgniteCheckedException("Communication client was closed: " + this);
  
          assert writeBuf.hasArray();
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpCommunicationClient.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpCommunicationClient.java
index fbca363,a20ea24..632ce35
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpCommunicationClient.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpCommunicationClient.java
@@@ -182,10 -182,10 +182,10 @@@ public class GridTcpCommunicationClien
      }
  
      /** {@inheritDoc} */
 -    @Override public void sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
 +    @Override public boolean sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
-         throws GridException {
+         throws IgniteCheckedException {
          if (closed())
-             throw new GridException("Client was closed: " + this);
+             throw new IgniteCheckedException("Client was closed: " + this);
  
          assert writeBuf.hasArray();
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpNioCommunicationClient.java
index 55997d3,49053d3..3d8668d
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpNioCommunicationClient.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridTcpNioCommunicationClient.java
@@@ -96,8 -98,8 +95,8 @@@ public class GridTcpNioCommunicationCli
      }
  
      /** {@inheritDoc} */
 -    @Override public void sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
 +    @Override public boolean sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg)
-         throws GridException {
+         throws IgniteCheckedException {
          // Node ID is never provided in asynchronous send mode.
          assert nodeId == null;
  
@@@ -108,23 -113,9 +107,23 @@@
                  fut.get();
              }
              catch (IOException e) {
 -                throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
 +                if (log.isDebugEnabled())
 +                    log.debug("Failed to send message [client=" + this + ", err=" +e + ']');
 +
 +                return true;
 +            }
-             catch (GridException e) {
++            catch (IgniteCheckedException e) {
 +                if (log.isDebugEnabled())
 +                    log.debug("Failed to send message [client=" + this + ", err=" +e + ']');
 +
 +                if (e.getCause() instanceof IOException)
 +                    return true;
 +                else
-                     throw new GridException("Failed to send message [client=" + this + ']', e);
++                    throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
              }
          }
 +
 +        return false;
      }
  
      /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
index 86d68a7,0dfb5fd..07b5059
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/GridAbstractCommunicationSelfTest.java
@@@ -9,13 -9,13 +9,13 @@@
  
  package org.apache.ignite.spi.communication;
  
 -import mx4j.tools.adaptor.http.*;
+ import org.apache.ignite.*;
  import org.apache.ignite.cluster.*;
  import org.apache.ignite.lang.*;
- import org.gridgain.grid.*;
  import org.gridgain.grid.util.direct.*;
 +import org.gridgain.grid.util.typedef.*;
 +import org.gridgain.grid.util.typedef.internal.*;
  import org.gridgain.testframework.*;
 -import org.gridgain.testframework.config.*;
  import org.gridgain.testframework.junits.*;
  import org.gridgain.testframework.junits.spi.*;
  

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index 0d650fb,80f2226..1889cd2
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@@ -9,12 -9,13 +9,13 @@@
  
  package org.apache.ignite.spi.communication.tcp;
  
 -import mx4j.tools.adaptor.http.*;
+ import org.apache.ignite.*;
  import org.apache.ignite.cluster.*;
  import org.apache.ignite.lang.*;
- import org.gridgain.grid.*;
  import org.apache.ignite.spi.communication.*;
+ import org.gridgain.grid.*;
  import org.gridgain.grid.util.direct.*;
 +import org.gridgain.grid.util.lang.*;
  import org.gridgain.grid.util.nio.*;
  import org.gridgain.grid.util.typedef.*;
  import org.gridgain.grid.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/test/java/org/gridgain/grid/kernal/managers/GridManagerStopSelfTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4a735398/modules/core/src/test/java/org/gridgain/grid/util/nio/impl/GridNioFilterChainSelfTest.java
----------------------------------------------------------------------


[4/7] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-21

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/origin/master' into ignite-21


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

Branch: refs/heads/master
Commit: ce518b187e6af490da9e67a91e3d861c0c9f270f
Parents: dae4b94 682e47f
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 10 13:52:44 2014 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 10 13:52:44 2014 +0300

----------------------------------------------------------------------
 .../ignite/spi/deployment/DeploymentSpi.java    |  2 +-
 .../grid/kernal/visor/VisorMultiNodeTask.java   | 31 +++++++---
 .../visor/node/VisorNodeDataCollectorJob.java   | 64 ++++++++++++++++----
 .../visor/node/VisorNodeDataCollectorTask.java  | 24 ++++----
 modules/core/src/test/config/tests.properties   |  2 +-
 ...loymentClassLoaderMultiThreadedSelfTest.java |  4 +-
 .../GridUriDeploymentFileProcessorSelfTest.java |  2 +-
 .../GridUriDeploymentMultiScannersSelfTest.java |  2 +-
 .../file/GridFileDeploymentSelfTest.java        | 24 ++++----
 .../GridFileDeploymentUndeploySelfTest.java     |  4 +-
 .../uri/scanners/ftp/GridFtpDeploymentTest.java |  2 +-
 .../http/GridHttpDeploymentSelfTest.java        |  2 +-
 12 files changed, 110 insertions(+), 53 deletions(-)
----------------------------------------------------------------------



[3/7] incubator-ignite git commit: # ignite-21

Posted by sb...@apache.org.
# ignite-21


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

Branch: refs/heads/master
Commit: dae4b9428cfe04fb2d0f78020e7f0ea29eb37c04
Parents: 20872dc
Author: sboikov <sb...@gridgain.com>
Authored: Wed Dec 10 10:41:56 2014 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Dec 10 13:22:50 2014 +0300

----------------------------------------------------------------------
 .../rest/protocols/tcp/GridMockNioSession.java  |   11 +
 .../communication/tcp/TcpCommunicationSpi.java  | 1151 +++++++++++++++---
 .../tcp/TcpCommunicationSpiMBean.java           |   52 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |    3 +-
 .../GridTcpCommunicationMessageAdapter.java     |    7 +
 .../GridTcpCommunicationMessageFactory.java     |   10 +-
 .../grid/util/nio/GridCommunicationClient.java  |    3 +-
 .../grid/util/nio/GridNioFinishedFuture.java    |    5 +
 .../gridgain/grid/util/nio/GridNioFuture.java   |   15 +-
 .../grid/util/nio/GridNioFutureImpl.java        |    7 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |  356 ++++++
 .../gridgain/grid/util/nio/GridNioServer.java   |  167 ++-
 .../gridgain/grid/util/nio/GridNioSession.java  |   10 +
 .../grid/util/nio/GridNioSessionImpl.java       |   10 +
 .../util/nio/GridSelectorNioSessionImpl.java    |   97 ++
 .../util/nio/GridShmemCommunicationClient.java  |    4 +-
 .../util/nio/GridTcpCommunicationClient.java    |    4 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   39 +-
 .../GridAbstractCommunicationSelfTest.java      |   85 +-
 .../spi/communication/GridTestMessage.java      |   15 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   26 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |  398 ++++++
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    4 +
 ...cpCommunicationSpiMultithreadedSelfTest.java |   77 +-
 ...pCommunicationSpiMultithreadedShmemTest.java |    2 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |  426 +++++++
 ...GridTcpCommunicationSpiRecoverySelfTest.java |  713 +++++++++++
 .../GridTcpCommunicationSpiShmemSelfTest.java   |    2 +-
 ...cpCommunicationSpiTcpNoDelayOffSelfTest.java |   20 +
 .../managers/GridManagerStopSelfTest.java       |    2 +
 .../grid/spi/GridSpiStartStopAbstractTest.java  |    2 +
 .../nio/impl/GridNioFilterChainSelfTest.java    |   10 +
 .../gridgain/testframework/GridTestNode.java    |   12 +-
 .../GridSpiCommunicationSelfTestSuite.java      |    8 +-
 34 files changed, 3434 insertions(+), 319 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/clients/src/test/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMockNioSession.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMockNioSession.java b/modules/clients/src/test/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMockNioSession.java
index 7071ed5..bf82f22 100644
--- a/modules/clients/src/test/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMockNioSession.java
+++ b/modules/clients/src/test/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMockNioSession.java
@@ -10,6 +10,7 @@ package org.gridgain.grid.kernal.processors.rest.protocols.tcp;
 
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.nio.*;
+import org.jetbrains.annotations.*;
 
 import java.net.*;
 
@@ -132,4 +133,14 @@ public class GridMockNioSession extends GridMetadataAwareAdapter implements Grid
     @Override public boolean readsPaused() {
         return false;
     }
+
+    /** {@inheritDoc} */
+    @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+        return null;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
index a80421e..1911b5b 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java
@@ -24,6 +24,7 @@ import org.gridgain.grid.kernal.managers.eventstorage.*;
 import org.apache.ignite.spi.communication.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.direct.*;
+import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.ipc.*;
 import org.gridgain.grid.util.ipc.shmem.*;
 import org.gridgain.grid.util.lang.*;
@@ -51,7 +52,7 @@ import static org.apache.ignite.events.IgniteEventType.*;
  * TCP/IP protocol and Java NIO to communicate with other nodes.
  * <p>
  * To enable communication with other nodes, this SPI adds {@link #ATTR_ADDRS}
- * and {@link #ATTR_PORT} local node attributes (see {@link org.apache.ignite.cluster.ClusterNode#attributes()}.
+ * and {@link #ATTR_PORT} local node attributes (see {@link ClusterNode#attributes()}.
  * <p>
  * At startup, this SPI tries to start listening to local port specified by
  * {@link #setLocalPort(int)} method. If local port is occupied, then SPI will
@@ -84,9 +85,7 @@ import static org.apache.ignite.events.IgniteEventType.*;
  * <li>Direct or heap buffer allocation for sending (see {@link #setDirectSendBuffer(boolean)})</li>
  * <li>Count of selectors and selector threads for NIO server (see {@link #setSelectorsCount(int)})</li>
  * <li>{@code TCP_NODELAY} socket option for sockets (see {@link #setTcpNoDelay(boolean)})</li>
- * <li>Async message sending (see {@link #setAsyncSend(boolean)})</li>
  * <li>Message queue limit (see {@link #setMessageQueueLimit(int)})</li>
- * <li>Dual socket connection (see {@link #setDualSocketConnection(boolean)})</li>
  * <li>Minimum buffered message count (see {@link #setMinimumBufferedMessageCount(int)})</li>
  * <li>Buffer size ratio (see {@link #setBufferSizeRatio(double)})</li>
  * <li>Connect timeout (see {@link #setConnectTimeout(long)})</li>
@@ -95,6 +94,9 @@ import static org.apache.ignite.events.IgniteEventType.*;
  * <li>Local port to accept shared memory connections (see {@link #setSharedMemoryPort(int)})</li>
  * <li>Socket receive buffer size (see {@link #setSocketReceiveBuffer(int)})</li>
  * <li>Socket send buffer size (see {@link #setSocketSendBuffer(int)})</li>
+ * <li>Socket write timeout (see {@link #setSocketWriteTimeout(long)})</li>
+ * <li>Number of received messages after which acknowledgment is sent (see {@link #setAckSendThreshold(int)})</li>
+ * <li>Maximum number of unacknowledged messages (see {@link #setUnacknowledgedMessagesBufferSize(int)})</li>
  * </ul>
  * <h2 class="header">Java Example</h2>
  * GridTcpCommunicationSpi is used by default and should be explicitly configured
@@ -131,7 +133,7 @@ import static org.apache.ignite.events.IgniteEventType.*;
  * <img src="http://www.gridgain.com/images/spring-small.png">
  * <br>
  * For information about Spring framework visit <a href="http://www.springframework.org/">www.springframework.org</a>
- * @see org.apache.ignite.spi.communication.CommunicationSpi
+ * @see CommunicationSpi
  */
 @IgniteSpiMultipleInstancesSupport(true)
 @IgniteSpiConsistencyChecked(optional = false)
@@ -187,9 +189,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Default message queue limit per connection (for incoming and outgoing . */
     public static final int DFLT_MSG_QUEUE_LIMIT = GridNioServer.DFLT_SEND_QUEUE_LIMIT;
 
-    /** Default value for dualSocketConnection flag. */
-    public static final boolean DFLT_DUAL_SOCKET_CONNECTION = false;
-
     /**
      * Default count of selectors for TCP server equals to
      * {@code "Math.min(4, Runtime.getRuntime().availableProcessors())"}.
@@ -211,6 +210,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Default value for {@code TCP_NODELAY} socket option (value is <tt>true</tt>). */
     public static final boolean DFLT_TCP_NODELAY = true;
 
+    /** Default received messages threshold for sending ack. */
+    public static final int DFLT_ACK_SND_THRESHOLD = 512;
+
+    /** Default socket write timeout. */
+    public static final long DFLT_SOCK_WRITE_TIMEOUT = GridNioServer.DFLT_SES_WRITE_TIMEOUT;
+
     /** No-op runnable. */
     private static final IgniteRunnable NOOP = new IgniteRunnable() {
         @Override public void run() {
@@ -221,9 +226,27 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Node ID message type. */
     public static final byte NODE_ID_MSG_TYPE = -1;
 
+    /** */
+    public static final byte RECOVERY_LAST_ID_MSG_TYPE = -2;
+
+    /** */
+    public static final byte HANDSHAKE_MSG_TYPE = -3;
+
     /** Server listener. */
     private final GridNioServerListener<GridTcpCommunicationMessageAdapter> srvLsnr =
         new GridNioServerListenerAdapter<GridTcpCommunicationMessageAdapter>() {
+            @Override public void onSessionWriteTimeout(GridNioSession ses) {
+                LT.warn(log, null, "Communication SPI Session write timed out (consider increasing " +
+                    "'socketWriteTimeout' " + "configuration property) [remoteAddr=" + ses.remoteAddress() +
+                    ", writeTimeout=" + sockWriteTimeout + ']');
+
+                if (log.isDebugEnabled())
+                    log.debug("Closing communication SPI session on write timeout [remoteAddr=" + ses.remoteAddress() +
+                        ", writeTimeout=" + sockWriteTimeout + ']');
+
+                ses.close();
+            }
+
             @Override public void onConnected(GridNioSession ses) {
                 if (ses.accepted()) {
                     if (log.isDebugEnabled())
@@ -231,8 +254,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                     ses.send(nodeIdMsg);
                 }
-                else
-                    assert asyncSnd;
             }
 
             @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
@@ -243,9 +264,28 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                     if (rmv instanceof GridTcpNioCommunicationClient &&
                         ((GridTcpNioCommunicationClient)rmv).session() == ses &&
-                        clients.remove(id, rmv))
+                        clients.remove(id, rmv)) {
                         rmv.forceClose();
 
+                        if (!stopping) {
+                            GridNioRecoveryDescriptor recoveryData = ses.recoveryDescriptor();
+
+                            if (recoveryData != null) {
+                                if (recoveryData.nodeAlive(getSpiContext().node(id))) {
+                                    if (!recoveryData.messagesFutures().isEmpty()) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Session was closed but there are unacknowledged messages, " +
+                                                "will try to reconnect [rmtNode=" + recoveryData.node().id() + ']');
+
+                                        recoveryWorker.addReconnectRequest(recoveryData);
+                                    }
+                                }
+                                else
+                                    recoveryData.onNodeLeft();
+                            }
+                        }
+                    }
+
                     CommunicationListener<GridTcpCommunicationMessageAdapter> lsnr0 = lsnr;
 
                     if (lsnr0 != null)
@@ -259,20 +299,22 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 if (sndId == null) {
                     assert ses.accepted();
 
-                    assert msg instanceof NodeIdMessage;
+                    if (msg instanceof NodeIdMessage)
+                        sndId = U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes, 0);
+                    else {
+                        assert msg instanceof HandshakeMessage : msg;
 
-                    sndId = U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes, 0);
+                        sndId = ((HandshakeMessage)msg).nodeId();
+                    }
 
                     if (log.isDebugEnabled())
                         log.debug("Remote node ID received: " + sndId);
 
-                    UUID old = ses.addMeta(NODE_ID_META, sndId);
+                    final UUID old = ses.addMeta(NODE_ID_META, sndId);
 
                     assert old == null;
 
-                    IgniteProductVersion locVer = getSpiContext().localNode().version();
-
-                    ClusterNode rmtNode = getSpiContext().node(sndId);
+                    final ClusterNode rmtNode = getSpiContext().node(sndId);
 
                     if (rmtNode == null) {
                         ses.close();
@@ -280,25 +322,110 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         return;
                     }
 
+                    ClusterNode locNode = getSpiContext().localNode();
+
+                    IgniteProductVersion locVer = locNode.version();
+
                     IgniteProductVersion rmtVer = rmtNode.version();
 
                     if (!locVer.equals(rmtVer))
                         ses.addMeta(GridNioServer.DIFF_VER_NODE_ID_META_KEY, sndId);
 
-                    if (asyncSnd && ses.remoteAddress() != null && !dualSockConn) {
-                        Object sync = locks.tryLock(sndId);
+                    if (ses.remoteAddress() == null)
+                        return;
+
+                    GridCommunicationClient oldClient = clients.get(sndId);
+
+                    boolean hasShmemClient = false;
+
+                    if (oldClient != null) {
+                        if (oldClient instanceof GridTcpNioCommunicationClient) {
+                            if (log.isDebugEnabled())
+                                log.debug("Received incoming connection when already connected " +
+                                    "to this node, rejecting [locNode=" + locNode.id() +
+                                    ", rmtNode=" + sndId + ']');
+
+                            ses.send(new RecoveryLastReceivedMessage(-1));
+
+                            return;
+                        }
+                        else {
+                            assert oldClient instanceof GridShmemCommunicationClient;
+
+                            hasShmemClient = true;
+                        }
+                    }
+
+                    GridFutureAdapterEx<GridCommunicationClient> fut = new GridFutureAdapterEx<>();
+
+                    GridFutureAdapterEx<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(sndId, fut);
+
+                    assert msg instanceof HandshakeMessage : msg;
+
+                    HandshakeMessage msg0 = (HandshakeMessage)msg;
+
+                    final GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(rmtNode);
+
+                    if (oldFut == null) {
+                        oldClient = clients.get(sndId);
+
+                        if (oldClient != null) {
+                            if (oldClient instanceof GridTcpNioCommunicationClient) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Received incoming connection when already connected " +
+                                        "to this node, rejecting [locNode=" + locNode.id() +
+                                        ", rmtNode=" + sndId + ']');
+
+                                ses.send(new RecoveryLastReceivedMessage(-1));
+
+                                return;
+                            }
+                            else {
+                                assert oldClient instanceof GridShmemCommunicationClient;
+
+                                hasShmemClient = true;
+                            }
+                        }
+
+                        boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+                            new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
 
-                        if (sync != null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Received incoming connection from remote node " +
+                                "[rmtNode=" + rmtNode.id() + ", reserved=" + reserved + ']');
+
+                        if (reserved) {
                             try {
-                                if (clients.get(sndId) == null) {
-                                    if (log.isDebugEnabled())
-                                        log.debug("Will reuse session for node: " + sndId);
+                                GridTcpNioCommunicationClient client =
+                                    connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
 
-                                    clients.put(sndId, new GridTcpNioCommunicationClient(ses));
-                                }
+                                fut.onDone(client);
                             }
                             finally {
-                                locks.unlock(sndId, sync);
+                                clientFuts.remove(rmtNode.id(), fut);
+                            }
+                        }
+                    }
+                    else {
+                        if (oldFut instanceof ConnectFuture && locNode.order() < rmtNode.order()) {
+                            if (log.isDebugEnabled()) {
+                                log.debug("Received incoming connection from remote node while " +
+                                    "connecting to this node, rejecting [locNode=" + locNode.id() +
+                                    ", locNodeOrder=" + locNode.order() + ", rmtNode=" + rmtNode.id() +
+                                    ", rmtNodeOrder=" + rmtNode.order() + ']');
+                            }
+
+                            ses.send(new RecoveryLastReceivedMessage(-1));
+                        }
+                        else {
+                            boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+                                new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
+
+                            if (reserved) {
+                                GridTcpNioCommunicationClient client =
+                                    connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
+
+                                fut.onDone(client);
                             }
                         }
                     }
@@ -306,6 +433,35 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 else {
                     rcvdMsgsCnt.increment();
 
+                    GridNioRecoveryDescriptor recovery = ses.recoveryDescriptor();
+
+                    if (recovery != null) {
+                        if (msg instanceof RecoveryLastReceivedMessage) {
+                            RecoveryLastReceivedMessage msg0 = (RecoveryLastReceivedMessage)msg;
+
+                            if (log.isDebugEnabled())
+                                log.debug("Received recovery acknowledgement [rmtNode=" + sndId +
+                                    ", rcvCnt=" + msg0.received() + ']');
+
+                            recovery.ackReceived(msg0.received());
+
+                            return;
+                        }
+                        else {
+                            long rcvCnt = recovery.onReceived();
+
+                            if (rcvCnt % ackSndThreshold == 0) {
+                                if (log.isDebugEnabled())
+                                    log.debug("Send recovery acknowledgement [rmtNode=" + sndId +
+                                        ", rcvCnt=" + rcvCnt + ']');
+
+                                nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(rcvCnt));
+
+                                recovery.lastAcknowledged(rcvCnt);
+                            }
+                        }
+                    }
+
                     IgniteRunnable c;
 
                     if (msgQueueLimit > 0) {
@@ -328,6 +484,135 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     notifyListener(sndId, msg, c);
                 }
             }
+
+            /**
+             * @param recovery Recovery descriptor.
+             * @param ses Session.
+             * @param node Node.
+             * @param rcvCnt Number of received messages..
+             * @param sndRes If {@code true} sends response for recovery handshake.
+             * @param createClient If {@code true} creates NIO communication client.
+             * @return Client.
+             */
+            private GridTcpNioCommunicationClient connected(
+                GridNioRecoveryDescriptor recovery,
+                GridNioSession ses,
+                ClusterNode node,
+                long rcvCnt,
+                boolean sndRes,
+                boolean createClient) {
+                recovery.onHandshake(rcvCnt);
+
+                ses.recoveryDescriptor(recovery);
+
+                nioSrvr.resend(ses);
+
+                if (sndRes)
+                    nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.receivedCount()));
+
+                recovery.connected();
+
+                GridTcpNioCommunicationClient client = null;
+
+                if (createClient) {
+                    client = new GridTcpNioCommunicationClient(ses, log);
+
+                    GridCommunicationClient oldClient = clients.putIfAbsent(node.id(), client);
+
+                    assert oldClient == null;
+                }
+
+                return client;
+            }
+
+            /**
+             *
+             */
+            @SuppressWarnings("PackageVisibleInnerClass")
+            class ConnectClosure implements IgniteInClosure<Boolean> {
+                /** */
+                private static final long serialVersionUID = 0L;
+
+                /** */
+                private final GridNioSession ses;
+
+                /** */
+                private final GridNioRecoveryDescriptor recoveryDesc;
+
+                /** */
+                private final ClusterNode rmtNode;
+
+                /** */
+                private final HandshakeMessage msg;
+
+                /** */
+                private final GridFutureAdapterEx<GridCommunicationClient> fut;
+
+                /** */
+                private final boolean createClient;
+
+                /**
+                 * @param ses Incoming session.
+                 * @param recoveryDesc Recovery descriptor.
+                 * @param rmtNode Remote node.
+                 * @param msg Handshake message.
+                 * @param createClient If {@code true} creates NIO communication client..
+                 * @param fut Connect future.
+                 */
+                ConnectClosure(GridNioSession ses,
+                    GridNioRecoveryDescriptor recoveryDesc,
+                    ClusterNode rmtNode,
+                    HandshakeMessage msg,
+                    boolean createClient,
+                    GridFutureAdapterEx<GridCommunicationClient> fut) {
+                    this.ses = ses;
+                    this.recoveryDesc = recoveryDesc;
+                    this.rmtNode = rmtNode;
+                    this.msg = msg;
+                    this.createClient = createClient;
+                    this.fut = fut;
+                }
+
+                /** {@inheritDoc} */
+                @Override public void apply(Boolean success) {
+                    if (success) {
+                        IgniteInClosure<GridNioFuture<?>> lsnr = new IgniteInClosure<GridNioFuture<?>>() {
+                            @Override public void apply(GridNioFuture<?> msgFut) {
+                                try {
+                                    msgFut.get();
+
+                                    GridTcpNioCommunicationClient client =
+                                        connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient);
+
+                                    fut.onDone(client);
+                                }
+                                catch (GridException | IOException e) {
+                                    if (log.isDebugEnabled())
+                                        log.debug("Failed to send recovery handshake " +
+                                            "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
+
+                                    recoveryDesc.release();
+
+                                    fut.onDone();
+                                }
+                                finally {
+                                    clientFuts.remove(rmtNode.id(), fut);
+                                }
+                            }
+                        };
+
+                        nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.receivedCount()), lsnr);
+                    }
+                    else {
+                        try {
+                            fut.onDone();
+                        }
+                        finally {
+                            clientFuts.remove(rmtNode.id(), fut);
+                        }
+                    }
+                }
+            }
         };
 
     /** Logger. */
@@ -402,9 +687,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Buffer size ratio. */
     private double bufSizeRatio = IgniteSystemProperties.getDouble(GG_COMMUNICATION_BUF_RESIZE_RATIO, 0.8);
 
-    /** Dual socket connection flag. */
-    private boolean dualSockConn = DFLT_DUAL_SOCKET_CONNECTION;
-
     /** NIO server. */
     private GridNioServer<GridTcpCommunicationMessageAdapter> nioSrvr;
 
@@ -414,8 +696,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** {@code TCP_NODELAY} option value for created sockets. */
     private boolean tcpNoDelay = DFLT_TCP_NODELAY;
 
-    /** Use async client flag. */
-    private boolean asyncSnd = true;
+    /** Number of received messages after which acknowledgment is sent. */
+    private int ackSndThreshold = DFLT_ACK_SND_THRESHOLD;
+
+    /** Maximum number of unacknowledged messages. */
+    private int unackedMsgsBufSize;
+
+    /** Socket write timeout. */
+    private long sockWriteTimeout = DFLT_SOCK_WRITE_TIMEOUT;
 
     /** Shared memory accept worker. */
     private ShmemAcceptWorker shmemAcceptWorker;
@@ -429,6 +717,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Socket timeout worker. */
     private SocketTimeoutWorker sockTimeoutWorker;
 
+    /** Recovery worker. */
+    private RecoveryWorker recoveryWorker;
+
     /** Shared memory workers. */
     private final Collection<ShmemWorker> shmemWorkers = new ConcurrentLinkedDeque8<>();
 
@@ -468,6 +759,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Context initialization latch. */
     private final CountDownLatch ctxInitLatch = new CountDownLatch(1);
 
+    /** Stopping flag. */
+    private volatile boolean stopping;
+
     /** metrics listener. */
     private final GridNioMetricsListener metricsLsnr = new GridNioMetricsListener() {
         @Override public void onBytesSent(int bytesCnt) {
@@ -479,8 +773,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         }
     };
 
-    /** Locks. */
-    private final GridKeyLock locks = new GridKeyLock();
+    /** Client connect futures. */
+    private final ConcurrentMap<UUID, GridFutureAdapterEx<GridCommunicationClient>> clientFuts =
+        GridConcurrentFactory.newMap();
+
+    /** */
+    private final ConcurrentMap<ClientKey, GridNioRecoveryDescriptor> recoveryDescs = GridConcurrentFactory.newMap();
 
     /** Discovery listener. */
     private final GridLocalEventListener discoLsnr = new GridLocalEventListener() {
@@ -696,6 +994,58 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return idleConnTimeout;
     }
 
+    /** {@inheritDoc} */
+    @Override public long getSocketWriteTimeout() {
+        return sockWriteTimeout;
+    }
+
+    /**
+     * Sets socket write timeout for TCP connection. If message can not be written to
+     * socket within this time then connection is closed and reconnect is attempted.
+     * <p>
+     * Default to {@link #DFLT_SOCK_WRITE_TIMEOUT}.
+     *
+     * @param sockWriteTimeout Socket write timeout for TCP connection.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setSocketWriteTimeout(long sockWriteTimeout) {
+        this.sockWriteTimeout = sockWriteTimeout;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getAckSendThreshold() {
+        return ackSndThreshold;
+    }
+
+    /**
+     * Sets number of received messages per connection to node after which acknowledgment message is sent.
+     * <p>
+     * Default to {@link #DFLT_ACK_SND_THRESHOLD}.
+     *
+     * @param ackSndThreshold Number of received messages after which acknowledgment is sent.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setAckSendThreshold(int ackSndThreshold) {
+        this.ackSndThreshold = ackSndThreshold;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getUnacknowledgedMessagesBufferSize() {
+        return unackedMsgsBufSize;
+    }
+
+    /**
+     * Sets maximum number of stored unacknowledged messages per connection to node.
+     * If number of unacknowledged messages exceeds this number then connection to node is
+     * closed and reconnect is attempted.
+     *
+     * @param unackedMsgsBufSize Maximum number of unacknowledged messages.
+     */
+    @IgniteSpiConfiguration(optional = true)
+    public void setUnacknowledgedMessagesBufferSize(int unackedMsgsBufSize) {
+        this.unackedMsgsBufSize = unackedMsgsBufSize;
+    }
+
     /**
      * Sets connection buffer size. If set to {@code 0} connection buffer is disabled.
      * <p>
@@ -838,24 +1188,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return selectorsCnt;
     }
 
-    /** {@inheritDoc} */
-    @Override public boolean isAsyncSend() {
-        return asyncSnd;
-    }
-
-    /**
-     * Sets flag defining whether asynchronous (NIO) or synchronous (blocking) IO
-     * should be used to send messages.
-     * <p>
-     * If not provided, default value is {@code true}.
-     *
-     * @param asyncSnd {@code True} if asynchronous IO should be used to send messages.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setAsyncSend(boolean asyncSnd) {
-        this.asyncSnd = asyncSnd;
-    }
-
     /**
      * Sets value for {@code TCP_NODELAY} socket option. Each
      * socket will be opened using provided value.
@@ -915,33 +1247,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     }
 
     /**
-     * Sets flag indicating whether dual-socket connection between nodes should be enforced. If set to
-     * {@code true}, two separate connections will be established between communicating nodes: one for outgoing
-     * messages, and one for incoming. When set to {@code false}, single {@code TCP} connection will be used
-     * for both directions.
-     * <p>
-     * This flag is useful on some operating systems, when {@code TCP_NODELAY} flag is disabled and
-     * messages take too long to get delivered.
-     * <p>
-     * If not provided, default is {@code false}.
-     *
-     * @param dualSockConn Whether dual-socket connection should be enforced.
-     */
-    @IgniteSpiConfiguration(optional = true)
-    public void setDualSocketConnection(boolean dualSockConn) {
-        this.dualSockConn = dualSockConn;
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isDualSocketConnection() {
-        return dualSockConn;
-    }
-
-    /**
      * Sets message queue limit for incoming and outgoing messages.
      * <p>
-     * This parameter only used when {@link #isAsyncSend()} set to {@code true}.
-     * <p>
      * When set to positive number send queue is limited to the configured value.
      * {@code 0} disables the size limitations.
      * <p>
@@ -963,7 +1270,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * Sets the minimum number of messages for this SPI, that are buffered
      * prior to sending.
      * <p>
-     * Defaults to either {@code 512} or {@link org.apache.ignite.IgniteSystemProperties#GG_MIN_BUFFERED_COMMUNICATION_MSG_CNT}
+     * Defaults to either {@code 512} or {@link IgniteSystemProperties#GG_MIN_BUFFERED_COMMUNICATION_MSG_CNT}
      * system property (if specified).
      *
      * @param minBufferedMsgCnt Minimum buffered message count.
@@ -982,7 +1289,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * Sets the buffer size ratio for this SPI. As messages are sent,
      * the buffer size is adjusted using this ratio.
      * <p>
-     * Defaults to either {@code 0.8} or {@link org.apache.ignite.IgniteSystemProperties#GG_COMMUNICATION_BUF_RESIZE_RATIO}
+     * Defaults to either {@code 0.8} or {@link IgniteSystemProperties#GG_COMMUNICATION_BUF_RESIZE_RATIO}
      * system property (if specified).
      *
      * @param bufSizeRatio Buffer size ratio.
@@ -1064,6 +1371,17 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         assertParameter(bufSizeRatio > 0 && bufSizeRatio < 1, "bufSizeRatio > 0 && bufSizeRatio < 1");
         assertParameter(connTimeout >= 0, "connTimeout >= 0");
         assertParameter(maxConnTimeout >= connTimeout, "maxConnTimeout >= connTimeout");
+        assertParameter(sockWriteTimeout >= 0, "sockWriteTimeout >= 0");
+        assertParameter(ackSndThreshold > 0, "ackSndThreshold > 0");
+        assertParameter(unackedMsgsBufSize >= 0, "unackedMsgsBufSize >= 0");
+
+        if (unackedMsgsBufSize > 0) {
+            assertParameter(unackedMsgsBufSize >= msgQueueLimit * 5,
+                "Specified 'unackedMsgsBufSize' is too low, it should be at least 'msgQueueLimit * 5'.");
+
+            assertParameter(unackedMsgsBufSize >= ackSndThreshold * 5,
+                "Specified 'unackedMsgsBufSize' is too low, it should be at least 'ackSndThreshold * 5'.");
+        }
 
         try {
             locHost = U.resolveLocalHost(locAddr);
@@ -1125,18 +1443,19 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("connBufSize", connBufSize));
             log.debug(configInfo("connBufFlushFreq", connBufFlushFreq));
             log.debug(configInfo("selectorsCnt", selectorsCnt));
-            log.debug(configInfo("asyncSend", asyncSnd));
             log.debug(configInfo("tcpNoDelay", tcpNoDelay));
             log.debug(configInfo("sockSndBuf", sockSndBuf));
             log.debug(configInfo("sockRcvBuf", sockRcvBuf));
             log.debug(configInfo("shmemPort", shmemPort));
             log.debug(configInfo("msgQueueLimit", msgQueueLimit));
-            log.debug(configInfo("dualSockConn", dualSockConn));
             log.debug(configInfo("minBufferedMsgCnt", minBufferedMsgCnt));
             log.debug(configInfo("bufSizeRatio", bufSizeRatio));
             log.debug(configInfo("connTimeout", connTimeout));
             log.debug(configInfo("maxConnTimeout", maxConnTimeout));
             log.debug(configInfo("reconCnt", reconCnt));
+            log.debug(configInfo("sockWriteTimeout", sockWriteTimeout));
+            log.debug(configInfo("ackSndThreshold", ackSndThreshold));
+            log.debug(configInfo("unackedMsgsBufSize", unackedMsgsBufSize));
         }
 
         if (connBufSize > 8192)
@@ -1162,6 +1481,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
         idleClientWorker.start();
 
+        recoveryWorker = new RecoveryWorker();
+
+        recoveryWorker.start();
+
         if (connBufSize > 0) {
             clientFlushWorker = new ClientFlushWorker();
 
@@ -1242,6 +1565,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         .directMode(true)
                         .metricsListener(metricsLsnr)
                         .messageWriter(msgWriter)
+                        .writeTimeout(sockWriteTimeout)
                         .filters(new GridNioCodecFilter(new GridDirectParser(msgReader, this), log, true),
                             new GridConnectionBytesVerifyFilter(log))
                         .build();
@@ -1321,6 +1645,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
     /** {@inheritDoc} */
     @Override public void spiStop() throws IgniteSpiException {
+        assert stopping;
+
         unregisterMBean();
 
         // Stop TCP server.
@@ -1333,10 +1659,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         U.interrupt(idleClientWorker);
         U.interrupt(clientFlushWorker);
         U.interrupt(sockTimeoutWorker);
+        U.interrupt(recoveryWorker);
 
         U.join(idleClientWorker, log);
         U.join(clientFlushWorker, log);
         U.join(sockTimeoutWorker, log);
+        U.join(recoveryWorker, log);
 
         U.cancel(shmemWorkers);
         U.join(shmemWorkers, log);
@@ -1360,6 +1688,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
     /** {@inheritDoc} */
     @Override protected void onContextDestroyed0() {
+        stopping = true;
+
         if (ctxInitLatch.getCount() > 0)
             // Safety.
             ctxInitLatch.countDown();
@@ -1428,20 +1758,33 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             GridCommunicationClient client = null;
 
             try {
-                client = reserveClient(node);
+                boolean retry;
+
+                do {
+                    client = reserveClient(node);
 
-                UUID nodeId = null;
+                    UUID nodeId = null;
 
-                if (!client.async() && !getSpiContext().localNode().version().equals(node.version()))
-                    nodeId = node.id();
+                    if (!client.async() && !getSpiContext().localNode().version().equals(node.version()))
+                        nodeId = node.id();
 
-                client.sendMessage(nodeId, msg);
+                    retry = client.sendMessage(nodeId, msg);
 
-                client.release();
+                    client.release();
 
-                client = null;
+                    client = null;
+
+                    if (!retry)
+                        sentMsgsCnt.increment();
+                    else {
+                        ClusterNode node0 = getSpiContext().node(node.id());
 
-                sentMsgsCnt.increment();
+                        if (node0 == null)
+                            throw new GridException("Failed to send message to remote node " +
+                                "(node has left the grid): " + node.id());
+                    }
+                }
+                while (retry);
             }
             catch (GridException e) {
                 throw new IgniteSpiException("Failed to send message to remote node: " + node, e);
@@ -1469,23 +1812,49 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             GridCommunicationClient client = clients.get(nodeId);
 
             if (client == null) {
+                if (stopping)
+                    throw new IgniteSpiException("Grid is stopping.");
+
                 // Do not allow concurrent connects.
-                Object sync = locks.lock(nodeId);
+                GridFutureAdapterEx<GridCommunicationClient> fut = new ConnectFuture();
 
-                try {
-                    client = clients.get(nodeId);
+                GridFutureAdapterEx<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(nodeId, fut);
+
+                if (oldFut == null) {
+                    try {
+                        GridCommunicationClient client0 = clients.get(nodeId);
+
+                        if (client0 == null) {
+                            client0 = createNioClient(node);
+
+                            if (client0 != null) {
+                                GridCommunicationClient old = clients.put(nodeId, client0);
+
+                                assert old == null;
+                            }
+                            else
+                                U.sleep(200);
+                        }
 
-                    if (client == null) {
-                        GridCommunicationClient old = clients.put(nodeId, client = createNioClient(node));
+                        fut.onDone(client0);
+                    }
+                    catch (Throwable e) {
+                        fut.onDone(e);
 
-                        assert old == null;
+                        if (e instanceof Error)
+                            throw (Error)e;
+                    }
+                    finally {
+                        clientFuts.remove(nodeId, fut);
                     }
                 }
-                finally {
-                    locks.unlock(nodeId, sync);
-                }
+                else
+                    fut = oldFut;
 
-                assert client != null;
+                client = fut.get();
+
+                if (client == null)
+                    continue;
 
                 if (getSpiContext().node(nodeId) == null) {
                     if (clients.remove(nodeId, client))
@@ -1570,7 +1939,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             }
 
             try {
-                safeHandshake(client, node.id(), connTimeout0);
+                safeHandshake(client, null, node.id(), connTimeout0);
             }
             catch (HandshakeTimeoutException e) {
                 if (log.isDebugEnabled())
@@ -1661,59 +2030,74 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
             while (!conn) { // Reconnection on handshake timeout.
                 try {
-                    if (asyncSnd) {
-                        SocketChannel ch = SocketChannel.open();
+                    SocketChannel ch = SocketChannel.open();
 
-                        ch.configureBlocking(true);
+                    ch.configureBlocking(true);
 
-                        ch.socket().setTcpNoDelay(tcpNoDelay);
-                        ch.socket().setKeepAlive(true);
+                    ch.socket().setTcpNoDelay(tcpNoDelay);
+                    ch.socket().setKeepAlive(true);
 
-                        if (sockRcvBuf > 0)
-                            ch.socket().setReceiveBufferSize(sockRcvBuf);
+                    if (sockRcvBuf > 0)
+                        ch.socket().setReceiveBufferSize(sockRcvBuf);
 
-                        if (sockSndBuf > 0)
-                            ch.socket().setSendBufferSize(sockSndBuf);
+                    if (sockSndBuf > 0)
+                        ch.socket().setSendBufferSize(sockSndBuf);
 
-                        ch.socket().connect(addr, (int)connTimeout);
+                    GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(node);
 
-                        safeHandshake(ch, node.id(), connTimeout0);
+                    if (!recoveryDesc.reserve()) {
+                        U.closeQuiet(ch);
 
-                        UUID diffVerNodeId = null;
+                        return null;
+                    }
 
-                        IgniteProductVersion locVer = getSpiContext().localNode().version();
-                        IgniteProductVersion rmtVer = node.version();
+                    long rcvCnt = -1;
 
-                        if (!locVer.equals(rmtVer))
-                            diffVerNodeId = node.id();
+                    try {
+                        ch.socket().connect(addr, (int)connTimeout);
 
-                        GridNioSession ses = nioSrvr.createSession(
-                            ch,
-                            F.asMap(
-                                NODE_ID_META, node.id(),
-                                GridNioServer.DIFF_VER_NODE_ID_META_KEY, diffVerNodeId)
-                        ).get();
+                        rcvCnt = safeHandshake(ch, recoveryDesc, node.id(), connTimeout0);
 
-                        client = new GridTcpNioCommunicationClient(ses);
+                        if (rcvCnt == -1)
+                            return null;
                     }
-                    else {
-                        client = new GridTcpCommunicationClient(
-                            metricsLsnr,
-                            msgWriter,
-                            addr,
-                            locHost,
-                            connTimeout,
-                            tcpNoDelay,
-                            sockRcvBuf,
-                            sockSndBuf,
-                            connBufSize,
-                            minBufferedMsgCnt,
-                            bufSizeRatio);
-
-                        safeHandshake(client, node.id(), connTimeout0);
+                    finally {
+                        if (recoveryDesc != null && rcvCnt == -1)
+                            recoveryDesc.release();
                     }
 
-                    conn = true;
+                    UUID diffVerNodeId = null;
+
+                    IgniteProductVersion locVer = getSpiContext().localNode().version();
+                    IgniteProductVersion rmtVer = node.version();
+
+                    if (!locVer.equals(rmtVer))
+                        diffVerNodeId = node.id();
+
+                    try {
+                        Map<Integer, Object> meta = new HashMap<>();
+
+                        meta.put(NODE_ID_META, node.id());
+                        meta.put(GridNioServer.DIFF_VER_NODE_ID_META_KEY, diffVerNodeId);
+
+                        if (recoveryDesc != null) {
+                            recoveryDesc.onHandshake(rcvCnt);
+
+                            meta.put(-1, recoveryDesc);
+                        }
+
+                        GridNioSession ses = nioSrvr.createSession(ch, meta).get();
+
+                        client = new GridTcpNioCommunicationClient(ses, log);
+
+                        conn = true;
+                    }
+                    finally {
+                        if (!conn) {
+                            if (recoveryDesc != null)
+                                recoveryDesc.release();
+                        }
+                    }
                 }
                 catch (HandshakeTimeoutException e) {
                     if (client != null) {
@@ -1812,16 +2196,25 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * Performs handshake in timeout-safe way.
      *
      * @param client Client.
+     * @param recovery Recovery descriptor if use recovery handshake, otherwise {@code null}.
      * @param rmtNodeId Remote node.
      * @param timeout Timeout for handshake.
      * @throws GridException If handshake failed or wasn't completed withing timeout.
+     * @return Handshake response.
      */
     @SuppressWarnings("ThrowFromFinallyBlock")
-    private <T> void safeHandshake(T client, UUID rmtNodeId, long timeout) throws GridException {
+    private <T> long safeHandshake(
+        T client,
+        @Nullable GridNioRecoveryDescriptor recovery,
+        UUID rmtNodeId,
+        long timeout
+    ) throws GridException {
         HandshakeTimeoutObject<T> obj = new HandshakeTimeoutObject<>(client, U.currentTimeMillis() + timeout);
 
         sockTimeoutWorker.addTimeoutObject(obj);
 
+        long rcvCnt = 0;
+
         try {
             if (client instanceof GridCommunicationClient)
                 ((GridCommunicationClient)client).doHandshake(new HandshakeClosure(rmtNodeId));
@@ -1851,9 +2244,62 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         log.debug("Received remote node ID: " + rmtNodeId0);
 
                     ch.write(ByteBuffer.wrap(U.GG_HEADER));
-                    ch.write(ByteBuffer.wrap(nodeIdMsg.nodeIdBytesWithType));
 
-                    success = true;
+                    if (recovery != null) {
+                        HandshakeMessage msg = new HandshakeMessage(locNodeId,
+                            recovery.incrementConnectCount(),
+                            recovery.receivedCount());
+
+                        if (log.isDebugEnabled())
+                            log.debug("Write handshake message [rmtNode=" + rmtNodeId + ", msg=" + msg + ']');
+
+                        buf = ByteBuffer.allocate(33);
+
+                        buf.order(ByteOrder.nativeOrder());
+
+                        boolean written = msg.writeTo(buf);
+
+                        assert written;
+
+                        buf.flip();
+
+                        ch.write(buf);
+                    }
+                    else
+                        ch.write(ByteBuffer.wrap(nodeIdMsg.nodeIdBytesWithType));
+
+                    if (recovery != null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Waiting for handshake [rmtNode=" + rmtNodeId + ']');
+
+                        buf = ByteBuffer.allocate(9);
+
+                        buf.order(ByteOrder.nativeOrder());
+
+                        for (int i = 0; i < 9; ) {
+                            int read = ch.read(buf);
+
+                            if (read == -1)
+                                throw new GridException("Failed to read remote node recovery handshake " +
+                                    "(connection closed).");
+
+                            i += read;
+                        }
+
+                        rcvCnt = buf.getLong(1);
+
+                        if (log.isDebugEnabled())
+                            log.debug("Received handshake message [rmtNode=" + rmtNodeId + ", rcvCnt=" + rcvCnt + ']');
+
+                        if (rcvCnt == -1) {
+                            if (log.isDebugEnabled())
+                                log.debug("Connection rejected, will retry client creation [rmtNode=" + rmtNodeId + ']');
+                        }
+                        else
+                            success = true;
+                    }
+                    else
+                        success = true;
                 }
                 catch (IOException e) {
                     if (log.isDebugEnabled())
@@ -1878,6 +2324,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 throw new HandshakeTimeoutException("Failed to perform handshake due to timeout (consider increasing " +
                     "'connectionTimeout' configuration property).");
         }
+
+        return rcvCnt;
     }
 
     /**
@@ -1896,11 +2344,83 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 "is node stopping?) [senderNodeId=" + sndId + ", msg=" + msg + ']');
     }
 
+    /**
+     * @param node Node.
+     * @return Recovery receive data for given node.
+     */
+    private GridNioRecoveryDescriptor recoveryDescriptor(ClusterNode node) {
+        ClientKey id = new ClientKey(node.id(), node.order());
+
+        GridNioRecoveryDescriptor recovery = recoveryDescs.get(id);
+
+        if (recovery == null) {
+            int maxSize = Math.max(msgQueueLimit, ackSndThreshold);
+
+            int queueLimit = unackedMsgsBufSize != 0 ? unackedMsgsBufSize : (maxSize * 5);
+
+            GridNioRecoveryDescriptor old =
+                recoveryDescs.put(id, recovery = new GridNioRecoveryDescriptor(queueLimit, node, log));
+
+            if (old != null)
+                recovery = old;
+        }
+
+        return recovery;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(TcpCommunicationSpi.class, this);
     }
 
+    /**
+     *
+     */
+    private static class ClientKey {
+        /** */
+        private UUID nodeId;
+
+        /** */
+        private long order;
+
+        /**
+         * @param nodeId Node ID.
+         * @param order Node order.
+         */
+        private ClientKey(UUID nodeId, long order) {
+            this.nodeId = nodeId;
+            this.order = order;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean equals(Object obj) {
+            if (this == obj)
+                return true;
+
+            if (obj == null || getClass() != obj.getClass())
+                return false;
+
+            ClientKey other = (ClientKey)obj;
+
+            return order == other.order && nodeId.equals(other.nodeId);
+
+        }
+
+        /** {@inheritDoc} */
+        @Override public int hashCode() {
+            int res = nodeId.hashCode();
+
+            res = 31 * res + (int)(order ^ (order >>> 32));
+
+            return res;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ClientKey.class, this);
+        }
+    }
+
     /** Internal exception class for proper timeout handling. */
     private static class HandshakeTimeoutException extends GridException {
         /** */
@@ -2032,12 +2552,16 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         @SuppressWarnings({"BusyWait"})
         @Override protected void body() throws InterruptedException {
             while (!isInterrupted()) {
+                cleanupRecovery();
+
                 for (Map.Entry<UUID, GridCommunicationClient> e : clients.entrySet()) {
                     UUID nodeId = e.getKey();
 
                     GridCommunicationClient client = e.getValue();
 
-                    if (getSpiContext().node(nodeId) == null) {
+                    ClusterNode node = getSpiContext().node(nodeId);
+
+                    if (node == null) {
                         if (log.isDebugEnabled())
                             log.debug("Forcing close of non-existent node connection: " + nodeId);
 
@@ -2048,9 +2572,39 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         continue;
                     }
 
+                    GridNioRecoveryDescriptor recovery = null;
+
+                    if (client instanceof GridTcpNioCommunicationClient) {
+                        recovery = recoveryDescs.get(new ClientKey(node.id(), node.order()));
+
+                        if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
+                            RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
+
+                            if (log.isDebugEnabled())
+                                log.debug("Send recovery acknowledgement on timeout [rmtNode=" + nodeId +
+                                    ", rcvCnt=" + msg.received() + ']');
+
+                            nioSrvr.sendSystem(((GridTcpNioCommunicationClient)client).session(), msg);
+
+                            recovery.lastAcknowledged(msg.received());
+
+                            continue;
+                        }
+                    }
+
                     long idleTime = client.getIdleTime();
 
                     if (idleTime >= idleConnTimeout) {
+                        if (recovery != null &&
+                            recovery.nodeAlive(getSpiContext().node(nodeId)) &&
+                            !recovery.messagesFutures().isEmpty()) {
+                            if (log.isDebugEnabled())
+                                log.debug("Node connection is idle, but there are unacknowledged messages, " +
+                                    "will wait: " + nodeId);
+
+                            continue;
+                        }
+
                         if (log.isDebugEnabled())
                             log.debug("Closing idle node connection: " + nodeId);
 
@@ -2062,6 +2616,38 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 Thread.sleep(idleConnTimeout);
             }
         }
+
+        /**
+         *
+         */
+        private void cleanupRecovery() {
+            Set<ClientKey> left = null;
+
+            for (Map.Entry<ClientKey, GridNioRecoveryDescriptor> e : recoveryDescs.entrySet()) {
+                if (left != null && left.contains(e.getKey()))
+                    continue;
+
+                GridNioRecoveryDescriptor recoverySnd = e.getValue();
+
+                if (!recoverySnd.nodeAlive(getSpiContext().node(recoverySnd.node().id()))) {
+                    if (left == null)
+                        left = new HashSet<>();
+
+                    left.add(e.getKey());
+                }
+            }
+
+            if (left != null) {
+                assert !left.isEmpty();
+
+                for (ClientKey id : left) {
+                    GridNioRecoveryDescriptor recoverySnd = recoveryDescs.remove(id);
+
+                    if (recoverySnd != null)
+                        recoverySnd.onNodeLeft();
+                }
+            }
+        }
     }
 
     /**
@@ -2213,6 +2799,84 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      *
      */
+    private class RecoveryWorker extends IgniteSpiThread {
+        /** */
+        private final BlockingQueue<GridNioRecoveryDescriptor> q = new LinkedBlockingQueue<>();
+
+        /**
+         *
+         */
+        private RecoveryWorker() {
+            super(gridName, "tcp-comm-recovery-worker", log);
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void body() throws InterruptedException {
+            if (log.isDebugEnabled())
+                log.debug("Recovery worker has been started.");
+
+            while (!isInterrupted()) {
+                GridNioRecoveryDescriptor recoveryDesc = q.take();
+
+                assert recoveryDesc != null;
+
+                ClusterNode node = recoveryDesc.node();
+
+                if (clients.containsKey(node.id()) || !recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
+                    continue;
+
+                try {
+                    if (log.isDebugEnabled())
+                        log.debug("Recovery reconnect [rmtNode=" + recoveryDesc.node().id() + ']');
+
+                    GridCommunicationClient client = reserveClient(node);
+
+                    client.release();
+                }
+                catch (GridException e) {
+                    if (recoveryDesc.nodeAlive(getSpiContext().node(node.id()))) {
+                        if (log.isDebugEnabled())
+                            log.debug("Recovery reconnect failed, will retry " +
+                                "[rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+
+                        addReconnectRequest(recoveryDesc);
+                    }
+                    else if (log.isDebugEnabled())
+                        log.debug("Recovery reconnect failed, " +
+                            "node left [rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+
+                }
+            }
+        }
+
+        /**
+         * @param recoverySnd Recovery send data.
+         */
+        void addReconnectRequest(GridNioRecoveryDescriptor recoverySnd) {
+            boolean add = q.add(recoverySnd);
+
+            assert add;
+        }
+    }
+
+    /**
+     *
+     */
+    private static class ConnectFuture extends GridFutureAdapterEx<GridCommunicationClient> {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /**
+         * Empty constructor required for {@link Externalizable}.
+         */
+        public ConnectFuture() {
+            // No-op.
+        }
+    }
+
+    /**
+     *
+     */
     private static class HandshakeTimeoutObject<T> {
         /** */
         private static final AtomicLong idGen = new AtomicLong();
@@ -2355,6 +3019,206 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     }
 
     /**
+     * Handshake message.
+     */
+    @SuppressWarnings("PublicInnerClass")
+    public static class HandshakeMessage extends GridTcpCommunicationMessageAdapter {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private UUID nodeId;
+
+        /** */
+        private long rcvCnt;
+
+        /** */
+        private long connectCnt;
+
+        /**
+         * Default constructor required by {@link GridTcpCommunicationMessageAdapter}.
+         */
+        public HandshakeMessage() {
+            // No-op.
+        }
+
+        /**
+         * @param nodeId Node ID.
+         * @param connectCnt Connect count.
+         * @param rcvCnt Number of received messages.
+         */
+        public HandshakeMessage(UUID nodeId, long connectCnt, long rcvCnt) {
+            assert nodeId != null;
+            assert rcvCnt >= 0 : rcvCnt;
+
+            this.nodeId = nodeId;
+            this.connectCnt = connectCnt;
+            this.rcvCnt = rcvCnt;
+        }
+
+        /**
+         * @return Connect count.
+         */
+        public long connectCount() {
+            return connectCnt;
+        }
+
+        /**
+         * @return Number of received messages.
+         */
+        public long received() {
+            return rcvCnt;
+        }
+
+        /**
+         * @return Node ID.
+         */
+        public UUID nodeId() {
+            return nodeId;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean writeTo(ByteBuffer buf) {
+            if (buf.remaining() < 33)
+                return false;
+
+            buf.put(HANDSHAKE_MSG_TYPE);
+
+            byte[] bytes = U.uuidToBytes(nodeId);
+
+            assert bytes.length == 16 : bytes.length;
+
+            buf.put(bytes);
+
+            buf.putLong(rcvCnt);
+
+            buf.putLong(connectCnt);
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean readFrom(ByteBuffer buf) {
+            if (buf.remaining() < 32)
+                return false;
+
+            byte[] nodeIdBytes = new byte[16];
+
+            buf.get(nodeIdBytes);
+
+            nodeId = U.bytesToUuid(nodeIdBytes, 0);
+
+            rcvCnt = buf.getLong();
+
+            connectCnt = buf.getLong();
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public byte directType() {
+            return HANDSHAKE_MSG_TYPE;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings("CloneDoesntCallSuperClone")
+        @Override public GridTcpCommunicationMessageAdapter clone() {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void clone0(GridTcpCommunicationMessageAdapter msg) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(HandshakeMessage.class, this);
+        }
+    }
+
+    /**
+     * Recovery acknowledgment message.
+     */
+    @SuppressWarnings("PublicInnerClass")
+    public static class RecoveryLastReceivedMessage extends GridTcpCommunicationMessageAdapter {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private long rcvCnt;
+
+        /**
+         * Default constructor required by {@link GridTcpCommunicationMessageAdapter}.
+         */
+        public RecoveryLastReceivedMessage() {
+            // No-op.
+        }
+
+        /**
+         * @param rcvCnt Number of received messages.
+         */
+        public RecoveryLastReceivedMessage(long rcvCnt) {
+            this.rcvCnt = rcvCnt;
+        }
+
+        /**
+         * @return Number of received messages.
+         */
+        public long received() {
+            return rcvCnt;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean writeTo(ByteBuffer buf) {
+            if (buf.remaining() < 9)
+                return false;
+
+            buf.put(RECOVERY_LAST_ID_MSG_TYPE);
+
+            buf.putLong(rcvCnt);
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean readFrom(ByteBuffer buf) {
+            if (buf.remaining() < 8)
+                return false;
+
+            rcvCnt = buf.getLong();
+
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public byte directType() {
+            return RECOVERY_LAST_ID_MSG_TYPE;
+        }
+
+        /** {@inheritDoc} */
+        @SuppressWarnings({"CloneDoesntCallSuperClone", "CloneCallsConstructors"})
+        @Override public GridTcpCommunicationMessageAdapter clone() {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override protected void clone0(GridTcpCommunicationMessageAdapter msg) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean skipRecovery() {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(RecoveryLastReceivedMessage.class, this);
+        }
+    }
+
+    /**
      * Node ID message.
      */
     @SuppressWarnings("PublicInnerClass")
@@ -2426,5 +3290,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         @Override protected void clone0(GridTcpCommunicationMessageAdapter _msg) {
             // No-op.
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(NodeIdMessage.class, this);
+        }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
index b35e735..1255925 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpiMBean.java
@@ -70,15 +70,6 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
     public int getSelectorsCount();
 
     /**
-     * Gets flag defining whether asynchronous (NIO) or synchronous (blocking) IO
-     * should be used to send messages.
-     *
-     * @return {@code True} if asynchronous IO should be used to send messages.
-     */
-    @IgniteMBeanDescription("Asynchronous send.")
-    public boolean isAsyncSend();
-
-    /**
      * Gets sent messages count.
      *
      * @return Sent messages count.
@@ -220,23 +211,7 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
     public int getSocketSendBuffer();
 
     /**
-     * Gets flag indicating whether dual-socket connection between nodes should be enforced. If set to
-     * {@code true}, two separate connections will be established between communicating nodes: one for outgoing
-     * messages, and one for incoming. When set to {@code false}, single {@code TCP} connection will be used
-     * for both directions.
-     * <p>
-     * This flag is useful on some operating systems, when {@code TCP_NODELAY} flag is disabled and
-     * messages take too long to get delivered.
-     *
-     * @return Whether dual-socket connection should be enforced.
-     */
-    @IgniteMBeanDescription("Dual-socket connection.")
-    public boolean isDualSocketConnection();
-
-    /**
      * Gets message queue limit for incoming and outgoing messages.
-     * <p>
-     * This parameter only used when {@link #isAsyncSend()} set to {@code true}.
      *
      * @return Send queue size limit.
      */
@@ -260,4 +235,31 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
      */
     @IgniteMBeanDescription("Buffer size ratio.")
     public double getBufferSizeRatio();
+
+    /**
+     * Gets socket write timeout for TCP connections. If message can not be written to
+     * socket within this time then connection is closed and reconnect is attempted.
+     *
+     * @return Socket write timeout for TCP connections.
+     */
+    @IgniteMBeanDescription("Socket write timeout.")
+    public long getSocketWriteTimeout();
+
+    /**
+     * Gets number of received messages per connection to node after which acknowledgment message is sent.
+     *
+     * @return Number of received messages after which acknowledgment is sent.
+     */
+    @IgniteMBeanDescription("Number of received messages after which acknowledgment is sent.")
+    public int getAckSendThreshold();
+
+    /**
+     * Gets maximum number of stored unacknowledged messages per connection to node.
+     * If number of unacknowledged messages exceeds this number then connection to node is
+     * closed and reconnect is attempted.
+     *
+     * @return Maximum number of unacknowledged messages.
+     */
+    @IgniteMBeanDescription("Maximum number of unacknowledged messages.")
+    public int getUnacknowledgedMessagesBufferSize();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/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
index 5377e18..ffa7b3d 100644
--- 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
@@ -310,7 +310,8 @@ abstract class TcpDiscoverySpiAdapter extends IgniteSpiAdapter implements Discov
     @Override protected void onContextDestroyed0() {
         super.onContextDestroyed0();
 
-        ipFinder.onSpiContextDestroyed();
+        if (ipFinder != null)
+            ipFinder.onSpiContextDestroyed();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageAdapter.java b/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageAdapter.java
index 8c479ff..af63a2f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageAdapter.java
@@ -168,6 +168,13 @@ public abstract class GridTcpCommunicationMessageAdapter implements Serializable
     protected abstract void clone0(GridTcpCommunicationMessageAdapter _msg);
 
     /**
+     * @return {@code True} if should skip recovery for this message.
+     */
+    public boolean skipRecovery() {
+        return false;
+    }
+
+    /**
      * @param arr Array.
      * @return Array iterator.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageFactory.java b/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageFactory.java
index 460c827..7c92065 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageFactory.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/direct/GridTcpCommunicationMessageFactory.java
@@ -282,8 +282,14 @@ public class GridTcpCommunicationMessageFactory {
      * @return New message.
      */
     public static GridTcpCommunicationMessageAdapter create(byte type) {
-        return type == TcpCommunicationSpi.NODE_ID_MSG_TYPE ? new TcpCommunicationSpi.NodeIdMessage() :
-            create0(type);
+        if (type == TcpCommunicationSpi.NODE_ID_MSG_TYPE)
+            return new TcpCommunicationSpi.NodeIdMessage();
+        else if (type == TcpCommunicationSpi.RECOVERY_LAST_ID_MSG_TYPE)
+            return new TcpCommunicationSpi.RecoveryLastReceivedMessage();
+        else if (type == TcpCommunicationSpi.HANDSHAKE_MSG_TYPE)
+            return new TcpCommunicationSpi.HandshakeMessage();
+        else
+            return create0(type);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridCommunicationClient.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridCommunicationClient.java
index 4128475..5b0db53 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridCommunicationClient.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridCommunicationClient.java
@@ -87,8 +87,9 @@ public interface GridCommunicationClient {
      * @param nodeId Node ID (provided only if versions of local and remote nodes are different).
      * @param msg Message to send.
      * @throws GridException If failed.
+     * @return {@code True} if should try to resend message.
      */
-    void sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg) throws GridException;
+    boolean sendMessage(@Nullable UUID nodeId, GridTcpCommunicationMessageAdapter msg) throws GridException;
 
     /**
      * @param timeout Timeout.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFinishedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFinishedFuture.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFinishedFuture.java
index 980b548..d6fcb60 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFinishedFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFinishedFuture.java
@@ -99,4 +99,9 @@ public class GridNioFinishedFuture<R> implements GridNioFuture<R> {
     @Override public boolean messageThread() {
         return msgThread;
     }
+
+    /** {@inheritDoc} */
+    @Override public boolean skipRecovery() {
+        return true;
+    }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFuture.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFuture.java
index 2775c55..a3ab1ef 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFuture.java
@@ -28,7 +28,7 @@ public interface GridNioFuture<R> {
      *
      * @return Operation result.
      * @throws GridInterruptedException Subclass of {@link GridException} thrown if the wait was interrupted.
-     * @throws org.apache.ignite.lang.IgniteFutureCancelledException Subclass of {@link GridException} throws if operation was cancelled.
+     * @throws IgniteFutureCancelledException Subclass of {@link GridException} throws if operation was cancelled.
      * @throws GridException If operation failed.
      * @throws IOException If IOException occurred while performing operation.
      */
@@ -42,8 +42,8 @@ public interface GridNioFuture<R> {
      * @param timeout The maximum time to wait in milliseconds.
      * @return Operation result.
      * @throws GridInterruptedException Subclass of {@link GridException} thrown if the wait was interrupted.
-     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException Subclass of {@link GridException} thrown if the wait was timed out.
-     * @throws org.apache.ignite.lang.IgniteFutureCancelledException Subclass of {@link GridException} throws if operation was cancelled.
+     * @throws IgniteFutureTimeoutException Subclass of {@link GridException} thrown if the wait was timed out.
+     * @throws IgniteFutureCancelledException Subclass of {@link GridException} throws if operation was cancelled.
      * @throws GridException If operation failed.
      * @throws IOException If IOException occurred while performing operation.
      */
@@ -57,8 +57,8 @@ public interface GridNioFuture<R> {
      * @param unit The time unit of the {@code timeout} argument.
      * @return Operation result.
      * @throws GridInterruptedException Subclass of {@link GridException} thrown if the wait was interrupted.
-     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException Subclass of {@link GridException} thrown if the wait was timed out.
-     * @throws org.apache.ignite.lang.IgniteFutureCancelledException Subclass of {@link GridException} throws if operation was cancelled.
+     * @throws IgniteFutureTimeoutException Subclass of {@link GridException} thrown if the wait was timed out.
+     * @throws IgniteFutureCancelledException Subclass of {@link GridException} throws if operation was cancelled.
      * @throws GridException If operation failed.
      * @throws IOException If IOException occurred while performing operation.
      */
@@ -104,4 +104,9 @@ public interface GridNioFuture<R> {
      * @return {@code True} if future was created in thread that was processing message.
      */
     public boolean messageThread();
+
+    /**
+     * @return {@code True} if skip recovery for this operation.
+     */
+    public boolean skipRecovery();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFutureImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFutureImpl.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFutureImpl.java
index 3d5c4e9..7ab2e14 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFutureImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioFutureImpl.java
@@ -118,7 +118,7 @@ public class GridNioFutureImpl<R> extends AbstractQueuedSynchronizer implements
      * @param nanosTimeout Timeout (nanoseconds).
      * @return Result.
      * @throws InterruptedException If interrupted.
-     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException If timeout reached before computation completed.
+     * @throws IgniteFutureTimeoutException If timeout reached before computation completed.
      * @throws GridException If error occurred.
      */
     @Nullable protected R get0(long nanosTimeout) throws InterruptedException, GridException {
@@ -307,6 +307,11 @@ public class GridNioFutureImpl<R> extends AbstractQueuedSynchronizer implements
     }
 
     /** {@inheritDoc} */
+    @Override public boolean skipRecovery() {
+        return false;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridNioFutureImpl.class, this);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/dae4b942/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioRecoveryDescriptor.java
new file mode 100644
index 0000000..846a388
--- /dev/null
+++ b/modules/core/src/main/java/org/gridgain/grid/util/nio/GridNioRecoveryDescriptor.java
@@ -0,0 +1,356 @@
+/* @java.file.header */
+
+/*  _________        _____ __________________        _____
+ *  __  ____/___________(_)______  /__  ____/______ ____(_)_______
+ *  _  / __  __  ___/__  / _  __  / _  / __  _  __ `/__  / __  __ \
+ *  / /_/ /  _  /    _  /  / /_/ /  / /_/ /  / /_/ / _  /  _  / / /
+ *  \____/   /_/     /_/   \_,__/   \____/   \__,_/  /_/   /_/ /_/
+ */
+
+package org.gridgain.grid.util.nio;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.lang.*;
+import org.gridgain.grid.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ * Recovery information for single node.
+ */
+public class GridNioRecoveryDescriptor {
+    /** Number of acknowledged messages. */
+    private long acked;
+
+    /** Unacknowledged message futures. */
+    private final ArrayDeque<GridNioFuture<?>> msgFuts;
+
+    /** Number of messages to resend. */
+    private int resendCnt;
+
+    /** Number of received messages. */
+    private long rcvCnt;
+
+    /** Reserved flag. */
+    private boolean reserved;
+
+    /** Last acknowledged message. */
+    private long lastAck;
+
+    /** Node left flag. */
+    private boolean nodeLeft;
+
+    /** Target node. */
+    private final ClusterNode node;
+
+    /** Logger. */
+    private final IgniteLogger log;
+
+    /** Incoming connection request from remote node. */
+    private IgniteBiTuple<Long, IgniteInClosure<Boolean>> handshakeReq;
+
+    /** Connected flag. */
+    private boolean connected;
+
+    /** Number of outgoing connect attempts. */
+    private long connectCnt;
+
+    /** Maximum size of unacknowledged messages queue. */
+    private final int queueLimit;
+
+    /**
+     * @param queueLimit Maximum size of unacknowledged messages queue.
+     * @param node Node.
+     * @param log Logger.
+     */
+    public GridNioRecoveryDescriptor(int queueLimit, ClusterNode node, IgniteLogger log) {
+        assert !node.isLocal() : node;
+        assert queueLimit > 0;
+
+        msgFuts = new ArrayDeque<>(queueLimit);
+
+        this.queueLimit = queueLimit;
+        this.node = node;
+        this.log = log;
+    }
+
+    /**
+     * @return Connect count.
+     */
+    public long incrementConnectCount() {
+        return connectCnt++;
+    }
+
+    /**
+     * @return Node.
+     */
+    public ClusterNode node() {
+        return node;
+    }
+
+    /**
+     * Increments received messages counter.
+     *
+     * @return Number of received messages.
+     */
+    public long onReceived() {
+        rcvCnt++;
+
+        return rcvCnt;
+    }
+
+    /**
+     * @return Number of received messages.
+     */
+    public long received() {
+        return rcvCnt;
+    }
+
+    /**
+     * @param lastAck Last acknowledged message.
+     */
+    public void lastAcknowledged(long lastAck) {
+        this.lastAck = lastAck;
+    }
+
+    /**
+     * @return Last acknowledged message.
+     */
+    public long lastAcknowledged() {
+        return lastAck;
+    }
+
+    /**
+     * @return Received messages count.
+     */
+    public long receivedCount() {
+        return rcvCnt;
+    }
+
+    /**
+     * @return Maximum size of unacknowledged messages queue.
+     */
+    public int queueLimit() {
+        return queueLimit;
+    }
+
+    /**
+     * @param fut NIO future.
+     * @return {@code False} if queue limit is exceeded.
+     */
+    public boolean add(GridNioFuture<?> fut) {
+        assert fut != null;
+
+        if (!fut.skipRecovery()) {
+            if (resendCnt == 0) {
+                msgFuts.addLast(fut);
+
+                return msgFuts.size() < queueLimit;
+            }
+            else
+                resendCnt--;
+        }
+
+        return true;
+    }
+
+    /**
+     * @param rcvCnt Number of messages received by remote node.
+     */
+    public void ackReceived(long rcvCnt) {
+        if (log.isDebugEnabled())
+            log.debug("Handle acknowledgment [acked=" + acked + ", rcvCnt=" + rcvCnt +
+                ", msgFuts=" + msgFuts.size() + ']');
+
+        while (acked < rcvCnt) {
+            GridNioFuture<?> fut = msgFuts.pollFirst();
+
+            assert fut != null;
+
+            ((GridNioFutureImpl)fut).onDone();
+
+            acked++;
+        }
+    }
+
+    /**
+     * Node left callback.
+     */
+    public void onNodeLeft() {
+        GridNioFuture<?>[] futs = null;
+
+        synchronized (this) {
+            nodeLeft = true;
+
+            if (!reserved && !msgFuts.isEmpty()) {
+                futs = msgFuts.toArray(new GridNioFuture<?>[msgFuts.size()]);
+
+                msgFuts.clear();
+            }
+        }
+
+        if (futs != null)
+            completeOnNodeLeft(futs);
+    }
+
+    /**
+     * @return Message futures for unacknowledged messages.
+     */
+    public Deque<GridNioFuture<?>> messagesFutures() {
+        return msgFuts;
+    }
+
+    /**
+     * @param node Node.
+     * @return {@code True} if node is not null and has the same order as initial remtoe node.
+     */
+    public boolean nodeAlive(@Nullable ClusterNode node) {
+        return node != null && node.order() == this.node.order();
+    }
+
+    /**
+     * @throws InterruptedException If interrupted.
+     * @return {@code True} if reserved.
+     */
+    public boolean reserve() throws InterruptedException {
+        synchronized (this) {
+            while (!connected && reserved)
+                wait();
+
+            if (!connected)
+                reserved = true;
+
+            return !connected;
+        }
+    }
+
+    /**
+     * @param rcvCnt Number of messages received by remote node.
+     */
+    public void onHandshake(long rcvCnt) {
+        ackReceived(rcvCnt);
+
+        resendCnt = msgFuts.size();
+    }
+
+    /**
+     *
+     */
+    public void connected() {
+        synchronized (this) {
+            assert reserved;
+            assert !connected;
+
+            connected = true;
+
+            if (handshakeReq != null) {
+                IgniteInClosure<Boolean> c = handshakeReq.get2();
+
+                assert c != null;
+
+                c.apply(false);
+
+                handshakeReq = null;
+            }
+
+            notifyAll();
+        }
+    }
+
+    /**
+     *
+     */
+    public void release() {
+        GridNioFuture<?>[] futs = null;
+
+        synchronized (this) {
+            connected = false;
+
+            if (handshakeReq != null) {
+                IgniteInClosure<Boolean> c = handshakeReq.get2();
+
+                assert c != null;
+
+                handshakeReq = null;
+
+                c.apply(true);
+            }
+            else {
+                reserved = false;
+
+                notifyAll();
+            }
+
+            if (nodeLeft && !msgFuts.isEmpty()) {
+                futs = msgFuts.toArray(new GridNioFuture<?>[msgFuts.size()]);
+
+                msgFuts.clear();
+            }
+        }
+
+        if (futs != null)
+            completeOnNodeLeft(futs);
+    }
+
+    /**
+     * @param id Handshake ID.
+     * @param c Closure to run on reserve.
+     * @return {@code True} if reserved.
+     */
+    public boolean tryReserve(long id, IgniteInClosure<Boolean> c) {
+        synchronized (this) {
+            if (connected) {
+                c.apply(false);
+
+                return false;
+            }
+
+            if (reserved) {
+                if (handshakeReq != null) {
+                    assert handshakeReq.get1() != null;
+
+                    long id0 = handshakeReq.get1();
+
+                    assert id0 != id : id0;
+
+                    if (id > id0) {
+                        IgniteInClosure<Boolean> c0 = handshakeReq.get2();
+
+                        assert c0 != null;
+
+                        c0.apply(false);
+
+                        handshakeReq = new IgniteBiTuple<>(id, c);
+                    }
+                    else
+                        c.apply(false);
+                }
+                else
+                    handshakeReq = new IgniteBiTuple<>(id, c);
+
+                return false;
+            }
+            else {
+                reserved = true;
+
+                return true;
+            }
+        }
+    }
+
+    /**
+     * @param futs Futures to complete.
+     */
+    private void completeOnNodeLeft(GridNioFuture<?>[] futs) {
+        for (GridNioFuture<?> msg : futs)
+            ((GridNioFutureImpl)msg).onDone(new IOException("Failed to send message, node has left: " + node.id()));
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(GridNioRecoveryDescriptor.class, this);
+    }
+}


[6/7] incubator-ignite git commit: # Merge remote-tracking branch 'remotes/origin/master' into ignite-21

Posted by sb...@apache.org.
# Merge remote-tracking branch 'remotes/origin/master' into ignite-21


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

Branch: refs/heads/master
Commit: 4a7353989a7c7bd2fde6d3bbc6d470752f47ccde
Parents: ce518b1 52f2b40
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 12 11:12:47 2014 +0400
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 12 11:12:47 2014 +0400

----------------------------------------------------------------------
 examples/config/example-cache.xml               |   88 +-
 .../gridgain/examples/ComputeNodeStartup.java   |    4 +-
 .../compute/ComputeBroadcastExample.java        |   10 +-
 .../compute/ComputeCallableExample.java         |    4 +-
 .../examples/compute/ComputeClosureExample.java |    4 +-
 .../compute/ComputeContinuousMapperExample.java |   16 +-
 .../compute/ComputeExecutorServiceExample.java  |    2 +-
 .../ComputeFibonacciContinuationExample.java    |    8 +-
 .../compute/ComputeProjectionExample.java       |    6 +-
 .../examples/compute/ComputeReducerExample.java |    4 +-
 .../compute/ComputeRunnableExample.java         |    4 +-
 .../compute/ComputeScheduleExample.java         |    8 +-
 .../examples/compute/ComputeTaskMapExample.java |    4 +-
 .../compute/ComputeTaskSplitExample.java        |    4 +-
 .../failover/ComputeFailoverExample.java        |    6 +-
 .../failover/ComputeFailoverNodeStartup.java    |    8 +-
 .../compute/montecarlo/CreditRiskExample.java   |    4 +-
 .../examples/datagrid/CacheAffinityExample.java |   12 +-
 .../examples/datagrid/CacheApiExample.java      |   10 +-
 .../datagrid/CacheContinuousQueryExample.java   |    4 +-
 .../datagrid/CacheDataLoaderExample.java        |    2 +-
 .../examples/datagrid/CacheEventsExample.java   |    4 +-
 .../examples/datagrid/CacheNodeStartup.java     |    4 +-
 .../datagrid/CachePopularNumbersExample.java    |   10 +-
 .../examples/datagrid/CachePutGetExample.java   |   10 +-
 .../examples/datagrid/CacheQueryExample.java    |   34 +-
 .../datagrid/CacheTransactionExample.java       |    8 +-
 .../datastructures/CacheAtomicLongExample.java  |    4 +-
 .../CacheAtomicReferenceExample.java            |    8 +-
 .../CacheAtomicSequenceExample.java             |    8 +-
 .../CacheAtomicStampedExample.java              |    8 +-
 .../CacheCountDownLatchExample.java             |    4 +-
 .../datastructures/CacheQueueExample.java       |   24 +-
 .../datastructures/CacheSetExample.java         |   20 +-
 .../hibernate/HibernateL2CacheExample.java      |    4 +-
 .../HibernateL2CacheExampleNodeStartup.java     |    8 +-
 .../starschema/CacheStarSchemaExample.java      |   18 +-
 .../store/CacheNodeWithStoreStartup.java        |    8 +-
 .../datagrid/store/CacheStoreExample.java       |    2 +-
 .../store/CacheStoreLoadDataExample.java        |    2 +-
 .../store/dummy/CacheDummyPersonStore.java      |    8 +-
 .../hibernate/CacheHibernatePersonStore.java    |   24 +-
 .../store/jdbc/CacheJdbcPersonStore.java        |   34 +-
 .../gridgain/examples/events/EventsExample.java |    8 +-
 .../org/gridgain/examples/ggfs/GgfsExample.java |   30 +-
 .../examples/ggfs/GgfsMapReduceExample.java     |    6 +-
 .../gridgain/examples/ggfs/GgfsNodeStartup.java |    4 +-
 .../examples/messaging/MessagingExample.java    |   10 +-
 .../messaging/MessagingPingPongExample.java     |    8 +-
 .../MessagingPingPongListenActorExample.java    |    8 +-
 .../MemcacheRestExampleNodeStartup.java         |   24 +-
 .../misc/deployment/DeploymentExample.java      |    8 +-
 .../misc/lifecycle/LifecycleExample.java        |    4 +-
 .../misc/springbean/SpringBeanExample.java      |    2 +-
 .../streaming/StreamingCheckInExample.java      |   18 +-
 .../streaming/StreamingNodeStartup.java         |    4 +-
 .../StreamingPopularNumbersExample.java         |   10 +-
 .../streaming/StreamingPriceBarsExample.java    |   12 +-
 .../StreamingRunningAverageExample.java         |    6 +-
 .../org/gridgain/examples/ComputeExample.java   |    2 +-
 .../org/gridgain/examples/MessagingExample.java |   10 +-
 .../examples/ScalarCacheAffinityExample1.scala  |    7 +-
 .../examples/ScalarCacheAffinityExample2.scala  |    1 +
 .../ScalarCachePopularNumbersExample.scala      |    6 +-
 .../examples/ScalarCacheQueryExample.scala      |    5 +-
 .../scalar/examples/ScalarClosureExample.scala  |    1 +
 ..._util_ipc_shmem_GridIpcSharedMemoryUtils.cpp |    4 +-
 .../aop/aspectj/GridifyAspectJAspect.java       |    4 +-
 .../aspectj/GridifySetToSetAspectJAspect.java   |    2 +-
 .../aspectj/GridifySetToValueAspectJAspect.java |    2 +-
 .../aop/spring/GridifySetToSetSpringAspect.java |    2 +-
 .../spring/GridifySetToValueSpringAspect.java   |    2 +-
 .../gridify/aop/spring/GridifySpringAspect.java |    4 +-
 .../grid/gridify/GridBasicAopSelfTest.java      |    8 +-
 .../grid/gridify/GridTestGridifyJob.java        |    2 +-
 .../grid/gridify/GridTestGridifyTask.java       |    6 +-
 .../GridSingleSplitTestJobTarget.java           |    6 +-
 .../singlesplit/GridSingleSplitTestTask.java    |    8 +-
 .../singlesplit/GridSingleSplitsLoadTest.java   |    2 +-
 .../loadtests/gridify/GridifyLoadTestTask.java  |    8 +-
 .../test/gridify/GridExternalGridifyTask.java   |    4 +-
 .../spi/checkpoint/s3/GridS3CheckpointSpi.java  |   14 +-
 .../s3/GridS3CheckpointSpiSelfTest.java         |   13 +-
 ...GridClientAbstractMultiThreadedSelfTest.java |    4 +-
 .../client/GridClientGetAffinityTask.java       |    4 +-
 .../org/gridgain/client/GridClientHttpTask.java |    8 +-
 .../gridgain/client/GridClientNodeStartup.java  |    4 +-
 .../client/GridClientPortableArgumentTask.java  |    4 +-
 .../client/GridClientPutPortableTask.java       |    2 +-
 .../client/GridClientReconnectionSelfTest.java  |    6 +-
 .../client/GridClientSslNodeStartup.java        |    4 +-
 .../client/GridClientStartNodeTask.java         |   10 +-
 .../gridgain/client/GridClientStopNodeTask.java |    6 +-
 .../client/GridClientStringLengthTask.java      |    8 +-
 .../org/gridgain/client/GridClientTcpTask.java  |    8 +-
 .../GridClientTestPortableAffinityKeyTask.java  |   16 +-
 .../client/GridClientTestRestServer.java        |    6 +-
 .../org/gridgain/client/GridHashMapStore.java   |   10 +-
 .../org/gridgain/client/GridSleepTestTask.java  |    8 +-
 .../client/GridTaskSingleJobSplitAdapter.java   |   16 +-
 .../GridClientAbstractMultiNodeSelfTest.java    |    4 +-
 .../integration/GridClientAbstractSelfTest.java |   34 +-
 .../GridClientPreferDirectSelfTest.java         |    4 +-
 .../router/GridClientFailedInitSelfTest.java    |    8 +-
 .../router/GridTcpRouterAbstractSelfTest.java   |    6 +-
 .../router/GridTcpRouterMultiNodeSelfTest.java  |    6 +-
 .../client/router/GridTcpSslRouterSelfTest.java |    3 +-
 .../kernal/GridTaskEventSubjectIdSelfTest.java  |   16 +-
 .../rest/GridRestBinaryProtocolSelfTest.java    |   20 +-
 .../rest/GridRestMemcacheProtocolSelfTest.java  |    5 +-
 .../processors/rest/GridRestProcessorTest.java  |   12 +-
 .../rest/GridTaskCommandHandlerSelfTest.java    |    4 +-
 .../processors/rest/GridTestBinaryClient.java   |   84 +-
 .../processors/rest/GridTestMemcacheClient.java |   94 +-
 .../grid/kernal/processors/rest/TestTask1.java  |    6 +-
 .../grid/kernal/processors/rest/TestTask2.java  |    6 +-
 .../jdbc/GridJdbcLocalCachesSelfTest.java       |   10 +-
 .../client/GridClientCacheBenchmark.java        |    4 +-
 .../clients/src/test/resources/spring-cache.xml |   29 +-
 .../src/main/java/org/apache/ignite/Ignite.java |   26 +-
 .../java/org/apache/ignite/IgniteCache.java     |  300 +++
 .../apache/ignite/IgniteCheckedException.java   |  108 +
 .../java/org/apache/ignite/IgniteCluster.java   |   32 +-
 .../java/org/apache/ignite/IgniteCompute.java   |   72 +-
 .../org/apache/ignite/IgniteDataLoader.java     |   32 +-
 .../java/org/apache/ignite/IgniteEvents.java    |   20 +-
 .../java/org/apache/ignite/IgniteException.java |    3 +-
 .../main/java/org/apache/ignite/IgniteFs.java   |   80 +-
 .../java/org/apache/ignite/IgniteManaged.java   |   30 +-
 .../java/org/apache/ignite/IgniteMessaging.java |   20 +-
 .../java/org/apache/ignite/IgniteScheduler.java |    2 +-
 .../java/org/apache/ignite/IgniteStreamer.java  |   16 +-
 .../org/apache/ignite/IgniteTransactions.java   |    8 +-
 .../main/java/org/apache/ignite/Ignition.java   |   16 +-
 .../apache/ignite/cache/CacheConfiguration.java |   23 +
 .../apache/ignite/cache/CacheEntryEvent.java    |   36 +
 .../java/org/apache/ignite/cache/CacheFlag.java |   68 +
 .../org/apache/ignite/cache/CachePeekMode.java  |   72 +
 .../ignite/cache/eviction/EvictableEntry.java   |   39 +
 .../cache/query/QueryAffinityPredicate.java     |  124 +
 .../cache/query/QueryContinuousPredicate.java   |  204 ++
 .../apache/ignite/cache/query/QueryCursor.java  |   29 +
 .../ignite/cache/query/QueryPredicate.java      |   68 +
 .../apache/ignite/cache/query/QueryReducer.java |   22 +
 .../ignite/cache/query/QuerySqlPredicate.java   |  107 +
 .../ignite/cache/query/QueryTextPredicate.java  |   79 +
 .../query/annotations/QueryGroupIndex.java      |   50 +
 .../cache/query/annotations/QuerySqlField.java  |  125 +
 .../query/annotations/QuerySqlFunction.java     |   59 +
 .../cache/query/annotations/QueryTextField.java |   25 +
 .../org/apache/ignite/cluster/ClusterGroup.java |    4 +-
 .../cluster/ClusterTopologyException.java       |    4 +-
 .../ComputeExecutionRejectedException.java      |    4 +-
 .../org/apache/ignite/compute/ComputeJob.java   |    9 +-
 .../compute/ComputeJobContinuationAdapter.java  |    6 +-
 .../compute/ComputeJobFailoverException.java    |    4 +-
 .../compute/ComputeJobMasterLeaveAware.java     |    6 +-
 .../apache/ignite/compute/ComputeJobResult.java |    6 +-
 .../ignite/compute/ComputeJobSibling.java       |    5 +-
 .../ignite/compute/ComputeLoadBalancer.java     |   11 +-
 .../org/apache/ignite/compute/ComputeTask.java  |   22 +-
 .../ignite/compute/ComputeTaskAdapter.java      |   13 +-
 .../compute/ComputeTaskCancelledException.java  |    4 +-
 .../compute/ComputeTaskContinuousMapper.java    |   18 +-
 .../ignite/compute/ComputeTaskFuture.java       |    8 +-
 .../ignite/compute/ComputeTaskSession.java      |   42 +-
 .../ignite/compute/ComputeTaskSplitAdapter.java |   16 +-
 .../compute/ComputeTaskTimeoutException.java    |    4 +-
 .../compute/ComputeUserUndeclaredException.java |    4 +-
 .../apache/ignite/compute/gridify/Gridify.java  |    2 +-
 .../compute/gridify/GridifyInterceptor.java     |    7 +-
 .../gridify/GridifyRuntimeException.java        |    4 +-
 .../gridify/aop/GridifyDefaultRangeTask.java    |    8 +-
 .../compute/gridify/aop/GridifyDefaultTask.java |    4 +-
 .../aop/GridifySetToSetAbstractAspect.java      |   24 +-
 .../aop/GridifySetToValueAbstractAspect.java    |   28 +-
 .../configuration/GridQueryConfiguration.java   |  194 ++
 .../configuration/IgniteAddressResolver.java    |    6 +-
 .../configuration/IgniteConfiguration.java      |   36 +-
 .../dataload/IgniteDataLoadCacheUpdater.java    |    6 +-
 .../events/IgniteCacheQueryReadEvent.java       |    6 +-
 .../org/apache/ignite/fs/IgniteFsException.java |    4 +-
 .../apache/ignite/fs/IgniteFsFileSystem.java    |   58 +-
 .../IgniteFsInputStreamJobAdapter.java          |    6 +-
 .../apache/ignite/fs/mapreduce/IgniteFsJob.java |    4 +-
 .../fs/mapreduce/IgniteFsRecordResolver.java    |    4 +-
 .../ignite/fs/mapreduce/IgniteFsTask.java       |   14 +-
 .../IgniteFsByteDelimiterRecordResolver.java    |    2 +-
 .../IgniteFsFixedLengthRecordResolver.java      |    2 +-
 .../ignite/lang/IgniteAsyncSupportAdapter.java  |    6 +-
 .../org/apache/ignite/lang/IgniteFuture.java    |   33 +-
 .../lang/IgniteFutureCancelledException.java    |    4 +-
 .../lang/IgniteFutureTimeoutException.java      |    4 +-
 .../apache/ignite/lifecycle/LifecycleAware.java |   10 +-
 .../apache/ignite/lifecycle/LifecycleBean.java  |    6 +-
 .../ignite/logger/java/IgniteJavaLogger.java    |    2 +-
 .../java/IgniteJavaLoggerFileHandler.java       |    6 +-
 .../marshaller/IgniteAbstractMarshaller.java    |    8 +-
 .../ignite/marshaller/IgniteMarshaller.java     |   18 +-
 .../marshaller/jdk/IgniteJdkMarshaller.java     |   12 +-
 .../optimized/IgniteOptimizedMarshaller.java    |   42 +-
 .../IgniteOptimizedObjectStreamRegistry.java    |    5 +-
 .../optimized-classnames.previous.properties    |    2 +-
 .../optimized/optimized-classnames.properties   |    2 +-
 .../ignite/messaging/MessagingListenActor.java  |   18 +-
 .../apache/ignite/plugin/PluginProvider.java    |   14 +-
 .../ignite/plugin/security/GridSecurity.java    |    6 +-
 .../GridSecurityCredentialsBasicProvider.java   |    4 +-
 .../GridSecurityCredentialsProvider.java        |    6 +-
 .../plugin/security/GridSecurityException.java  |    4 +-
 .../segmentation/GridSegmentationResolver.java  |    6 +-
 .../ignite/portables/PortableException.java     |    4 +-
 .../ignite/portables/PortableRawReader.java     |   13 +
 .../ignite/portables/PortableRawWriter.java     |   13 +
 .../apache/ignite/portables/PortableReader.java |   15 +
 .../apache/ignite/portables/PortableWriter.java |   15 +
 .../product/IgniteProductLicenseException.java  |    4 +-
 .../ignite/product/IgniteProductVersion.java    |    4 +-
 .../ignite/scheduler/SchedulerFuture.java       |   25 +-
 .../org/apache/ignite/spi/IgniteSpiAdapter.java |   26 +-
 .../ignite/spi/IgniteSpiCloseableIterator.java  |    6 +-
 .../org/apache/ignite/spi/IgniteSpiContext.java |   58 +-
 .../apache/ignite/spi/IgniteSpiException.java   |    4 +-
 .../AuthenticationAclProvider.java              |    6 +-
 .../checkpoint/cache/CacheCheckpointSpi.java    |    6 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |    8 +-
 .../sharedfs/SharedFsTimeoutTask.java           |    2 +-
 .../spi/checkpoint/sharedfs/SharedFsUtils.java  |    8 +-
 .../jobstealing/JobStealingDisabled.java        |    2 +-
 .../PriorityQueueCollisionSpi.java              |    4 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   92 +-
 .../DiscoverySpiNodeAuthenticator.java          |    8 +-
 .../discovery/tcp/TcpClientDiscoverySpi.java    |   17 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   40 +-
 .../discovery/tcp/TcpDiscoverySpiAdapter.java   |   18 +-
 .../TcpDiscoveryMulticastIpFinder.java          |   16 +-
 .../sharedfs/TcpDiscoverySharedFsIpFinder.java  |    2 +-
 .../tcp/ipfinder/vm/TcpDiscoveryVmIpFinder.java |    2 +-
 .../ignite/spi/failover/FailoverContext.java    |    6 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |    2 +-
 .../jobstealing/JobStealingFailoverSpi.java     |    2 +-
 .../spi/indexing/GridIndexingQueryFilter.java   |   27 +
 .../ignite/spi/indexing/GridIndexingSpi.java    |  104 +
 .../spi/indexing/GridNoopIndexingSpi.java       |   58 +
 .../ignite/spi/indexing/IndexDescriptor.java    |   42 -
 .../apache/ignite/spi/indexing/IndexType.java   |   24 -
 .../ignite/spi/indexing/IndexingEntity.java     |   50 -
 .../spi/indexing/IndexingEntityAdapter.java     |   57 -
 .../spi/indexing/IndexingFieldMetadata.java     |   46 -
 .../spi/indexing/IndexingFieldsResult.java      |   35 -
 .../indexing/IndexingFieldsResultAdapter.java   |   49 -
 .../spi/indexing/IndexingKeyValueRow.java       |   42 -
 .../indexing/IndexingKeyValueRowAdapter.java    |   82 -
 .../ignite/spi/indexing/IndexingMarshaller.java |   38 -
 .../spi/indexing/IndexingQueryFilter.java       |   27 -
 .../apache/ignite/spi/indexing/IndexingSpi.java |  188 --
 .../spi/indexing/IndexingTypeDescriptor.java    |   84 -
 .../ignite/spi/indexing/NoopIndexingSpi.java    |  119 -
 .../spi/loadbalancing/LoadBalancingSpi.java     |   12 +-
 .../adaptive/AdaptiveLoadBalancingSpi.java      |   18 +-
 .../RoundRobinGlobalLoadBalancer.java           |   18 +-
 .../roundrobin/RoundRobinLoadBalancingSpi.java  |    8 +-
 .../WeightedRandomLoadBalancingSpi.java         |    6 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   18 +-
 .../apache/ignite/streamer/StreamerContext.java |   26 +-
 .../apache/ignite/streamer/StreamerStage.java   |    6 +-
 .../apache/ignite/streamer/StreamerWindow.java  |   46 +-
 .../ignite/streamer/index/StreamerIndex.java    |    2 +-
 .../streamer/index/StreamerIndexProvider.java   |   10 +-
 .../index/StreamerIndexProviderAdapter.java     |   26 +-
 .../streamer/index/StreamerIndexUpdater.java    |    6 +-
 .../index/hash/StreamerHashIndexProvider.java   |    8 +-
 .../index/tree/StreamerTreeIndexProvider.java   |   16 +-
 .../window/StreamerBoundedSizeBatchWindow.java  |    9 +-
 .../StreamerBoundedSizeWindowAdapter.java       |    6 +-
 .../window/StreamerBoundedTimeBatchWindow.java  |   11 +-
 .../window/StreamerBoundedTimeWindow.java       |   10 +-
 .../streamer/window/StreamerWindowAdapter.java  |   40 +-
 .../gridgain/client/impl/GridClientImpl.java    |    4 +-
 .../GridClientConnectionManagerAdapter.java     |    8 +-
 .../connection/GridClientNioTcpConnection.java  |    3 +-
 .../GridClientOptimizedMarshaller.java          |    9 +-
 .../client/router/GridRouterFactory.java        |    6 +-
 .../impl/GridRouterCommandLineStartup.java      |    8 +-
 .../client/router/impl/GridTcpRouterImpl.java   |   16 +-
 .../router/impl/GridTcpRouterNioParser.java     |    6 +-
 .../grid/GridAuthenticationException.java       |    4 +-
 .../gridgain/grid/GridBasicWarmupClosure.java   |    4 +-
 .../gridgain/grid/GridDeploymentException.java  |    4 +-
 .../java/org/gridgain/grid/GridException.java   |  108 -
 .../gridgain/grid/GridInterruptedException.java |    6 +-
 .../org/gridgain/grid/GridRuntimeException.java |   85 -
 .../java/org/gridgain/grid/cache/GridCache.java |   26 +-
 .../GridCacheAtomicUpdateTimeoutException.java  |    4 +-
 .../grid/cache/GridCacheConfiguration.java      |   40 +
 .../org/gridgain/grid/cache/GridCacheEntry.java |   77 +-
 .../grid/cache/GridCacheFlagException.java      |    4 +-
 .../cache/GridCachePartialUpdateException.java  |    4 +-
 .../grid/cache/GridCacheProjection.java         |  150 +-
 .../org/gridgain/grid/cache/GridCacheTx.java    |   13 +-
 .../cache/GridCacheTxHeuristicException.java    |    4 +-
 .../cache/GridCacheTxOptimisticException.java   |    4 +-
 .../cache/GridCacheTxRollbackException.java     |    4 +-
 .../grid/cache/GridCacheTxTimeoutException.java |    4 +-
 .../GridCacheRendezvousAffinityFunction.java    |    9 +-
 .../grid/cache/cloner/GridCacheBasicCloner.java |    4 +-
 .../grid/cache/cloner/GridCacheCloner.java      |    6 +-
 .../grid/cache/cloner/GridCacheDeepCloner.java  |    6 +-
 .../datastructures/GridCacheAtomicLong.java     |   38 +-
 .../GridCacheAtomicReference.java               |   14 +-
 .../datastructures/GridCacheAtomicSequence.java |   22 +-
 .../datastructures/GridCacheAtomicStamped.java  |   22 +-
 .../datastructures/GridCacheCountDownLatch.java |   25 +-
 .../GridCacheDataStructureInvalidException.java |    4 +-
 ...cheDataStructureInvalidRuntimeException.java |    4 +-
 .../GridCacheDataStructureRemovedException.java |    4 +-
 ...cheDataStructureRemovedRuntimeException.java |    4 +-
 .../datastructures/GridCacheDataStructures.java |   62 +-
 .../cache/datastructures/GridCacheQueue.java    |   60 +-
 .../grid/cache/datastructures/GridCacheSet.java |    8 +-
 .../fifo/GridCacheFifoEvictionPolicy.java       |    3 +-
 .../GridCacheGgfsPerBlockLruEvictionPolicy.java |    8 +-
 .../random/GridCacheRandomEvictionPolicy.java   |    3 +-
 .../cache/query/GridCacheContinuousQuery.java   |   14 +-
 .../grid/cache/query/GridCacheQuery.java        |    6 +-
 .../query/GridCacheQueryConfiguration.java      |  117 +
 .../grid/cache/query/GridCacheQueryFuture.java  |   14 +-
 .../cache/query/GridCacheQuerySqlField.java     |    2 +-
 .../cache/query/GridCacheQuerySqlFunction.java  |    4 +-
 .../grid/cache/query/GridCacheQueryType.java    |    5 +-
 .../store/GridCacheLoadOnlyStoreAdapter.java    |   20 +-
 .../grid/cache/store/GridCacheStore.java        |   58 +-
 .../grid/cache/store/GridCacheStoreAdapter.java |   16 +-
 .../store/GridCacheStoreBalancingWrapper.java   |   22 +-
 .../store/jdbc/GridCacheJdbcBlobStore.java      |   36 +-
 .../org/gridgain/grid/hadoop/GridHadoop.java    |   18 +-
 .../grid/hadoop/GridHadoopCounterWriter.java    |    6 +-
 .../org/gridgain/grid/hadoop/GridHadoopJob.java |   24 +-
 .../gridgain/grid/hadoop/GridHadoopJobInfo.java |    4 +-
 .../grid/hadoop/GridHadoopMapReducePlanner.java |    4 +-
 .../grid/hadoop/GridHadoopSerialization.java    |   14 +-
 .../gridgain/grid/hadoop/GridHadoopTask.java    |    5 +-
 .../grid/hadoop/GridHadoopTaskContext.java      |   26 +-
 .../grid/hadoop/GridHadoopTaskInput.java        |    6 +-
 .../grid/hadoop/GridHadoopTaskOutput.java       |    8 +-
 .../grid/kernal/ClusterGroupAdapter.java        |    2 +-
 .../org/gridgain/grid/kernal/GridComponent.java |   14 +-
 .../gridgain/grid/kernal/GridComponentType.java |   44 +-
 .../grid/kernal/GridEventConsumeHandler.java    |   19 +-
 .../org/gridgain/grid/kernal/GridGainEx.java    |  127 +-
 .../grid/kernal/GridInternalException.java      |    4 +-
 .../grid/kernal/GridJobContextImpl.java         |    4 +-
 .../grid/kernal/GridJobExecuteResponse.java     |   16 +-
 .../gridgain/grid/kernal/GridJobResultImpl.java |    9 +-
 .../grid/kernal/GridJobSessionImpl.java         |   22 +-
 .../grid/kernal/GridJobSiblingImpl.java         |    8 +-
 .../grid/kernal/GridJobSiblingsResponse.java    |    6 +-
 .../org/gridgain/grid/kernal/GridKernal.java    |  139 +-
 .../gridgain/grid/kernal/GridKernalContext.java |    8 +
 .../grid/kernal/GridKernalContextImpl.java      |   14 +-
 .../org/gridgain/grid/kernal/GridKillTask.java  |    8 +-
 .../gridgain/grid/kernal/GridLoggerProxy.java   |    4 +-
 .../grid/kernal/GridMessageListenHandler.java   |    7 +-
 .../grid/kernal/GridPluginComponent.java        |    8 +-
 .../grid/kernal/GridTaskFutureImpl.java         |    8 +-
 .../grid/kernal/GridTaskSessionImpl.java        |   40 +-
 .../grid/kernal/GridUpdateNotifier.java         |    6 +-
 .../grid/kernal/IgniteClusterAsyncImpl.java     |   18 +-
 .../gridgain/grid/kernal/IgniteComputeImpl.java |   36 +-
 .../gridgain/grid/kernal/IgniteEventsImpl.java  |   10 +-
 .../gridgain/grid/kernal/IgniteManagedImpl.java |   16 +-
 .../grid/kernal/IgniteMessagingImpl.java        |   10 +-
 .../kernal/executor/GridExecutorService.java    |   30 +-
 .../ggfs/common/GridGgfsControlResponse.java    |   20 +-
 .../kernal/ggfs/common/GridGgfsMarshaller.java  |   14 +-
 .../grid/kernal/managers/GridManager.java       |    7 +-
 .../kernal/managers/GridManagerAdapter.java     |   76 +-
 .../kernal/managers/GridNoopManagerAdapter.java |   10 +-
 .../checkpoint/GridCheckpointManager.java       |   20 +-
 .../collision/GridCollisionManager.java         |    8 +-
 .../managers/communication/GridIoManager.java   |   91 +-
 .../managers/deployment/GridDeployment.java     |   26 +-
 .../deployment/GridDeploymentClassLoader.java   |    8 +-
 .../deployment/GridDeploymentCommunication.java |   14 +-
 .../deployment/GridDeploymentLocalStore.java    |   11 +-
 .../deployment/GridDeploymentManager.java       |   20 +-
 .../GridDeploymentPerLoaderStore.java           |   10 +-
 .../GridDeploymentPerVersionStore.java          |    8 +-
 .../deployment/GridDeploymentStore.java         |   14 +-
 .../deployment/GridDeploymentStoreAdapter.java  |    4 +-
 .../discovery/GridDiscoveryManager.java         |   43 +-
 .../eventstorage/GridEventStorageManager.java   |   41 +-
 .../failover/GridFailoverContextImpl.java       |    9 +-
 .../managers/failover/GridFailoverManager.java  |    9 +-
 .../managers/indexing/GridIndexingManager.java  | 1790 +------------
 .../loadbalancer/GridLoadBalancerManager.java   |   26 +-
 .../securesession/GridSecureSessionManager.java |   10 +-
 .../os/GridOsSecureSessionManager.java          |    6 +-
 .../managers/security/GridSecurityImpl.java     |    8 +-
 .../managers/security/GridSecurityManager.java  |   20 +-
 .../security/os/GridOsSecurityManager.java      |    8 +-
 .../swapspace/GridSwapSpaceManager.java         |   88 +-
 .../grid/kernal/processors/GridProcessor.java   |    6 +-
 .../kernal/processors/GridProcessorAdapter.java |   14 +-
 .../affinity/GridAffinityAssignmentCache.java   |    4 +-
 .../affinity/GridAffinityProcessor.java         |   74 +-
 .../processors/affinity/GridAffinityUtils.java  |    8 +-
 .../processors/cache/GridCacheAdapter.java      |  388 +--
 .../cache/GridCacheAffinityManager.java         |    6 +-
 .../cache/GridCacheClearAllRunnable.java        |    6 +-
 .../cache/GridCacheConcurrentMap.java           |   12 +-
 .../processors/cache/GridCacheContext.java      |   74 +-
 .../GridCacheDefaultAffinityKeyMapper.java      |    4 +-
 .../cache/GridCacheDeploymentManager.java       |   30 +-
 .../processors/cache/GridCacheEntryEx.java      |  101 +-
 .../processors/cache/GridCacheEntryImpl.java    |   63 +-
 .../processors/cache/GridCacheEntryInfo.java    |   16 +-
 .../GridCacheEntryInfoCollectSwapListener.java  |    2 +-
 .../processors/cache/GridCacheEntrySet.java     |    5 +-
 .../cache/GridCacheEvictionEntry.java           |   50 +-
 .../cache/GridCacheEvictionManager.java         |   49 +-
 .../cache/GridCacheEvictionRequest.java         |   10 +-
 .../cache/GridCacheEvictionResponse.java        |    8 +-
 .../cache/GridCacheFilterEvaluationEntry.java   |   41 +-
 .../processors/cache/GridCacheGateway.java      |    8 +-
 .../cache/GridCacheIndexUpdateException.java    |    4 +-
 .../processors/cache/GridCacheIoManager.java    |   48 +-
 .../processors/cache/GridCacheIterator.java     |    6 +-
 .../processors/cache/GridCacheKeySet.java       |    5 +-
 .../processors/cache/GridCacheMBeanAdapter.java |    4 +-
 .../processors/cache/GridCacheManager.java      |   10 +-
 .../cache/GridCacheManagerAdapter.java          |   12 +-
 .../processors/cache/GridCacheMapAdapter.java   |   33 +-
 .../processors/cache/GridCacheMapEntry.java     |  168 +-
 .../processors/cache/GridCacheMessage.java      |   78 +-
 .../processors/cache/GridCacheMvccManager.java  |    4 +-
 .../GridCachePartitionExchangeManager.java      |   21 +-
 .../processors/cache/GridCachePreloader.java    |   10 +-
 .../cache/GridCachePreloaderAdapter.java        |    4 +-
 .../processors/cache/GridCacheProcessor.java    |   81 +-
 .../processors/cache/GridCacheProjectionEx.java |   54 +-
 .../cache/GridCacheProjectionImpl.java          |  116 +-
 .../processors/cache/GridCacheProxyImpl.java    |  116 +-
 .../cache/GridCacheSharedContext.java           |   12 +-
 .../cache/GridCacheSharedManager.java           |   10 +-
 .../cache/GridCacheSharedManagerAdapter.java    |   12 +-
 .../processors/cache/GridCacheStoreManager.java |  195 +-
 .../processors/cache/GridCacheSwapManager.java  |  180 +-
 .../processors/cache/GridCacheTtlManager.java   |    5 +-
 .../processors/cache/GridCacheTxAdapter.java    |   18 +-
 .../processors/cache/GridCacheTxEntry.java      |   20 +-
 .../kernal/processors/cache/GridCacheTxEx.java  |    6 +-
 .../processors/cache/GridCacheTxHandler.java    |   44 +-
 .../cache/GridCacheTxLocalAdapter.java          |  163 +-
 .../processors/cache/GridCacheTxLocalEx.java    |   14 +-
 .../processors/cache/GridCacheTxManager.java    |   32 +-
 .../processors/cache/GridCacheTxProxyImpl.java  |    9 +-
 .../kernal/processors/cache/GridCacheUtils.java |   42 +-
 .../cache/GridCacheValueCollection.java         |    8 +-
 .../cache/GridCacheVersionManager.java          |    6 +-
 .../cache/GridCacheWriteBehindStore.java        |   41 +-
 .../GridAtomicCacheQueueImpl.java               |   24 +-
 .../GridCacheAnnotationHelper.java              |   18 +-
 .../datastructures/GridCacheAtomicLongImpl.java |   45 +-
 .../GridCacheAtomicReferenceImpl.java           |   18 +-
 .../GridCacheAtomicSequenceImpl.java            |   24 +-
 .../GridCacheAtomicStampedImpl.java             |   26 +-
 .../GridCacheCountDownLatchImpl.java            |   20 +-
 .../GridCacheDataStructuresImpl.java            |   32 +-
 .../GridCacheDataStructuresManager.java         |  160 +-
 .../GridCacheDataStructuresProxy.java           |   32 +-
 .../datastructures/GridCacheQueueAdapter.java   |   62 +-
 .../datastructures/GridCacheQueueProxy.java     |  114 +-
 .../cache/datastructures/GridCacheSetImpl.java  |   42 +-
 .../cache/datastructures/GridCacheSetProxy.java |   57 +-
 .../GridTransactionalCacheQueueImpl.java        |   24 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |    4 +-
 ...dCachePessimisticCheckCommittedTxFuture.java |    4 +-
 ...achePessimisticCheckCommittedTxResponse.java |    6 +-
 .../distributed/GridCacheTxFinishSync.java      |    4 +-
 .../distributed/GridDistributedBaseMessage.java |    8 +-
 .../distributed/GridDistributedLockRequest.java |   12 +-
 .../GridDistributedLockResponse.java            |   12 +-
 .../GridDistributedTxFinishRequest.java         |    8 +-
 .../GridDistributedTxPrepareRequest.java        |    8 +-
 .../GridDistributedTxPrepareResponse.java       |    8 +-
 .../GridDistributedTxRemoteAdapter.java         |   34 +-
 .../GridDistributedUnlockRequest.java           |   10 +-
 .../GridPartitionedCacheEntryImpl.java          |   27 +-
 .../dht/GridClientPartitionTopology.java        |    2 +-
 .../dht/GridDhtAffinityAssignmentResponse.java  |    6 +-
 .../dht/GridDhtAssignmentFetchFuture.java       |    2 +-
 .../cache/distributed/dht/GridDhtCache.java     |    4 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |   38 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   12 +-
 .../distributed/dht/GridDhtCacheEntryImpl.java  |   18 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |    2 +-
 .../distributed/dht/GridDhtLocalPartition.java  |    8 +-
 .../distributed/dht/GridDhtLockFuture.java      |   12 +-
 .../distributed/dht/GridDhtLockRequest.java     |   18 +-
 .../distributed/dht/GridDhtLockResponse.java    |   10 +-
 .../dht/GridDhtPartitionTopology.java           |   10 +-
 .../dht/GridDhtPartitionTopologyImpl.java       |    8 +-
 .../distributed/dht/GridDhtTopologyFuture.java  |    6 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |   46 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |    8 +-
 .../distributed/dht/GridDhtTxFinishRequest.java |    8 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |   42 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |   28 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |   16 +-
 .../dht/GridDhtTxPrepareRequest.java            |   10 +-
 .../dht/GridDhtTxPrepareResponse.java           |    8 +-
 .../cache/distributed/dht/GridDhtTxRemote.java  |    6 +-
 .../distributed/dht/GridDhtUnlockRequest.java   |    8 +-
 .../dht/GridPartitionedGetFuture.java           |   28 +-
 .../dht/atomic/GridDhtAtomicCache.java          |   75 +-
 .../dht/atomic/GridDhtAtomicCacheEntry.java     |    4 +-
 .../dht/atomic/GridDhtAtomicUpdateFuture.java   |    4 +-
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |    8 +-
 .../dht/atomic/GridDhtAtomicUpdateResponse.java |   14 +-
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   10 +-
 .../dht/atomic/GridNearAtomicUpdateRequest.java |    8 +-
 .../atomic/GridNearAtomicUpdateResponse.java    |   14 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   27 +-
 .../colocated/GridDhtColocatedLockFuture.java   |   40 +-
 .../colocated/GridDhtDetachedCacheEntry.java    |   12 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |    4 +-
 .../dht/preloader/GridDhtForceKeysRequest.java  |    6 +-
 .../dht/preloader/GridDhtForceKeysResponse.java |    8 +-
 .../GridDhtPartitionDemandMessage.java          |    8 +-
 .../preloader/GridDhtPartitionDemandPool.java   |   14 +-
 .../GridDhtPartitionSupplyMessage.java          |   16 +-
 .../preloader/GridDhtPartitionSupplyPool.java   |    6 +-
 .../GridDhtPartitionsExchangeFuture.java        |   26 +-
 .../preloader/GridDhtPartitionsFullMessage.java |    8 +-
 .../GridDhtPartitionsSingleMessage.java         |    8 +-
 .../dht/preloader/GridDhtPreloader.java         |   10 +-
 .../distributed/near/GridNearAtomicCache.java   |   66 +-
 .../distributed/near/GridNearCacheAdapter.java  |   32 +-
 .../distributed/near/GridNearCacheEntry.java    |   18 +-
 .../distributed/near/GridNearGetFuture.java     |   29 +-
 .../distributed/near/GridNearGetRequest.java    |   12 +-
 .../distributed/near/GridNearGetResponse.java   |   10 +-
 .../distributed/near/GridNearLockFuture.java    |   28 +-
 .../distributed/near/GridNearLockRequest.java   |   16 +-
 .../distributed/near/GridNearLockResponse.java  |    8 +-
 .../near/GridNearTransactionalCache.java        |   20 +-
 .../near/GridNearTxFinishFuture.java            |    8 +-
 .../near/GridNearTxFinishResponse.java          |    6 +-
 .../cache/distributed/near/GridNearTxLocal.java |   58 +-
 .../near/GridNearTxPrepareFuture.java           |   24 +-
 .../near/GridNearTxPrepareResponse.java         |   10 +-
 .../distributed/near/GridNearTxRemote.java      |   20 +-
 .../processors/cache/dr/GridCacheDrManager.java |   20 +-
 .../cache/dr/os/GridOsCacheDrManager.java       |   12 +-
 .../cache/jta/GridCacheJtaManagerAdapter.java   |   10 +-
 .../cache/jta/GridCacheNoopJtaManager.java      |    6 +-
 .../processors/cache/local/GridLocalCache.java  |   10 +-
 .../cache/local/GridLocalLockFuture.java        |    6 +-
 .../processors/cache/local/GridLocalTx.java     |   22 +-
 .../cache/local/GridLocalTxFuture.java          |   12 +-
 .../local/atomic/GridLocalAtomicCache.java      |   72 +-
 .../GridCacheDistributedFieldsQueryFuture.java  |    8 +-
 .../query/GridCacheDistributedQueryFuture.java  |    5 +-
 .../query/GridCacheDistributedQueryManager.java |   35 +-
 .../query/GridCacheFieldsQueryErrorFuture.java  |    6 +-
 .../query/GridCacheLocalFieldsQueryFuture.java  |    8 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |    8 +-
 .../cache/query/GridCacheLocalQueryManager.java |   18 +-
 .../cache/query/GridCacheQueriesEx.java         |   13 +-
 .../cache/query/GridCacheQueriesImpl.java       |   25 +-
 .../cache/query/GridCacheQueriesProxy.java      |   16 +-
 .../cache/query/GridCacheQueryAdapter.java      |   12 +-
 .../cache/query/GridCacheQueryErrorFuture.java  |    6 +-
 .../query/GridCacheQueryFutureAdapter.java      |   46 +-
 .../cache/query/GridCacheQueryManager.java      |  514 ++--
 .../query/GridCacheQueryMetadataAware.java      |    4 +-
 .../cache/query/GridCacheQueryRequest.java      |   14 +-
 .../cache/query/GridCacheQueryResponse.java     |  143 +-
 .../cache/query/GridCacheQueryType.java         |    5 +
 .../cache/query/GridCacheSqlMetadata.java       |    3 +-
 .../GridCacheContinuousQueryAdapter.java        |   10 +-
 .../GridCacheContinuousQueryEntry.java          |   49 +-
 .../GridCacheContinuousQueryHandler.java        |   19 +-
 .../GridCacheContinuousQueryManager.java        |   12 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |    6 +-
 .../query/jdbc/GridCacheQueryJdbcTask.java      |   20 +-
 .../jdbc/GridCacheQueryJdbcValidationTask.java  |    4 +-
 .../transactions/IgniteTransactionsImpl.java    |    4 +-
 .../processors/clock/GridClockMessage.java      |    8 +-
 .../processors/clock/GridClockServer.java       |   16 +-
 .../clock/GridClockSyncProcessor.java           |   17 +-
 .../closure/GridClosureProcessor.java           |  100 +-
 .../continuous/GridContinuousHandler.java       |   14 +-
 .../continuous/GridContinuousProcessor.java     |   75 +-
 .../dataload/GridDataLoadCacheUpdaters.java     |   14 +-
 .../dataload/GridDataLoaderFuture.java          |    2 +-
 .../dataload/GridDataLoaderProcessor.java       |   14 +-
 .../dataload/IgniteDataLoaderImpl.java          |   56 +-
 .../dr/GridDrDataLoadCacheUpdater.java          |    2 +-
 .../processors/dr/GridRawVersionedEntry.java    |   10 +-
 .../processors/dr/GridVersionedEntry.java       |   10 +-
 .../email/GridEmailProcessorAdapter.java        |    9 +-
 .../processors/ggfs/GridGgfsAckMessage.java     |   12 +-
 .../processors/ggfs/GridGgfsAsyncImpl.java      |   66 +-
 .../ggfs/GridGgfsBlockLocationImpl.java         |    6 +-
 .../ggfs/GridGgfsCommunicationMessage.java      |   10 +-
 .../kernal/processors/ggfs/GridGgfsContext.java |   14 +-
 .../processors/ggfs/GridGgfsDataManager.java    |  114 +-
 .../processors/ggfs/GridGgfsDeleteMessage.java  |   12 +-
 .../processors/ggfs/GridGgfsDeleteWorker.java   |   14 +-
 .../grid/kernal/processors/ggfs/GridGgfsEx.java |   18 +-
 .../kernal/processors/ggfs/GridGgfsFileMap.java |   10 +-
 .../ggfs/GridGgfsFileWorkerBatch.java           |   19 +-
 .../processors/ggfs/GridGgfsFileWorkerTask.java |    6 +-
 .../ggfs/GridGgfsFragmentizerManager.java       |   35 +-
 .../kernal/processors/ggfs/GridGgfsHelper.java  |    6 +-
 .../processors/ggfs/GridGgfsHelperImpl.java     |    6 +-
 .../kernal/processors/ggfs/GridGgfsImpl.java    |  162 +-
 .../ggfs/GridGgfsInputStreamImpl.java           |   14 +-
 .../ggfs/GridGgfsInvalidRangeException.java     |    4 +-
 .../processors/ggfs/GridGgfsIpcHandler.java     |   28 +-
 .../kernal/processors/ggfs/GridGgfsJobImpl.java |    4 +-
 .../kernal/processors/ggfs/GridGgfsManager.java |   12 +-
 .../processors/ggfs/GridGgfsMetaManager.java    |  254 +-
 .../processors/ggfs/GridGgfsProcessor.java      |   42 +-
 .../kernal/processors/ggfs/GridGgfsServer.java  |   28 +-
 .../processors/ggfs/GridGgfsServerHandler.java  |    6 +-
 .../processors/ggfs/GridGgfsServerManager.java  |   14 +-
 .../processors/ggfs/GridNoopGgfsHelper.java     |    4 +-
 .../ggfs/IgniteFsOutputStreamAdapter.java       |   14 +-
 .../ggfs/IgniteFsOutputStreamImpl.java          |   24 +-
 .../hadoop/GridHadoopNoopProcessor.java         |   10 +-
 .../hadoop/GridHadoopProcessorAdapter.java      |   18 +-
 .../processors/interop/GridInteropAware.java    |   17 +-
 .../interop/GridInteropProcessor.java           |   16 +-
 .../processors/interop/GridInteropTarget.java   |   30 +-
 .../interop/os/GridOsInteropProcessor.java      |    9 +-
 .../kernal/processors/job/GridJobProcessor.java |   45 +-
 .../kernal/processors/job/GridJobWorker.java    |   22 +-
 .../jobmetrics/GridJobMetricsProcessor.java     |    8 +-
 .../offheap/GridOffHeapProcessor.java           |   42 +-
 .../processors/port/GridPortProcessor.java      |    5 +-
 .../query/GridQueryFieldMetadata.java           |   46 +
 .../processors/query/GridQueryFieldsResult.java |   34 +
 .../query/GridQueryFieldsResultAdapter.java     |   49 +
 .../query/GridQueryIndexDescriptor.java         |   42 +
 .../processors/query/GridQueryIndexType.java    |   24 +
 .../processors/query/GridQueryIndexing.java     |  162 ++
 .../processors/query/GridQueryProcessor.java    | 1673 ++++++++++++
 .../query/GridQueryTypeDescriptor.java          |   78 +
 .../resource/GridResourceBasicInjector.java     |    6 +-
 .../resource/GridResourceCustomInjector.java    |   32 +-
 .../resource/GridResourceInjector.java          |   10 +-
 .../processors/resource/GridResourceIoc.java    |   12 +-
 .../GridResourceJobContextInjector.java         |    6 +-
 .../resource/GridResourceLoggerInjector.java    |    4 +-
 .../resource/GridResourceProcessor.java         |   72 +-
 .../resource/GridResourceServiceInjector.java   |    6 +-
 .../processors/resource/GridResourceUtils.java  |   18 +-
 .../resource/GridSpringResourceContext.java     |    8 +-
 .../processors/rest/GridRestProcessor.java      |   58 +-
 .../processors/rest/GridRestProtocol.java       |    6 +-
 .../rest/GridRestProtocolHandler.java           |    6 +-
 .../handlers/cache/GridCacheCommandHandler.java |   70 +-
 .../cache/GridCacheQueryCommandHandler.java     |   34 +-
 .../handlers/log/GridLogCommandHandler.java     |   20 +-
 .../metadata/GridPortableMetadataHandler.java   |    3 +-
 .../handlers/task/GridTaskCommandHandler.java   |   38 +-
 .../top/GridTopologyCommandHandler.java         |    4 +-
 .../rest/protocols/GridRestProtocolAdapter.java |    8 +-
 .../tcp/GridMemcachedMessageWrapper.java        |   14 +-
 .../tcp/GridTcpMemcachedNioListener.java        |    4 +-
 .../protocols/tcp/GridTcpRestDirectParser.java  |   18 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |    2 +-
 .../rest/protocols/tcp/GridTcpRestParser.java   |   34 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |    8 +-
 .../schedule/GridNoopScheduleProcessor.java     |    6 +-
 .../GridServiceMethodNotFoundException.java     |    4 +-
 .../service/GridServiceNotFoundException.java   |    4 +-
 .../service/GridServiceProcessor.java           |   57 +-
 .../processors/service/GridServiceProxy.java    |    6 +-
 .../session/GridTaskSessionProcessor.java       |    6 +-
 .../processors/spring/GridSpringProcessor.java  |    8 +-
 .../streamer/GridStreamProcessor.java           |   18 +-
 .../streamer/GridStreamerContextDelegate.java   |   14 +-
 .../streamer/GridStreamerContextImpl.java       |   14 +-
 .../GridStreamerRouteFailedException.java       |    4 +-
 .../GridStreamerStageExecutionFuture.java       |    4 +-
 .../processors/streamer/IgniteStreamerEx.java   |    4 +-
 .../processors/streamer/IgniteStreamerImpl.java |   82 +-
 .../streamer/StreamerStageWrapper.java          |    4 +-
 .../task/GridStreamerBroadcastTask.java         |    8 +-
 .../streamer/task/GridStreamerQueryTask.java    |    8 +-
 .../streamer/task/GridStreamerReduceTask.java   |    8 +-
 .../processors/task/GridTaskProcessor.java      |   51 +-
 .../kernal/processors/task/GridTaskWorker.java  |   70 +-
 .../timeout/GridTimeoutProcessor.java           |    5 +-
 .../gridgain/grid/kernal/visor/VisorJob.java    |    5 +-
 .../grid/kernal/visor/VisorMultiNodeTask.java   |   23 +-
 .../grid/kernal/visor/VisorOneNodeTask.java     |    4 +-
 .../grid/kernal/visor/cache/VisorCache.java     |    6 +-
 .../kernal/visor/cache/VisorCacheClearTask.java |    4 +-
 .../visor/cache/VisorCacheCompactTask.java      |    4 +-
 .../kernal/visor/cache/VisorCacheLoadTask.java  |    4 +-
 .../visor/cache/VisorCacheMetadataTask.java     |    6 +-
 .../cache/VisorCacheMetricsCollectorTask.java   |    6 +-
 .../visor/cache/VisorCachePreloadTask.java      |    4 +-
 .../visor/cache/VisorCacheResetMetricsTask.java |    4 +-
 .../visor/cache/VisorCacheSwapBackupsTask.java  |    4 +-
 .../compute/VisorComputeCancelSessionsTask.java |    4 +-
 .../compute/VisorComputeResetMetricsTask.java   |    4 +-
 .../VisorComputeToggleMonitoringTask.java       |    8 +-
 .../kernal/visor/debug/VisorThreadDumpTask.java |    4 +-
 .../kernal/visor/file/VisorFileBlockTask.java   |    4 +-
 .../visor/file/VisorLatestTextFilesTask.java    |    4 +-
 .../grid/kernal/visor/ggfs/VisorGgfs.java       |    4 +-
 .../kernal/visor/ggfs/VisorGgfsFormatTask.java  |    6 +-
 .../visor/ggfs/VisorGgfsProfilerClearTask.java  |    4 +-
 .../visor/ggfs/VisorGgfsProfilerTask.java       |    6 +-
 .../visor/ggfs/VisorGgfsResetMetricsTask.java   |    6 +-
 .../visor/ggfs/VisorGgfsSamplingStateTask.java  |    6 +-
 .../kernal/visor/log/VisorLogSearchTask.java    |    8 +-
 .../grid/kernal/visor/misc/VisorAckTask.java    |    6 +-
 .../visor/misc/VisorLatestVersionTask.java      |    4 +-
 .../grid/kernal/visor/misc/VisorNopTask.java    |   10 +-
 .../visor/misc/VisorResolveHostNameTask.java    |    6 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    4 +-
 .../visor/node/VisorNodeDataCollectorTask.java  |   12 +-
 .../node/VisorNodeEventsCollectorTask.java      |    6 +-
 .../grid/kernal/visor/node/VisorNodeGcTask.java |    6 +-
 .../kernal/visor/node/VisorNodePingTask.java    |    6 +-
 .../kernal/visor/node/VisorNodeRestartTask.java |    4 +-
 .../kernal/visor/node/VisorNodeStopTask.java    |    4 +-
 .../visor/node/VisorSpisConfiguration.java      |    2 +-
 .../VisorPortableMetadataCollectorTask.java     |    2 +-
 .../visor/query/VisorQueryCleanupTask.java      |   94 +
 .../visor/query/VisorQueryNextPageTask.java     |    8 +-
 .../grid/kernal/visor/query/VisorQueryTask.java |   26 +-
 .../kernal/visor/query/VisorQueryUtils.java     |    6 +-
 .../streamer/VisorStreamerMetricsResetTask.java |    4 +-
 .../visor/streamer/VisorStreamerResetTask.java  |    4 +-
 .../grid/kernal/visor/util/VisorTaskUtils.java  |   39 +-
 .../grid/util/GridAtomicInitializer.java        |    7 +-
 .../gridgain/grid/util/GridCircularBuffer.java  |   14 +-
 .../grid/util/GridCloseableIteratorAdapter.java |   26 +-
 .../util/GridCloseableIteratorAdapterEx.java    |   32 +-
 .../grid/util/GridEmptyCloseableIterator.java   |    4 +-
 .../gridgain/grid/util/GridReflectionCache.java |   14 +-
 .../util/GridSpiCloseableIteratorWrapper.java   |    8 +-
 .../java/org/gridgain/grid/util/GridUtils.java  |  220 +-
 .../gridgain/grid/util/GridWeakIterator.java    |    6 +-
 .../grid/util/future/GridCompoundFuture.java    |   10 +-
 .../grid/util/future/GridEmbeddedFuture.java    |   15 +-
 .../grid/util/future/GridFinishedFuture.java    |    6 +-
 .../grid/util/future/GridFinishedFutureEx.java  |    8 +-
 .../grid/util/future/GridFutureAdapter.java     |   12 +-
 .../grid/util/future/GridFutureAdapterEx.java   |   15 +-
 .../util/gridify/GridifyArgumentBuilder.java    |   14 +-
 .../grid/util/gridify/GridifyJobAdapter.java    |   18 +-
 .../gridgain/grid/util/ipc/GridIpcEndpoint.java |   10 +-
 .../util/ipc/GridIpcEndpointBindException.java  |    4 +-
 .../grid/util/ipc/GridIpcEndpointFactory.java   |   14 +-
 .../grid/util/ipc/GridIpcServerEndpoint.java    |   10 +-
 .../ipc/GridIpcServerEndpointDeserializer.java  |   12 +-
 .../grid/util/ipc/GridIpcToNioAdapter.java      |   24 +-
 .../ipc/loopback/GridIpcClientTcpEndpoint.java  |   18 +-
 .../ipc/loopback/GridIpcServerTcpEndpoint.java  |   16 +-
 .../GridIpcOutOfSystemResourcesException.java   |    4 +-
 .../GridIpcSharedMemoryClientEndpoint.java      |   14 +-
 .../shmem/GridIpcSharedMemoryInputStream.java   |    8 +-
 .../shmem/GridIpcSharedMemoryNativeLoader.java  |   14 +-
 ...cSharedMemoryOperationTimedoutException.java |    4 +-
 .../shmem/GridIpcSharedMemoryOutputStream.java  |    4 +-
 .../GridIpcSharedMemoryServerEndpoint.java      |   18 +-
 .../ipc/shmem/GridIpcSharedMemorySpace.java     |   38 +-
 .../ipc/shmem/GridIpcSharedMemoryUtils.java     |   30 +-
 .../grid/util/lang/GridAbsClosureX.java         |   12 +-
 .../grid/util/lang/GridAbsPredicateX.java       |   12 +-
 .../grid/util/lang/GridCloseableIterator.java   |    6 +-
 .../gridgain/grid/util/lang/GridClosure3X.java  |   12 +-
 .../grid/util/lang/GridClosureException.java    |    6 +-
 .../grid/util/lang/GridComputeJobWrapper.java   |    3 +-
 .../org/gridgain/grid/util/lang/GridFunc.java   |   49 +-
 .../grid/util/lang/GridInClosure3X.java         |   12 +-
 .../grid/util/lang/GridIterableAdapter.java     |    9 +-
 .../gridgain/grid/util/lang/GridIterator.java   |   14 +-
 .../grid/util/lang/GridIteratorAdapter.java     |   14 +-
 .../grid/util/lang/GridPlainAbsClosure.java     |    6 +-
 .../grid/util/lang/GridPlainClosure.java        |    6 +-
 .../grid/util/lang/GridPlainClosure2.java       |    6 +-
 .../grid/util/lang/GridPlainFuture.java         |   10 +-
 .../grid/util/lang/GridPlainFutureAdapter.java  |   22 +-
 .../grid/util/lang/GridPlainInClosure.java      |    6 +-
 .../grid/util/lang/GridPlainOutClosure.java     |    6 +-
 .../grid/util/lang/GridPredicate3X.java         |   12 +-
 .../grid/util/lang/IgniteClosure2X.java         |   12 +-
 .../gridgain/grid/util/lang/IgniteClosureX.java |   12 +-
 .../grid/util/lang/IgniteInClosure2X.java       |   12 +-
 .../grid/util/lang/IgniteInClosureX.java        |   12 +-
 .../grid/util/lang/IgniteOutClosureX.java       |   12 +-
 .../grid/util/lang/IgnitePredicate2X.java       |   12 +-
 .../grid/util/lang/IgnitePredicateX.java        |   12 +-
 .../grid/util/lang/IgniteReducer2X.java         |   12 +-
 .../grid/util/lang/IgniteReducer3X.java         |   12 +-
 .../gridgain/grid/util/lang/IgniteReducerX.java |   12 +-
 .../grid/util/nio/GridBufferedParser.java       |    6 +-
 .../grid/util/nio/GridCommunicationClient.java  |   20 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |   16 +-
 .../grid/util/nio/GridDirectParser.java         |    6 +-
 .../grid/util/nio/GridNioAsyncNotifyFilter.java |   26 +-
 .../grid/util/nio/GridNioCodecFilter.java       |   16 +-
 .../grid/util/nio/GridNioEmbeddedFuture.java    |    4 +-
 .../grid/util/nio/GridNioException.java         |    4 +-
 .../gridgain/grid/util/nio/GridNioFilter.java   |   82 +-
 .../grid/util/nio/GridNioFilterAdapter.java     |   26 +-
 .../grid/util/nio/GridNioFilterChain.java       |   50 +-
 .../grid/util/nio/GridNioFinishedFuture.java    |   10 +-
 .../gridgain/grid/util/nio/GridNioFuture.java   |   17 +-
 .../grid/util/nio/GridNioFutureImpl.java        |   15 +-
 .../gridgain/grid/util/nio/GridNioParser.java   |   10 +-
 .../gridgain/grid/util/nio/GridNioServer.java   |   74 +-
 .../grid/util/nio/GridNioServerBuffer.java      |    8 +-
 .../grid/util/nio/GridNioSessionImpl.java       |   10 +-
 .../util/nio/GridShmemCommunicationClient.java  |   20 +-
 .../util/nio/GridTcpCommunicationClient.java    |   30 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   21 +-
 .../grid/util/nio/ssl/GridNioSslFilter.java     |   22 +-
 .../grid/util/nio/ssl/GridNioSslHandler.java    |   14 +-
 .../grid/util/nodestart/GridNodeStartUtils.java |   52 +-
 .../grid/util/offheap/unsafe/GridUnsafeMap.java |   18 +-
 .../unsafe/GridUnsafePartitionedMap.java        |   17 +-
 .../grid/util/tostring/GridToStringBuilder.java |    4 +-
 .../java/org/gridgain/grid/util/typedef/X.java  |   16 +-
 .../grid/util/worker/GridWorkerFuture.java      |    4 +-
 .../grid/util/worker/GridWorkerPool.java        |    6 +-
 .../org/gridgain/jdbc/util/GridJdbcUtils.java   |    5 +-
 .../GridOptimizedMarshallerSelfTest.java        |   14 +-
 .../optimized/GridOptimizedMarshallerTest.java  |   52 +-
 .../GridOptimizedObjectStreamSelfTest.java      |    6 +-
 ...heckpointSpiMultipleDirectoriesSelfTest.java |    5 +-
 .../collision/GridTestCollisionTaskSession.java |   14 +-
 .../GridAbstractCommunicationSelfTest.java      |    2 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |    2 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |   20 +-
 .../tcp/GridTcpCommunicationSpiLanTest.java     |    4 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |    9 +-
 .../local/GridLocalDeploymentSpiSelfTest.java   |   10 +-
 ...GridTcpDiscoveryMarshallerCheckSelfTest.java |    4 +-
 .../discovery/tcp/GridTcpDiscoverySelfTest.java |   10 +-
 .../spi/failover/GridFailoverTestContext.java   |    4 +-
 ...RobinLoadBalancingSpiNotPerTaskSelfTest.java |    3 +-
 .../roundrobin/GridRoundRobinTestUtils.java     |    9 +-
 .../index/GridStreamerIndexSelfTest.java        |   31 +-
 .../window/GridStreamerWindowSelfTest.java      |   33 +-
 .../grid/GridExceptionHelpLinksSelfTest.java    |   37 +-
 .../grid/GridSuppressedExceptionSelfTest.java   |   39 +-
 .../java/org/gridgain/grid/GridTestJob.java     |    2 +-
 .../org/gridgain/grid/GridTestJobResult.java    |    7 +-
 .../java/org/gridgain/grid/GridTestTask.java    |    2 +-
 .../org/gridgain/grid/GridTestTaskSession.java  |   11 +-
 .../store/GridCacheBalancingStoreSelfTest.java  |   18 +-
 .../GridCacheLoadOnlyStoreAdapterSelfTest.java  |    4 +-
 .../cache/store/GridGeneratingTestStore.java    |   20 +-
 .../ggfs/GridGgfsEventsAbstractSelfTest.java    |    4 +-
 .../grid/ggfs/GridGgfsPathSelfTest.java         |    6 +-
 .../grid/kernal/GridAffinityMappedTest.java     |    4 +-
 .../grid/kernal/GridAffinitySelfTest.java       |    4 +-
 .../GridAlwaysFailoverSpiFailSelfTest.java      |   18 +-
 .../kernal/GridCacheProjectionRemoveTest.java   |    6 +-
 .../kernal/GridCancelOnGridStopSelfTest.java    |   10 +-
 .../kernal/GridCancelUnusedJobSelfTest.java     |    4 +-
 .../GridCancelledJobsMetricsSelfTest.java       |   10 +-
 .../grid/kernal/GridCommunicationSelfTest.java  |    2 +-
 .../GridContinuousJobAnnotationSelfTest.java    |   14 +-
 .../GridContinuousJobSiblingsSelfTest.java      |   10 +-
 .../grid/kernal/GridContinuousTaskSelfTest.java |   24 +-
 .../GridDeploymentMultiThreadedSelfTest.java    |    6 +-
 .../grid/kernal/GridDeploymentSelfTest.java     |   14 +-
 .../grid/kernal/GridDiscoveryEventSelfTest.java |    4 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |   10 +-
 ...ventStorageRuntimeConfigurationSelfTest.java |    6 +-
 .../grid/kernal/GridEventStorageSelfTest.java   |   10 +-
 .../GridExplicitImplicitDeploymentSelfTest.java |    8 +-
 .../grid/kernal/GridFactoryVmShutdownTest.java  |    4 +-
 .../GridFailoverCustomTopologySelfTest.java     |    8 +-
 .../grid/kernal/GridFailoverSelfTest.java       |   12 +-
 .../GridFailoverTaskWithPredicateSelfTest.java  |   12 +-
 .../kernal/GridFailoverTopologySelfTest.java    |   12 +-
 .../grid/kernal/GridHomePathSelfTest.java       |    4 +-
 .../GridJobCheckpointCleanupSelfTest.java       |    6 +-
 .../kernal/GridJobCollisionCancelSelfTest.java  |    2 +-
 .../grid/kernal/GridJobContextSelfTest.java     |    4 +-
 .../kernal/GridJobMasterLeaveAwareSelfTest.java |   42 +-
 .../grid/kernal/GridJobStealingSelfTest.java    |   26 +-
 .../GridJobStealingZeroActiveJobsSelfTest.java  |   12 +-
 .../grid/kernal/GridJobSubjectIdSelfTest.java   |    6 +-
 .../grid/kernal/GridLifecycleAwareSelfTest.java |   16 +-
 .../grid/kernal/GridLifecycleBeanSelfTest.java  |   14 +-
 .../grid/kernal/GridListenActorSelfTest.java    |    6 +-
 .../grid/kernal/GridManagementJobSelfTest.java  |    8 +-
 .../grid/kernal/GridMultipleJobsSelfTest.java   |    6 +-
 .../grid/kernal/GridMultipleSpisSelfTest.java   |   12 +-
 .../GridMultipleVersionsDeploymentSelfTest.java |    8 +-
 .../GridMultithreadedJobStealingSelfTest.java   |   10 +-
 .../kernal/GridNonHistoryMetricsSelfTest.java   |    2 +-
 .../grid/kernal/GridProjectionAbstractTest.java |    6 +-
 .../grid/kernal/GridReleaseTypeSelfTest.java    |    6 +-
 .../kernal/GridRuntimeExceptionSelfTest.java    |   14 +-
 .../grid/kernal/GridSpiExceptionSelfTest.java   |    6 +-
 .../grid/kernal/GridStopWithCancelSelfTest.java |    8 +-
 .../grid/kernal/GridStopWithWaitSelfTest.java   |   16 +-
 .../GridTaskCancelSingleNodeSelfTest.java       |    2 +-
 .../GridTaskContinuousMapperSelfTest.java       |   32 +-
 .../GridTaskExecutionContextSelfTest.java       |    8 +-
 .../grid/kernal/GridTaskFailoverSelfTest.java   |    2 +-
 .../GridTaskFutureImplStopGridSelfTest.java     |    4 +-
 .../GridTaskInstanceExecutionSelfTest.java      |    4 +-
 .../kernal/GridTaskInstantiationSelfTest.java   |    6 +-
 .../grid/kernal/GridTaskJobRejectSelfTest.java  |    2 +-
 .../grid/kernal/GridTaskListenerSelfTest.java   |    2 +-
 .../grid/kernal/GridTaskMapAsyncSelfTest.java   |    8 +-
 .../kernal/GridTaskNameAnnotationSelfTest.java  |    8 +-
 .../kernal/GridTaskResultCacheSelfTest.java     |   10 +-
 .../grid/kernal/GridTaskTimeoutSelfTest.java    |    6 +-
 .../managers/GridManagerStopSelfTest.java       |   26 +-
 .../managers/GridNoopManagerSelfTest.java       |    6 +-
 .../GridCheckpointManagerAbstractSelfTest.java  |   70 +-
 .../checkpoint/GridCheckpointTaskSelfTest.java  |   12 +-
 ...idCommunicationManagerListenersSelfTest.java |    6 +-
 .../communication/GridIoManagerSelfTest.java    |   10 +-
 .../GridDiscoveryManagerAttributesSelfTest.java |    4 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   24 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java |   28 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |   32 +-
 .../GridCacheAbstractIteratorsSelfTest.java     |   10 +-
 .../GridCacheAbstractProjectionSelfTest.java    |   40 +-
 .../GridCacheAbstractRemoveFailureTest.java     |    4 +-
 .../cache/GridCacheAbstractSelfTest.java        |   14 +-
 .../cache/GridCacheAbstractTtlSelfTest.java     |    4 +-
 .../cache/GridCacheAbstractTxReadTest.java      |   30 +-
 .../cache/GridCacheAffinityRoutingSelfTest.java |   10 +-
 .../cache/GridCacheBasicApiAbstractTest.java    |   12 +-
 .../cache/GridCacheBasicStoreAbstractTest.java  |   10 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |   26 +-
 ...idCacheConfigurationConsistencySelfTest.java |   50 +-
 .../cache/GridCacheEntryMemorySizeSelfTest.java |   14 +-
 .../cache/GridCacheGenericTestStore.java        |   24 +-
 .../GridCacheGroupLockAbstractSelfTest.java     |   28 +-
 .../GridCacheGroupLockFailoverSelfTest.java     |   16 +-
 .../cache/GridCacheGroupLockPutTask.java        |    8 +-
 .../cache/GridCacheLifecycleAwareSelfTest.java  |    8 +-
 .../cache/GridCacheMemoryModeSelfTest.java      |    6 +-
 .../cache/GridCacheNestedTxAbstractTest.java    |   19 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |    4 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |   10 +-
 .../processors/cache/GridCachePutAllTask.java   |    8 +-
 .../GridCacheQueryIndexingDisabledSelfTest.java |   26 +-
 .../GridCacheRefreshAheadAbstractSelfTest.java  |   16 +-
 .../processors/cache/GridCacheStopSelfTest.java |    6 +-
 .../cache/GridCacheStorePutxSelfTest.java       |   22 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |   16 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   38 +-
 .../processors/cache/GridCacheTestStore.java    |   32 +-
 .../cache/GridCacheTtlManagerSelfTest.java      |    8 +-
 .../cache/GridCacheTxAbstractTest.java          |    8 +-
 .../GridCacheTxExceptionAbstractSelfTest.java   |   66 +-
 .../cache/GridCacheTxMultiNodeAbstractTest.java |   52 +-
 .../GridCacheTxMultiThreadedAbstractTest.java   |   24 +-
 .../GridCacheTxSingleThreadedAbstractTest.java  |   24 +-
 ...idCacheTxStoreExceptionAbstractSelfTest.java |   32 +-
 .../GridCacheVariableTopologySelfTest.java      |    6 +-
 ...idCacheWriteBehindStoreAbstractSelfTest.java |    4 +-
 .../GridCacheWriteBehindStoreAbstractTest.java  |    6 +-
 .../GridCacheWriteBehindStoreSelfTest.java      |    6 +-
 .../GridCacheAtomicLongApiSelfTest.java         |   10 +-
 ...CacheAtomicReferenceApiSelfAbstractTest.java |   14 +-
 ...cheAtomicReferenceMultiNodeAbstractTest.java |   12 +-
 ...idCacheAtomicStampedApiSelfAbstractTest.java |    6 +-
 .../GridCacheMultiNodeDataStructureTest.java    |   10 +-
 .../GridCacheQueueApiSelfAbstractTest.java      |   16 +-
 ...ridCacheQueueJoinedNodeSelfAbstractTest.java |    4 +-
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |    8 +-
 ...CacheQueueRotativeMultiNodeAbstractTest.java |   14 +-
 .../GridCacheSequenceApiSelfAbstractTest.java   |   10 +-
 ...dCacheSequenceMultiNodeAbstractSelfTest.java |    6 +-
 .../GridCacheSetAbstractSelfTest.java           |    6 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |    6 +-
 ...titionedAtomicSequenceMultiThreadedTest.java |   30 +-
 ...dCachePartitionedQueueEntryMoveSelfTest.java |    6 +-
 .../GridCacheAbstractJobExecutionTest.java      |    2 +-
 .../GridCacheAtomicTimeoutSelfTest.java         |    4 +-
 .../GridCacheBasicOpAbstractTest.java           |    4 +-
 .../distributed/GridCacheEventAbstractTest.java |   52 +-
 .../GridCacheMultiNodeLockAbstractTest.java     |    4 +-
 ...dCacheMultithreadedFailoverAbstractTest.java |    4 +-
 .../GridCacheNodeFailureAbstractTest.java       |    6 +-
 .../GridCacheTxTimeoutAbstractTest.java         |   14 +-
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |    4 +-
 .../dht/GridCacheColocatedDebugTest.java        |    4 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |    2 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |    2 +-
 .../dht/GridCacheDhtInternalEntrySelfTest.java  |    8 +-
 .../dht/GridCacheDhtMultiBackupTest.java        |    2 +-
 .../dht/GridCacheDhtPreloadBigDataSelfTest.java |    6 +-
 .../GridCacheDhtPreloadDisabledSelfTest.java    |    4 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |    8 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |    8 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |    6 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  |    6 +-
 ...cheGroupLockPartitionedAbstractSelfTest.java |    4 +-
 ...ockPartitionedMultiNodeAbstractSelfTest.java |    2 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   10 +-
 ...GridCacheValueConsistencyAtomicSelfTest.java |    6 +-
 .../near/GridCacheGetStoreErrorSelfTest.java    |   14 +-
 .../near/GridCacheNearMultiGetSelfTest.java     |    2 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |    6 +-
 .../near/GridCacheNearOneNodeSelfTest.java      |   14 +-
 ...idCacheNearOnlyMultiNodeFullApiSelfTest.java |    6 +-
 .../GridCacheNearReaderPreloadSelfTest.java     |    2 +-
 ...rtitionedAffinityHashIdResolverSelfTest.java |   10 +-
 .../GridCachePartitionedAffinitySelfTest.java   |    4 +-
 ...ionedClientOnlyNoPrimaryFullApiSelfTest.java |    4 +-
 .../GridCachePartitionedLoadCacheSelfTest.java  |   14 +-
 .../near/GridCachePartitionedLockSelfTest.java  |   12 +-
 ...achePartitionedMultiNodeCounterSelfTest.java |    2 +-
 ...idCachePartitionedMultiNodeLockSelfTest.java |    6 +-
 ...ePartitionedMultiThreadedPutGetSelfTest.java |   12 +-
 ...achePartitionedPreloadLifecycleSelfTest.java |    4 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |   12 +-
 .../GridCachePartitionedStorePutSelfTest.java   |   14 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |    2 +-
 .../near/GridPartitionedBackupLoadSelfTest.java |   12 +-
 .../GridCacheReplicatedInvalidateSelfTest.java  |    6 +-
 ...CacheReplicatedPreloadLifecycleSelfTest.java |    4 +-
 .../GridCacheReplicatedPreloadSelfTest.java     |    2 +-
 .../GridCacheBatchEvictUnswapSelfTest.java      |    2 +-
 .../GridCacheDistributedEvictionsSelfTest.java  |    2 +-
 .../cache/eviction/GridCacheMockEntry.java      |   36 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |    4 +-
 .../GridCacheRandomEvictionPolicySelfTest.java  |    2 +-
 .../local/GridCacheDaemonNodeLocalSelfTest.java |    2 +-
 .../local/GridCacheLocalLoadAllSelfTest.java    |    8 +-
 .../cache/local/GridCacheLocalLockSelfTest.java |    4 +-
 .../local/GridCacheLocalTxTimeoutSelfTest.java  |   14 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |    8 +-
 .../clock/GridTimeSyncProcessorSelfTest.java    |    2 +-
 .../closure/GridClosureProcessorSelfTest.java   |   28 +-
 .../continuous/GridEventConsumeSelfTest.java    |    4 +-
 .../dataload/GridDataLoaderImplSelfTest.java    |   96 +-
 .../GridDataLoaderProcessorSelfTest.java        |    2 +-
 ...heGgfsPerBlockLruEvictionPolicySelfTest.java |    2 +-
 .../ggfs/GridGgfsAbstractSelfTest.java          |   60 +-
 .../ggfs/GridGgfsDualAbstractSelfTest.java      |   22 +-
 .../ggfs/GridGgfsFileInfoSelfTest.java          |   12 +-
 .../processors/ggfs/GridGgfsModesSelfTest.java  |   12 +-
 .../ggfs/GridGgfsProcessorSelfTest.java         |   12 +-
 .../GridGgfsProcessorValidationSelfTest.java    |   14 +-
 ...IpcEndpointRegistrationAbstractSelfTest.java |   12 +-
 ...pcEndpointRegistrationOnWindowsSelfTest.java |    6 +-
 .../ggfs/GridGgfsStreamsSelfTest.java           |    4 +-
 .../processors/ggfs/GridGgfsTaskSelfTest.java   |    6 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |   22 +-
 .../GridServiceProcessorAbstractSelfTest.java   |    6 +-
 .../GridServiceProcessorSingleNodeSelfTest.java |    2 +-
 .../GridServiceReassignmentSelfTest.java        |    2 +-
 .../streamer/GridStreamerEvictionSelfTest.java  |    8 +-
 .../streamer/GridStreamerSelfTest.java          |    8 +-
 .../processors/streamer/GridTestStage.java      |    4 +-
 .../gridgain/grid/lang/GridFuncSelfTest.java    |    4 +-
 .../lang/GridFutureListenPerformanceTest.java   |    4 +-
 .../org/gridgain/grid/lang/GridXSelfTest.java   |    4 +-
 .../grid/loadtest/GridSingleExecutionTest.java  |   36 +-
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |    6 +-
 .../marshaller/GridMarshallerAbstractTest.java  |   12 +-
 .../GridMarshallerPerformanceTest.java          |   16 +-
 .../p2p/GridAbstractMultinodeRedeployTest.java  |    2 +-
 .../grid/p2p/GridP2PJobClassLoaderSelfTest.java |    6 +-
 .../p2p/GridP2PLocalDeploymentSelfTest.java     |    8 +-
 .../GridP2PMissedResourceCacheSizeSelfTest.java |    8 +-
 .../grid/p2p/GridP2PRecursionTaskSelfTest.java  |    6 +-
 .../p2p/GridP2PRemoteClassLoadersSelfTest.java  |    6 +-
 .../org/gridgain/grid/p2p/GridP2PTestJob.java   |    2 +-
 .../org/gridgain/grid/p2p/GridP2PTestTask.java  |    4 +-
 .../grid/p2p/GridP2PTestTaskExecutionTest.java  |    4 +-
 .../grid/p2p/GridP2PTimeoutSelfTest.java        |    4 +-
 ...SessionCancelSiblingsFromFutureSelfTest.java |   10 +-
 ...ridSessionCancelSiblingsFromJobSelfTest.java |   12 +-
 ...idSessionCancelSiblingsFromTaskSelfTest.java |   10 +-
 .../GridSessionCheckpointAbstractSelfTest.java  |   10 +-
 .../GridSessionCollisionSpiSelfTest.java        |    6 +-
 ...idSessionFutureWaitJobAttributeSelfTest.java |   12 +-
 ...dSessionFutureWaitTaskAttributeSelfTest.java |   10 +-
 .../session/GridSessionJobFailoverSelfTest.java |   14 +-
 ...GridSessionJobWaitTaskAttributeSelfTest.java |   14 +-
 .../grid/session/GridSessionLoadSelfTest.java   |    6 +-
 .../GridSessionSetFutureAttributeSelfTest.java  |   14 +-
 ...nSetFutureAttributeWaitListenerSelfTest.java |   10 +-
 .../GridSessionSetJobAttribute2SelfTest.java    |    8 +-
 ...GridSessionSetJobAttributeOrderSelfTest.java |   10 +-
 .../GridSessionSetJobAttributeSelfTest.java     |   14 +-
 ...sionSetJobAttributeWaitListenerSelfTest.java |   10 +-
 .../GridSessionSetTaskAttributeSelfTest.java    |   14 +-
 ...GridSessionTaskWaitJobAttributeSelfTest.java |   14 +-
 .../GridSessionWaitAttributeSelfTest.java       |   14 +-
 .../grid/spi/GridSpiLocalHostInjectionTest.java |   44 +-
 .../grid/spi/GridTcpSpiForwardingSelfTest.java  |    2 +-
 .../grid/util/GridSnapshotLockSelfTest.java     |    6 +-
 .../util/future/GridCompoundFutureSelfTest.java |    6 +-
 .../util/future/GridEmbeddedFutureSelfTest.java |    4 +-
 .../util/future/GridFinishedFutureSelfTest.java |    4 +-
 .../util/future/GridFutureAdapterSelfTest.java  |   14 +-
 .../future/GridFutureListenPerformanceTest.java |    4 +-
 .../util/future/nio/GridNioFutureSelfTest.java  |   12 +-
 ...idIpcServerEndpointDeserializerSelfTest.java |    8 +-
 .../shmem/GridGgfsSharedMemoryTestServer.java   |    8 +-
 ...idIpcSharedMemoryCrashDetectionSelfTest.java |   14 +-
 .../shmem/GridIpcSharedMemorySpaceSelfTest.java |    2 +-
 .../GridIpcSharedMemoryBenchmarkReader.java     |   10 +-
 .../GridIpcSharedMemoryBenchmarkWriter.java     |    8 +-
 .../gridgain/grid/util/nio/GridNioSelfTest.java |   32 +-
 .../grid/util/nio/GridNioSslSelfTest.java       |    8 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   30 +-
 .../offheap/GridOffHeapMapAbstractSelfTest.java |   11 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |    4 +-
 .../loadtests/GridCacheLoadPopulationTask.java  |    6 +-
 .../cache/GridCacheAbstractLoadTest.java        |   24 +-
 ...ridCacheAffinityTransactionsOffHeapTest.java |    4 +-
 .../cache/GridCacheDataStructuresLoadTest.java  |   28 +-
 .../loadtests/cache/GridCacheLoadTest.java      |    4 +-
 .../loadtests/cache/GridCacheSwapLoadTest.java  |    8 +-
 .../GridCacheWriteBehindStoreLoadTest.java      |    8 +-
 .../colocation/GridTestCacheStore.java          |   12 +-
 .../colocation/GridTestLifecycleBean.java       |    2 +-
 .../loadtests/colocation/GridTestMain.java      |    6 +-
 .../loadtests/colocation/spring-colocation.xml  |   13 -
 .../communication/GridIoManagerBenchmark.java   |    9 +-
 .../communication/GridIoManagerBenchmark0.java  |   13 +-
 .../GridTcpCommunicationBenchmark.java          |   10 +-
 .../GridContinuousOperationsLoadTest.java       |    4 +-
 .../direct/multisplit/GridLoadTestJob.java      |    2 +-
 .../direct/multisplit/GridLoadTestTask.java     |    4 +-
 .../multisplit/GridMultiSplitsLoadTest.java     |    2 +-
 .../GridMultiSplitsRedeployLoadTest.java        |    8 +-
 .../GridSingleSplitNewNodesTestJobTarget.java   |    8 +-
 .../GridSingleSplitNewNodesTestTask.java        |    8 +-
 .../GridSingleSplitsRedeployLoadTest.java       |    2 +-
 .../direct/session/GridSessionLoadTestJob.java  |    2 +-
 .../direct/session/GridSessionLoadTestTask.java |    6 +-
 .../stealing/GridStealingLoadTestJob.java       |    2 +-
 .../stealing/GridStealingLoadTestTask.java      |    6 +-
 .../loadtests/discovery/GridGcTimeoutTest.java  |    4 +-
 .../gridgain/loadtests/dsi/GridDsiClient.java   |    2 +-
 .../loadtests/dsi/GridDsiLifecycleBean.java     |    8 +-
 .../gridgain/loadtests/dsi/GridDsiPerfJob.java  |   16 +-
 .../loadtests/dsi/GridDsiRequestTask.java       |    6 +-
 .../loadtests/dsi/GridDsiResponseTask.java      |    6 +-
 .../job/GridJobExecutionLoadTestClient.java     |    6 +-
 ...GridJobExecutionLoadTestClientSemaphore.java |    4 +-
 .../job/GridJobExecutionLoadTestJob.java        |    4 +-
 .../job/GridJobExecutionLoadTestTask.java       |    8 +-
 .../job/GridJobExecutionSingleNodeLoadTest.java |   12 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |    8 +-
 .../loadtests/job/GridJobLoadTestJob.java       |    4 +-
 .../loadtests/job/GridJobLoadTestSubmitter.java |    8 +-
 .../loadtests/job/GridJobLoadTestTask.java      |    8 +-
 .../mapper/GridContinuousMapperLoadTest1.java   |    4 +-
 .../mapper/GridContinuousMapperTask1.java       |   16 +-
 .../mapper/GridContinuousMapperTask2.java       |    6 +-
 .../loadtests/mapper/GridNodeStartup.java       |    4 +-
 .../mergesort/GridMergeSortLoadTask.java        |    2 +-
 .../mergesort/GridMergeSortLoadTest.java        |    8 +-
 .../loadtests/nio/GridNioBenchmarkTest.java     |    8 +-
 .../loadtests/streamer/EventClosure.java        |    2 +-
 .../streamer/GridStreamerIndexLoadTest.java     |    5 +-
 .../loadtests/streamer/average/TestStage.java   |    4 +-
 .../swap/GridSwapEvictAllBenchmark.java         |    4 +-
 .../loadtests/util/GridLoadTestArgs.java        |    8 +-
 .../startup/GridRandomCommandLineLoader.java    |   20 +-
 .../gridgain/startup/GridVmNodesStarter.java    |   24 +-
 .../gridgain/testframework/GridFileLock.java    |   16 +-
 .../testframework/GridGgfsTestUtils.java        |    8 +-
 .../testframework/GridSpiTestContext.java       |   36 +-
 .../gridgain/testframework/GridTestUtils.java   |   22 +-
 .../testframework/junits/GridAbstractTest.java  |   24 +-
 .../junits/GridTestKernalContext.java           |    8 +-
 .../testframework/junits/GridTestResources.java |   10 +-
 .../cache/GridAbstractCacheStoreSelfTest.java   |   30 +-
 .../junits/common/GridCommonAbstractTest.java   |   58 +-
 .../junits/logger/GridTestLog4jLogger.java      |   22 +-
 .../core/src/test/webapp/META-INF/gg-config.xml |   53 +-
 .../processors/email/GridEmailProcessor.java    |   20 +-
 .../tests/p2p/GridCacheDeploymentTestTask1.java |    6 +-
 .../tests/p2p/GridCacheDeploymentTestTask2.java |    4 +-
 .../tests/p2p/GridCacheDeploymentTestTask3.java |    6 +-
 .../p2p/GridP2PContinuousDeploymentTask1.java   |    6 +-
 .../p2p/GridP2PContinuousDeploymentTask2.java   |    5 +-
 .../p2p/GridP2PEventFilterExternalPath1.java    |    2 +-
 .../p2p/GridP2PEventFilterExternalPath2.java    |    2 +-
 .../tests/p2p/GridP2PTestTaskExternalPath1.java |    8 +-
 .../tests/p2p/GridP2PTestTaskExternalPath2.java |    8 +-
 .../grid/tests/p2p/GridSingleSplitTestTask.java |    5 +-
 .../grid/tests/p2p/JobStealingTask.java         |    8 +-
 .../GridUriDeploymentAbstractTestTask.java      |    5 +-
 .../tasks/GridUriDeploymentInnerTestTask.java   |    5 +-
 .../GridUriDeploymentNonePublicTestTask.java    |    5 +-
 .../uri/tasks/GridUriDeploymentTestTask0.java   |    5 +-
 .../uri/tasks/GridUriDeploymentTestTask1.java   |    5 +-
 .../uri/tasks/GridUriDeploymentTestTask2.java   |    5 +-
 .../uri/tasks/GridUriDeploymentTestTask3.java   |    5 +-
 .../uri/tasks/GridUriDeploymentTestTask4.java   |    5 +-
 .../uri/tasks/GridUriDeploymentTestTask5.java   |    5 +-
 .../uri/tasks/GridUriDeploymentTestTask6.java   |    5 +-
 .../GridUriDeploymentTestWithNameTask0.java     |    5 +-
 .../GridUriDeploymentTestWithNameTask3.java     |    5 +-
 .../GridUriDeploymentTestWithNameTask4.java     |    5 +-
 .../GridUriDeploymentTestWithNameTask5.java     |    5 +-
 .../GridUriDeploymentTestWithNameTask6.java     |    5 +-
 .../GridUriDeploymentTestWithNameTask7.java     |    5 +-
 .../client/hadoop/GridHadoopClientProtocol.java |    3 +-
 .../GridHadoopClientProtocolProvider.java       |    3 +-
 .../hadoop/v1/GridGgfsHadoopFileSystem.java     |    3 +-
 .../hadoop/v2/GridGgfsHadoopFileSystem.java     |    3 +-
 .../grid/hadoop/GridHadoopDefaultJobInfo.java   |    4 +-
 .../grid/kernal/ggfs/hadoop/GridGgfsHadoop.java |   65 +-
 .../GridGgfsHadoopCommunicationException.java   |    3 +-
 .../ggfs/hadoop/GridGgfsHadoopEndpoint.java     |   20 +-
 .../ggfs/hadoop/GridGgfsHadoopFSProperties.java |    7 +-
 .../hadoop/GridGgfsHadoopFileSystemWrapper.java |   41 +-
 .../ggfs/hadoop/GridGgfsHadoopInProc.java       |   33 +-
 .../ggfs/hadoop/GridGgfsHadoopInputStream.java  |   19 +-
 .../kernal/ggfs/hadoop/GridGgfsHadoopIo.java    |   13 +-
 .../kernal/ggfs/hadoop/GridGgfsHadoopIpcIo.java |   31 +-
 .../ggfs/hadoop/GridGgfsHadoopOutProc.java      |   43 +-
 .../ggfs/hadoop/GridGgfsHadoopOutputStream.java |    3 +-
 .../GridGgfsHadoopStreamEventListener.java      |    5 +-
 .../kernal/ggfs/hadoop/GridGgfsHadoopUtils.java |    5 +-
 .../ggfs/hadoop/GridGgfsHadoopWrapper.java      |   48 +-
 .../hadoop/GridHadoopClassLoader.java           |   11 +-
 .../processors/hadoop/GridHadoopComponent.java  |    4 +-
 .../processors/hadoop/GridHadoopImpl.java       |    9 +-
 .../processors/hadoop/GridHadoopProcessor.java  |   23 +-
 .../GridHadoopTaskCancelledException.java       |    3 +-
 .../processors/hadoop/GridHadoopUtils.java      |   27 +-
 .../hadoop/counter/GridHadoopCountersImpl.java  |    3 +-
 .../counter/GridHadoopFSCounterWriter.java      |    5 +-
 .../hadoop/jobtracker/GridHadoopJobTracker.java |   65 +-
 .../GridHadoopDefaultMapReducePlanner.java      |   14 +-
 .../GridHadoopProtocolJobCountersTask.java      |    3 +-
 .../proto/GridHadoopProtocolJobStatusTask.java  |    3 +-
 .../proto/GridHadoopProtocolKillJobTask.java    |    3 +-
 .../proto/GridHadoopProtocolNextTaskIdTask.java |    3 +-
 .../proto/GridHadoopProtocolSubmitJobTask.java  |    3 +-
 .../proto/GridHadoopProtocolTaskAdapter.java    |   12 +-
 .../hadoop/shuffle/GridHadoopShuffle.java       |   31 +-
 .../hadoop/shuffle/GridHadoopShuffleJob.java    |   36 +-
 .../shuffle/GridHadoopShuffleMessage.java       |    7 +-
 .../GridHadoopConcurrentHashMultimap.java       |   25 +-
 .../collections/GridHadoopHashMultimap.java     |    9 +-
 .../collections/GridHadoopHashMultimapBase.java |   15 +-
 .../shuffle/collections/GridHadoopMultimap.java |   19 +-
 .../collections/GridHadoopMultimapBase.java     |   21 +-
 .../shuffle/collections/GridHadoopSkipList.java |   35 +-
 .../GridHadoopEmbeddedTaskExecutor.java         |   11 +-
 .../taskexecutor/GridHadoopRunnableTask.java    |   22 +-
 .../GridHadoopTaskExecutorAdapter.java          |    9 +-
 .../GridHadoopExternalTaskExecutor.java         |   40 +-
 .../child/GridHadoopChildProcessRunner.java     |   22 +-
 .../child/GridHadoopExternalProcessStarter.java |    2 +-
 .../GridHadoopCommunicationClient.java          |    5 +-
 .../GridHadoopExternalCommunication.java        |   86 +-
 .../GridHadoopHandshakeTimeoutException.java    |    3 +-
 .../GridHadoopIpcToNioAdapter.java              |   20 +-
 .../GridHadoopMarshallerFilter.java             |   17 +-
 .../GridHadoopTcpNioCommunicationClient.java    |    7 +-
 .../hadoop/v1/GridHadoopV1CleanupTask.java      |    5 +-
 .../hadoop/v1/GridHadoopV1MapTask.java          |    5 +-
 .../hadoop/v1/GridHadoopV1OutputCollector.java  |    3 +-
 .../hadoop/v1/GridHadoopV1ReduceTask.java       |    5 +-
 .../hadoop/v1/GridHadoopV1SetupTask.java        |    5 +-
 .../hadoop/v1/GridHadoopV1Splitter.java         |   13 +-
 .../v2/GridHadoopSerializationWrapper.java      |   17 +-
 .../hadoop/v2/GridHadoopV2CleanupTask.java      |    5 +-
 .../hadoop/v2/GridHadoopV2Context.java          |    3 +-
 .../processors/hadoop/v2/GridHadoopV2Job.java   |   16 +-
 .../v2/GridHadoopV2JobResourceManager.java      |   20 +-
 .../hadoop/v2/GridHadoopV2MapTask.java          |    5 +-
 .../hadoop/v2/GridHadoopV2ReduceTask.java       |    5 +-
 .../hadoop/v2/GridHadoopV2SetupTask.java        |    5 +-
 .../hadoop/v2/GridHadoopV2Splitter.java         |   13 +-
 .../processors/hadoop/v2/GridHadoopV2Task.java  |   17 +-
 .../hadoop/v2/GridHadoopV2TaskContext.java      |   42 +-
 .../v2/GridHadoopWritableSerialization.java     |    9 +-
 .../grid/ggfs/GridGgfsEventsTestSuite.java      |   20 +-
 ...dGgfsHadoop20FileSystemAbstractSelfTest.java |    2 +-
 ...ridGgfsHadoopFileSystemAbstractSelfTest.java |    2 +-
 .../GridGgfsHadoopFileSystemClientSelfTest.java |    2 +-
 ...fsHadoopFileSystemShmemAbstractSelfTest.java |    5 +-
 ...idHadoopDefaultMapReducePlannerSelfTest.java |  116 +-
 .../hadoop/GridHadoopMapReduceTest.java         |    9 +-
 .../processors/hadoop/GridHadoopStartup.java    |    3 +-
 .../hadoop/GridHadoopTaskExecutionSelfTest.java |    6 +-
 .../hadoop/GridHadoopTasksAllVersionsTest.java  |    9 +-
 .../GridHadoopTestRoundRobinMrPlanner.java      |    3 +-
 .../hadoop/GridHadoopTestTaskContext.java       |    9 +-
 .../hadoop/GridHadoopV2JobSelfTest.java         |    5 +-
 .../collections/GridHadoopAbstractMapTest.java  |   14 +-
 ...GridHadoopExternalTaskExecutionSelfTest.java |    2 +-
 .../loadtests/ggfs/GridGgfsNodeStartup.java     |    4 +-
 .../GridHibernateAccessStrategyAdapter.java     |   10 +-
 .../GridHibernateGeneralDataRegion.java         |    4 +-
 .../GridHibernateNonStrictAccessStrategy.java   |    8 +-
 .../GridHibernateReadOnlyAccessStrategy.java    |    2 +-
 .../GridHibernateReadWriteAccessStrategy.java   |   24 +-
 .../hibernate/GridHibernateRegionFactory.java   |    2 +-
 ...ridHibernateTransactionalAccessStrategy.java |   10 +-
 .../hibernate/GridCacheHibernateBlobStore.java  |   32 +-
 ...idHibernateL2CacheTransactionalSelfTest.java |    6 +-
 .../processors/query/h2/GridH2Indexing.java     | 1998 +++++++++++++++
 .../query/h2/GridH2ResultSetIterator.java       |  122 +
 .../query/h2/opt/GridH2AbstractKeyValueRow.java |  447 ++++
 .../processors/query/h2/opt/GridH2Cursor.java   |   62 +
 .../query/h2/opt/GridH2IndexBase.java           |  198 ++
 .../query/h2/opt/GridH2KeyValueRowOffheap.java  |  346 +++
 .../query/h2/opt/GridH2KeyValueRowOnheap.java   |   46 +
 .../processors/query/h2/opt/GridH2Row.java      |   40 +
 .../query/h2/opt/GridH2RowDescriptor.java       |  102 +
 .../query/h2/opt/GridH2SpatialIndex.java        |  318 +++
 .../processors/query/h2/opt/GridH2Table.java    |  888 +++++++
 .../query/h2/opt/GridH2TreeIndex.java           |  469 ++++
 .../processors/query/h2/opt/GridH2Utils.java    |  125 +
 .../query/h2/opt/GridLuceneDirectory.java       |  189 ++
 .../processors/query/h2/opt/GridLuceneFile.java |  186 ++
 .../query/h2/opt/GridLuceneIndex.java           |  384 +++
 .../query/h2/opt/GridLuceneInputStream.java     |  220 ++
 .../query/h2/opt/GridLuceneLockFactory.java     |   64 +
 .../query/h2/opt/GridLuceneOutputStream.java    |  230 ++
 .../query/h2/opt/GridSearchRowPointer.java      |   20 +
 .../kernal/processors/query/h2/package.html     |   15 +
 .../h2/GridH2IndexingSpaceConfiguration.java    |  156 --
 .../grid/spi/indexing/h2/GridH2IndexingSpi.java | 2398 ------------------
 .../spi/indexing/h2/GridH2IndexingSpiMBean.java |  222 --
 .../indexing/h2/GridH2ResultSetIterator.java    |  122 -
 .../h2/opt/GridH2AbstractKeyValueRow.java       |  446 ----
 .../grid/spi/indexing/h2/opt/GridH2Cursor.java  |   62 -
 .../spi/indexing/h2/opt/GridH2IndexBase.java    |  204 --
 .../h2/opt/GridH2KeyValueRowOffheap.java        |  346 ---
 .../h2/opt/GridH2KeyValueRowOnheap.java         |   46 -
 .../grid/spi/indexing/h2/opt/GridH2Row.java     |   40 -
 .../indexing/h2/opt/GridH2RowDescriptor.java    |  103 -
 .../spi/indexing/h2/opt/GridH2SpatialIndex.java |  318 ---
 .../grid/spi/indexing/h2/opt/GridH2Table.java   |  893 -------
 .../spi/indexing/h2/opt/GridH2TreeIndex.java    |  480 ----
 .../grid/spi/indexing/h2/opt/GridH2Utils.java   |  125 -
 .../indexing/h2/opt/GridLuceneDirectory.java    |  189 --
 .../spi/indexing/h2/opt/GridLuceneFile.java     |  186 --
 .../spi/indexing/h2/opt/GridLuceneIndex.java    |  391 ---
 .../indexing/h2/opt/GridLuceneInputStream.java  |  220 --
 .../indexing/h2/opt/GridLuceneLockFactory.java  |   64 -
 .../indexing/h2/opt/GridLuceneOutputStream.java |  230 --
 .../indexing/h2/opt/GridSearchRowPointer.java   |   20 -
 .../gridgain/grid/spi/indexing/h2/package.html  |   15 -
 .../GridCacheAbstractFieldsQuerySelfTest.java   |   82 +-
 .../cache/GridCacheAbstractQuerySelfTest.java   |   51 +-
 .../cache/GridCacheCrossCacheQuerySelfTest.java |    9 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |   32 +-
 .../cache/GridCacheQueryLoadSelfTest.java       |   20 +-
 .../cache/GridCacheQueryMetricsSelfTest.java    |   17 +-
 .../GridCacheQueryMultiThreadedSelfTest.java    |   73 +-
 .../GridCacheQueryUserResourceSelfTest.java     |    4 +-
 ...idCacheReduceQueryMultithreadedSelfTest.java |   13 +-
 .../GridCacheSqlQueryMultiThreadedSelfTest.java |    9 +-
 .../processors/cache/GridCacheSwapSelfTest.java |    6 +-
 .../cache/GridIndexingWithNoopSwapSelfTest.java |   21 +-
 .../GridCacheAtomicFieldsQuerySelfTest.java     |    4 +-
 ...GridCachePartitionedFieldsQuerySelfTest.java |    2 +-
 ...idCachePartitionedHitsAndMissesSelfTest.java |   26 +-
 .../near/GridCacheQueryNodeRestartSelfTest.java |   21 +-
 .../GridCacheReplicatedFieldsQuerySelfTest.java |   10 +-
 .../GridCacheReplicatedQuerySelfTest.java       |   22 +-
 .../local/GridCacheLocalQuerySelfTest.java      |    2 +-
 .../query/h2/GridH2IndexRebuildTest.java        |  241 ++
 .../query/h2/GridH2IndexingGeoSelfTest.java     |  240 ++
 .../query/h2/GridH2IndexingInMemSelfTest.java   |   17 +
 .../query/h2/GridH2IndexingOffheapSelfTest.java |   36 +
 .../h2/GridIndexingSpiAbstractSelfTest.java     |  553 ++++
 .../query/h2/opt/GridH2TableSelfTest.java       |  613 +++++
 .../GridIndexingSpiAbstractSelfTest.java        |  594 -----
 .../spi/indexing/h2/GridH2IndexRebuildTest.java |  236 --
 .../h2/GridH2IndexingSpiGeoSelfTest.java        |  240 --
 .../h2/GridH2IndexingSpiInMemSelfTest.java      |   21 -
 ...GridH2IndexingSpiInMemStartStopSelfTest.java |   21 -
 .../h2/GridH2IndexingSpiOffheapSelfTest.java    |   40 -
 ...idH2IndexingSpiOffheapStartStopSelfTest.java |   40 -
 .../indexing/h2/opt/GridH2TableSelfTest.java    |  613 -----
 .../h2/opt/GridLuceneIndexLoadTest.java         |  162 --
 .../h2indexing/GridH2IndexingSpiLoadTest.java   |  288 ---
 .../bamboo/GridH2IndexingSpiTestSuite.java      |   12 +-
 .../grid/cache/jta/GridCacheTmLookup.java       |    6 +-
 .../cache/jta/jndi/GridCacheJndiTmLookup.java   |    6 +-
 .../reflect/GridCacheReflectionTmLookup.java    |   11 +-
 .../cache/jta/GridCacheJtaManager.java          |   14 +-
 .../cache/jta/GridCacheXAResource.java          |    8 +-
 ...CacheJtaConfigurationValidationSelfTest.java |    7 +-
 .../processors/cache/GridCacheJtaSelfTest.java  |    4 +-
 .../GridTmLookupLifecycleAwareSelfTest.java     |    2 +-
 .../ignite/logger/log4j/IgniteLog4jLogger.java  |   22 +-
 .../http/jetty/GridJettyRestHandler.java        |   26 +-
 .../http/jetty/GridJettyRestProtocol.java       |   30 +-
 .../scalar/lang/ScalarAbsClosureX.scala         |    3 +-
 .../scalar/lang/ScalarAbsPredicateX.scala       |    3 +-
 .../gridgain/scalar/lang/ScalarClosure2X.scala  |    3 +-
 .../gridgain/scalar/lang/ScalarClosure3X.scala  |    3 +-
 .../gridgain/scalar/lang/ScalarClosureX.scala   |    3 +-
 .../scalar/lang/ScalarInClosure2X.scala         |    3 +-
 .../scalar/lang/ScalarInClosure3X.scala         |    3 +-
 .../gridgain/scalar/lang/ScalarInClosureX.scala |    3 +-
 .../scalar/lang/ScalarOutClosureX.scala         |    3 +-
 .../scalar/lang/ScalarPredicate2X.scala         |    3 +-
 .../scalar/lang/ScalarPredicate3X.scala         |    3 +-
 .../gridgain/scalar/lang/ScalarPredicateX.scala |    3 +-
 .../pimps/ScalarCacheProjectionPimp.scala       |    3 +-
 .../gridgain/scalar/pimps/ScalarGridPimp.scala  |    1 +
 .../scalar/pimps/ScalarProjectionPimp.scala     |   13 +-
 .../scalar/pimps/ScalarTaskThreadContext.scala  |    1 +
 .../main/scala/org/gridgain/scalar/scalar.scala |    1 +
 .../test/resources/spring-ping-pong-partner.xml |   21 -
 .../scalar/tests/ScalarCacheQueriesSpec.scala   |    1 +
 .../scalar/tests/ScalarProjectionSpec.scala     |    1 +
 .../schedule/GridScheduleProcessor.java         |    6 +-
 .../processors/schedule/ScheduleFutureImpl.java |   40 +-
 .../grid/kernal/GridScheduleSelfTest.java       |   32 +-
 .../java/org/gridgain/grid/GridGainSpring.java  |   16 +-
 .../java/org/gridgain/grid/GridSpringBean.java  |    7 +-
 .../cache/spring/GridSpringCacheManager.java    |   26 +-
 .../GridResourceSpringBeanInjector.java         |   11 +-
 .../resource/GridSpringResourceContextImpl.java |    6 +-
 .../spring/GridSpringProcessorImpl.java         |   16 +-
 .../resource/GridUserExternalResourceTask1.java |    6 +-
 .../resource/GridUserExternalResourceTask2.java |    4 +-
 .../grid/kernal/GridFactorySelfTest.java        |   14 +-
 .../GridResourceConcurrentUndeploySelfTest.java |    6 +-
 .../GridResourceEventFilterSelfTest.java        |    4 +-
 .../GridResourceFieldInjectionSelfTest.java     |   28 +-
 ...dResourceFieldOverrideInjectionSelfTest.java |    4 +-
 .../resource/GridResourceIocSelfTest.java       |   10 +-
 ...GridResourceIsolatedClassLoaderSelfTest.java |   18 +-
 .../GridResourceIsolatedTaskSelfTest.java       |   18 +-
 .../GridResourceMethodInjectionSelfTest.java    |    8 +-
 ...ResourceMethodOverrideInjectionSelfTest.java |    4 +-
 .../resource/GridResourceProcessorSelfTest.java |    6 +-
 .../GridResourceSharedUndeploySelfTest.java     |   10 +-
 .../resource/GridServiceInjectionSelfTest.java  |    6 +-
 .../GridP2PContinuousLocalDeploySelfTest.java   |    8 +-
 .../p2p/GridP2PUserVersionChangeSelfTest.java   |    4 +-
 .../GridProjectionStartStopRestartSelfTest.java |   48 +-
 .../uri/GridUriDeploymentFileProcessor.java     |    2 +-
 .../deployment/uri/GridUriDeploymentSpi.java    |    2 +-
 .../ftp/GridUriDeploymentFtpException.java      |    4 +-
 .../ftp/GridUriDeploymentFtpScanner.java        |    4 +-
 ...riDeploymentClassloaderRegisterSelfTest.java |    6 +-
 .../uri/GridUriDeploymentSimpleSelfTest.java    |   10 +-
 .../grid/p2p/GridP2PDisabledSelfTest.java       |    2 +-
 .../gridgain/visor/commands/VisorConsole.scala  |   48 +-
 .../visor/commands/VisorConsoleCommand.scala    |    1 +
 .../visor/commands/VisorTextTable.scala         |    5 +-
 .../visor/commands/ack/VisorAckCommand.scala    |   11 +-
 .../commands/alert/VisorAlertCommand.scala      |   23 +-
 .../commands/cache/VisorCacheClearCommand.scala |    4 +-
 .../commands/cache/VisorCacheCommand.scala      |   14 +-
 .../cache/VisorCacheCompactCommand.scala        |    4 +-
 .../commands/cache/VisorCacheScanCommand.scala  |    5 +-
 .../commands/cache/VisorCacheSwapCommand.scala  |    4 +-
 .../config/VisorConfigurationCommand.scala      |   11 +-
 .../commands/deploy/VisorDeployCommand.scala    |   10 +-
 .../commands/disco/VisorDiscoveryCommand.scala  |   18 +-
 .../commands/events/VisorEventsCommand.scala    |   13 +-
 .../visor/commands/gc/VisorGcCommand.scala      |    7 +-
 .../visor/commands/kill/VisorKillCommand.scala  |   17 +-
 .../visor/commands/node/VisorNodeCommand.scala  |   13 +-
 .../visor/commands/ping/VisorPingCommand.scala  |    9 +-
 .../commands/start/VisorStartCommand.scala      |   15 +-
 .../commands/tasks/VisorTasksCommand.scala      |   19 +-
 .../commands/top/VisorTopologyCommand.scala     |   21 +-
 .../visor/commands/vvm/VisorVvmCommand.scala    |   64 +-
 .../main/scala/org/gridgain/visor/visor.scala   |   63 +-
 .../gridgain/visor/VisorRuntimeBaseSpec.scala   |    2 +-
 .../cache/VisorCacheClearCommandSpec.scala      |    2 +-
 .../cache/VisorCacheCompactCommandSpec.scala    |    2 +-
 .../config/VisorConfigurationCommandSpec.scala  |    1 +
 .../disco/VisorDiscoveryCommandSpec.scala       |    2 +-
 .../events/VisorEventsCommandSpec.scala         |    1 +
 .../commands/open/VisorOpenCommandSpec.scala    |    4 +-
 .../cache/websession/GridWebSessionFilter.java  |   22 +-
 .../websession/GridWebSessionListener.java      |    4 +-
 .../GridServletContextListenerStartup.java      |   12 +-
 .../startup/servlet/GridServletStartup.java     |    2 +-
 pom.xml                                         |   34 +-
 1498 files changed, 23843 insertions(+), 23498 deletions(-)
----------------------------------------------------------------------