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 2016/12/09 11:27:23 UTC

[01/25] ignite git commit: ignite-4296 Optimize GridDhtPartitionsSingleMessage processing - optimized processing of GridDhtPartitionsSingleMessage - minor optimizations for RendezvousAffinityFunction - fixed heartbeats sending in tcp discovery

Repository: ignite
Updated Branches:
  refs/heads/ignite-4371 697e2fb81 -> 37a76190d


http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
deleted file mode 100644
index 377a55f..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtTestUtils.java
+++ /dev/null
@@ -1,232 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache.distributed.dht;
-
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.UUID;
-import org.apache.ignite.Ignite;
-import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.cache.affinity.Affinity;
-import org.apache.ignite.cache.affinity.AffinityFunction;
-import org.apache.ignite.cluster.ClusterNode;
-import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
-import org.apache.ignite.internal.processors.cache.GridCacheAdapter;
-import org.apache.ignite.internal.processors.cache.GridCacheConcurrentMap;
-import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.GridCacheEntryEx;
-import org.apache.ignite.internal.processors.cache.KeyCacheObject;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
-import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.P1;
-import org.apache.ignite.internal.util.typedef.internal.CU;
-
-import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.OWNING;
-
-/**
- * Utility methods for dht preloader testing.
- */
-public class GridCacheDhtTestUtils {
-    /**
-     * Ensure singleton.
-     */
-    private GridCacheDhtTestUtils() {
-        // No-op.
-    }
-
-    /**
-     * @param dht Cache.
-     * @param keyCnt Number of test keys to put into cache.
-     * @throws IgniteCheckedException If failed to prepare.
-     */
-    @SuppressWarnings({"UnusedAssignment", "unchecked"})
-    static void prepareKeys(GridDhtCache<Integer, String> dht, int keyCnt) throws IgniteCheckedException {
-        AffinityFunction aff = dht.context().config().getAffinity();
-
-        GridCacheConcurrentMap cacheMap;
-
-        try {
-            Field field = GridCacheAdapter.class.getDeclaredField("map");
-
-            field.setAccessible(true);
-
-            cacheMap = (GridCacheConcurrentMap)field.get(dht);
-        }
-        catch (Exception e) {
-            throw new IgniteCheckedException("Failed to get cache map.", e);
-        }
-
-        GridDhtPartitionTopology top = dht.topology();
-
-        GridCacheContext ctx = dht.context();
-
-        for (int i = 0; i < keyCnt; i++) {
-            KeyCacheObject cacheKey = ctx.toCacheKeyObject(i);
-
-            cacheMap.putEntryIfObsoleteOrAbsent(
-                AffinityTopologyVersion.NONE,
-                cacheKey,
-                ctx.toCacheKeyObject("value" + i),
-                false,
-                false);
-
-            dht.preloader().request(Collections.singleton(cacheKey), AffinityTopologyVersion.NONE);
-
-            GridDhtLocalPartition part = top.localPartition(aff.partition(i), false);
-
-            assert part != null;
-
-            part.own();
-        }
-    }
-
-    /**
-     * @param dht Dht cache.
-     * @param idx Cache index
-     */
-    static void printDhtTopology(GridDhtCache<Integer, String> dht, int idx) {
-        final Affinity<Integer> aff = dht.affinity();
-
-        Ignite ignite = dht.context().grid();
-        ClusterNode locNode = ignite.cluster().localNode();
-
-        GridDhtPartitionTopology top = dht.topology();
-
-        System.out.println("\nTopology of cache #" + idx + " (" + locNode.id() + ")" + ":");
-        System.out.println("----------------------------------");
-
-        List<Integer> affParts = new LinkedList<>();
-
-        GridDhtPartitionMap2 map = dht.topology().partitions(locNode.id());
-
-        if (map != null)
-            for (int p : map.keySet())
-                affParts.add(p);
-
-        Collections.sort(affParts);
-
-        System.out.println("Affinity partitions: " + affParts + "\n");
-
-        List<GridDhtLocalPartition> locals = new ArrayList<GridDhtLocalPartition>(top.localPartitions());
-
-        Collections.sort(locals);
-
-        for (final GridDhtLocalPartition part : locals) {
-            Collection<ClusterNode> partNodes = aff.mapKeyToPrimaryAndBackups(part.id());
-
-            String ownStr = !partNodes.contains(dht.context().localNode()) ? "NOT AN OWNER" :
-                F.eqNodes(CU.primary(partNodes), locNode) ? "PRIMARY" : "BACKUP";
-
-            Collection<Integer> keys = F.viewReadOnly(dht.keySet(), F.<Integer>identity(), new P1<Integer>() {
-                @Override public boolean apply(Integer k) {
-                    return aff.partition(k) == part.id();
-                }
-            });
-
-            System.out.println("Local partition: [" + part + "], [owning=" + ownStr + ", keyCnt=" + keys.size() +
-                ", keys=" + keys + "]");
-        }
-
-        System.out.println("\nNode map:");
-
-        for (Map.Entry<UUID, GridDhtPartitionMap2> e : top.partitionMap(false).entrySet()) {
-            List<Integer> list = new ArrayList<>(e.getValue().keySet());
-
-            Collections.sort(list);
-
-            System.out.println("[node=" + e.getKey() + ", parts=" + list + "]");
-        }
-
-        System.out.println("");
-    }
-
-    /**
-     * Checks consistency of partitioned cache.
-     * Any preload processes must be finished before this method call().
-     *
-     * @param dht Dht cache.
-     * @param idx Cache index.
-     * @param log Logger.
-     */
-    @SuppressWarnings("unchecked")
-    static void checkDhtTopology(GridDhtCache<Integer, String> dht, int idx, IgniteLogger log) {
-        assert dht != null;
-        assert idx >= 0;
-        assert log != null;
-
-        log.info("Checking balanced state of cache #" + idx);
-
-        Affinity<Object> aff = (Affinity)dht.affinity();
-
-        Ignite ignite = dht.context().grid();
-        ClusterNode locNode = ignite.cluster().localNode();
-
-        GridDhtPartitionTopology top = dht.topology();
-
-        // Expected partitions calculated with affinity function.
-        // They should be in topology in OWNING state.
-        Collection<Integer> affParts = new HashSet<>();
-
-        GridDhtPartitionMap2 map = dht.topology().partitions(locNode.id());
-
-        if (map != null)
-            for (int p : map.keySet())
-                affParts.add(p);
-
-        if (F.isEmpty(affParts))
-            return;
-
-        for (int p : affParts)
-            assert top.localPartition(p, false) != null :
-                "Partition does not exist in topology: [cache=" + idx + ", part=" + p + "]";
-
-        for (GridDhtLocalPartition p : top.localPartitions()) {
-            assert affParts.contains(p.id()) :
-                "Invalid local partition: [cache=" + idx + ", part=" + p + ", node partitions=" + affParts + "]";
-
-            assert p.state() == OWNING : "Invalid partition state [cache=" + idx + ", part=" + p + "]";
-
-            Collection<ClusterNode> partNodes = aff.mapPartitionToPrimaryAndBackups(p.id());
-
-            assert partNodes.contains(locNode) :
-                "Partition affinity nodes does not contain local node: [cache=" + idx + "]";
-        }
-
-        // Check keys.
-        for (GridCacheEntryEx e : dht.entries()) {
-            GridDhtCacheEntry entry = (GridDhtCacheEntry)e;
-
-            if (!affParts.contains(entry.partition()))
-                log.warning("Partition of stored entry is obsolete for node: [cache=" + idx + ", entry=" + entry +
-                    ", node partitions=" + affParts + "]");
-
-            int p = aff.partition(entry.key());
-
-            if (!affParts.contains(p))
-                log.warning("Calculated entry partition is not in node partitions: [cache=" + idx + ", part=" + p +
-                    ", entry=" + entry + ", node partitions=" + affParts + "]");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 48567da..40b11973 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -58,8 +58,7 @@ import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.GridCacheContext;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
-import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.query.GridCacheQueryMarshallable;
 import org.apache.ignite.internal.processors.cache.query.GridCacheSqlQuery;
 import org.apache.ignite.internal.processors.cache.query.GridCacheTwoStepQuery;
@@ -383,17 +382,10 @@ public class GridReduceQueryExecutor {
 
     /**
      * @param cctx Cache context.
-     * @return {@code true} If cache context
+     * @return {@code True} If cache has partitions in {@link GridDhtPartitionState#MOVING} state.
      */
     private boolean hasMovingPartitions(GridCacheContext<?, ?> cctx) {
-        GridDhtPartitionFullMap fullMap = cctx.topology().partitionMap(false);
-
-        for (GridDhtPartitionMap2 map : fullMap.values()) {
-            if (map.hasMovingPartitions())
-                return true;
-        }
-
-        return false;
+        return cctx.topology().hasMovingPartitions();
     }
 
     /**


[12/25] ignite git commit: IGNITE-4336 Manual rebalance can't be requested twice

Posted by sb...@apache.org.
IGNITE-4336 Manual rebalance can't be requested twice


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

Branch: refs/heads/ignite-4371
Commit: 9a691c4f6a709caab18ca1442217829b6f578e56
Parents: 59e6fec
Author: Ilya Lantukh <il...@gridgain.com>
Authored: Thu Dec 8 17:58:54 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 8 18:02:46 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  6 +-
 .../GridCachePartitionExchangeManager.java      | 11 ++-
 .../processors/cache/GridCachePreloader.java    |  8 ++-
 .../cache/GridCachePreloaderAdapter.java        | 12 ++--
 .../processors/cache/IgniteCacheProxy.java      |  4 +-
 .../dht/preloader/GridDhtPartitionDemander.java | 46 ++++++++++--
 .../GridDhtPartitionsExchangeFuture.java        | 16 ++++-
 .../dht/preloader/GridDhtPreloader.java         | 11 +--
 .../cache/CacheRebalancingSelfTest.java         | 75 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite5.java       |  9 ++-
 10 files changed, 166 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 2e24e67..0f39c69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -84,8 +84,8 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter;
@@ -4651,9 +4651,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> rebalance() {
-        ctx.preloader().forcePreload();
-
-        return ctx.preloader().syncFuture();
+        return ctx.preloader().forceRebalance();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index f04a6ce..8ea2169 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -711,9 +711,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      *
      * @param exchFut Exchange future.
      */
-    public void forcePreloadExchange(GridDhtPartitionsExchangeFuture exchFut) {
+    public IgniteInternalFuture<Boolean> forceRebalance(GridDhtPartitionsExchangeFuture exchFut) {
+        GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>();
+
         exchWorker.addFuture(
-            new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId()));
+            new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId(), fut));
+
+        return fut;
     }
 
     /**
@@ -1771,7 +1775,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                 Runnable cur = cacheCtx.preloader().addAssignments(assigns,
                                     forcePreload,
                                     cnt,
-                                    r);
+                                    r,
+                                    exchFut.forcedRebalanceFuture());
 
                                 if (cur != null) {
                                     rebList.add(U.maskName(cacheCtx.name()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 3c4456d..0c28691 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
 
@@ -90,7 +91,8 @@ public interface GridCachePreloader {
     public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
         boolean forcePreload,
         int cnt,
-        Runnable next);
+        Runnable next,
+        @Nullable GridFutureAdapter<Boolean> forcedRebFut);
 
     /**
      * @param p Preload predicate.
@@ -150,9 +152,9 @@ public interface GridCachePreloader {
         AffinityTopologyVersion topVer);
 
     /**
-     * Force preload process.
+     * Force Rebalance process.
      */
-    public void forcePreload();
+    public IgniteInternalFuture<Boolean> forceRebalance();
 
     /**
      * Unwinds undeploys.

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index 656a960..d7ec288 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
 
@@ -88,8 +89,8 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
-    @Override public void forcePreload() {
-        // No-op.
+    @Override public IgniteInternalFuture<Boolean> forceRebalance() {
+        return new GridFinishedFuture<>(true);
     }
 
     /** {@inheritDoc} */
@@ -165,8 +166,11 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
-    @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
-        int cnt, Runnable next) {
+    @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
+        boolean forcePreload,
+        int cnt,
+        Runnable next,
+        @Nullable GridFutureAdapter<Boolean> forcedRebFut) {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f87fa1d..f03a3b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -2326,9 +2326,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<?> rebalance() {
-        ctx.preloader().forcePreload();
-
-        return new IgniteFutureImpl<>(ctx.preloader().syncFuture());
+        return new IgniteFutureImpl<>(ctx.preloader().forceRebalance());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index a6808c7..02c31da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
 import org.apache.ignite.internal.util.GridLeanSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -64,6 +65,7 @@ import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.spi.IgniteSpiException;
@@ -216,9 +218,9 @@ public class GridDhtPartitionDemander {
     }
 
     /**
-     * Force preload.
+     * Force Rebalance.
      */
-    void forcePreload() {
+    IgniteInternalFuture<Boolean> forceRebalance() {
         GridTimeoutObject obj = lastTimeoutObj.getAndSet(null);
 
         if (obj != null)
@@ -230,14 +232,31 @@ public class GridDhtPartitionDemander {
             if (log.isDebugEnabled())
                 log.debug("Forcing rebalance event for future: " + exchFut);
 
+            final GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>();
+
             exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                 @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                    cctx.shared().exchange().forcePreloadExchange(exchFut);
+                    IgniteInternalFuture<Boolean> fut0 = cctx.shared().exchange().forceRebalance(exchFut);
+
+                    fut0.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
+                        @Override public void apply(IgniteInternalFuture<Boolean> future) {
+                            try {
+                                fut.onDone(future.get());
+                            }
+                            catch (Exception e) {
+                                fut.onDone(e);
+                            }
+                        }
+                    });
                 }
             });
+
+            return fut;
         }
         else if (log.isDebugEnabled())
             log.debug("Ignoring force rebalance request (no topology event happened yet).");
+
+        return new GridFinishedFuture<>(true);
     }
 
     /**
@@ -275,15 +294,19 @@ public class GridDhtPartitionDemander {
      * @param force {@code True} if dummy reassign.
      * @param cnt Counter.
      * @param next Runnable responsible for cache rebalancing start.
+     * @param forcedRebFut External future for forced rebalance.
      * @return Rebalancing runnable.
      */
     Runnable addAssignments(final GridDhtPreloaderAssignments assigns,
         boolean force,
         int cnt,
-        final Runnable next) {
+        final Runnable next,
+        @Nullable final GridFutureAdapter<Boolean> forcedRebFut) {
         if (log.isDebugEnabled())
             log.debug("Adding partition assignments: " + assigns);
 
+        assert force == (forcedRebFut != null);
+
         long delay = cctx.config().getRebalanceDelay();
 
         if (delay == 0 || force) {
@@ -301,6 +324,19 @@ public class GridDhtPartitionDemander {
                 });
             }
 
+             if (forcedRebFut != null) {
+                fut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+                    @Override public void apply(IgniteInternalFuture<Boolean> future) {
+                        try {
+                            forcedRebFut.onDone(future.get());
+                        }
+                        catch (Exception e) {
+                            forcedRebFut.onDone(e);
+                        }
+                    }
+                });
+            }
+
             rebalanceFut = fut;
 
             fut.sendRebalanceStartedEvent();
@@ -383,7 +419,7 @@ public class GridDhtPartitionDemander {
                 @Override public void onTimeout() {
                     exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> f) {
-                            cctx.shared().exchange().forcePreloadExchange(exchFut);
+                            cctx.shared().exchange().forceRebalance(exchFut);
                         }
                     });
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index e945de9..4f34401 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -54,7 +54,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
@@ -65,7 +64,6 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -194,6 +192,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     /** */
     private boolean centralizedAff;
 
+    /** Forced Rebalance future. */
+    private GridFutureAdapter<Boolean> forcedRebFut;
+
     /**
      * Dummy future created to trigger reassignments if partition
      * topology changed while preloading.
@@ -227,15 +228,17 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @param cctx Cache context.
      * @param discoEvt Discovery event.
      * @param exchId Exchange id.
+     * @param forcedRebFut Forced Rebalance future.
      */
     public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, DiscoveryEvent discoEvt,
-        GridDhtPartitionExchangeId exchId) {
+        GridDhtPartitionExchangeId exchId, GridFutureAdapter<Boolean> forcedRebFut) {
         dummy = false;
         forcePreload = true;
 
         this.exchId = exchId;
         this.discoEvt = discoEvt;
         this.cctx = cctx;
+        this.forcedRebFut = forcedRebFut;
 
         reassign = true;
 
@@ -398,6 +401,13 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
+     * @return Forced Rebalance future.
+     */
+    @Nullable public GridFutureAdapter<Boolean> forcedRebalanceFuture() {
+        return forcedRebFut;
+    }
+
+    /**
      * @return {@code true} if entered to busy state.
      */
     private boolean enterBusy() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 692e7c0..41bc2fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -413,8 +413,11 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
 
     /** {@inheritDoc} */
     @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
-        boolean forcePreload, int cnt, Runnable next) {
-        return demander.addAssignments(assignments, forcePreload, cnt, next);
+        boolean forceRebalance,
+        int cnt,
+        Runnable next,
+        @Nullable GridFutureAdapter<Boolean> forcedRebFut) {
+        return demander.addAssignments(assignments, forceRebalance, cnt, next, forcedRebFut);
     }
 
     /**
@@ -728,8 +731,8 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void forcePreload() {
-        demander.forcePreload();
+    @Override public IgniteInternalFuture<Boolean> forceRebalance() {
+        return demander.forceRebalance();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
new file mode 100644
index 0000000..8d1f67a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for rebalancing.
+ */
+public class CacheRebalancingSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCacheConfiguration(new CacheConfiguration());
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRebalanceFuture() throws Exception {
+        IgniteEx ignite0 = startGrid(0);
+        startGrid(1);
+
+        IgniteCache<Object, Object> cache = ignite0.cache(null);
+
+        IgniteFuture fut1 = cache.rebalance();
+
+        fut1.get();
+
+        startGrid(2);
+
+        IgniteFuture fut2 = cache.rebalance();
+
+        assert internalFuture(fut2) != internalFuture(fut1);
+
+        fut2.get();
+    }
+
+    /**
+     * @param future Future.
+     * @return Internal future.
+     */
+    private static IgniteInternalFuture internalFuture(IgniteFuture future) {
+        assert future instanceof IgniteFutureImpl;
+
+        return ((IgniteFutureImpl)future).internalFuture();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/9a691c4f/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
index 7f0e23c..144aac6 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
@@ -19,18 +19,19 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.CacheNearReaderUpdateTest;
+import org.apache.ignite.internal.processors.cache.CacheRebalancingSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheSerializableTransactionsTest;
-import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest;
 import org.apache.ignite.internal.processors.cache.EntryVersionConsistencyReadThroughTest;
+import org.apache.ignite.internal.processors.cache.GridCacheOffHeapCleanupTest;
+import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePutStackOverflowSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheReadThroughEvictionsVariationsSuite;
 import org.apache.ignite.internal.processors.cache.IgniteCacheStoreCollectionTest;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapCleanupTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentFairAffinityTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxIteratorSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest;
 import org.apache.ignite.internal.processors.cache.store.IgniteCacheWriteBehindNoUpdateSelfTest;
 
 /**
@@ -62,6 +63,8 @@ public class IgniteCacheTestSuite5 extends TestSuite {
 
         suite.addTestSuite(GridCacheOffHeapCleanupTest.class);
 
+        suite.addTestSuite(CacheRebalancingSelfTest.class);
+
         return suite;
     }
 }


[07/25] ignite git commit: Merge remote-tracking branch 'remotes/professional/ignite-1.7.4'

Posted by sb...@apache.org.
Merge remote-tracking branch 'remotes/professional/ignite-1.7.4'

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java


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

Branch: refs/heads/ignite-4371
Commit: 858b129ec53f500130b57824a787a4af7c4eb6e7
Parents: 5099f14 bbaa79a
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 8 12:44:49 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 8 12:44:49 2016 +0300

----------------------------------------------------------------------
 .../rendezvous/RendezvousAffinityFunction.java  |  80 ++++--
 .../discovery/GridDiscoveryManager.java         | 118 +-------
 .../GridCachePartitionExchangeManager.java      | 134 +++------
 .../processors/cache/GridCachePreloader.java    |  11 +-
 .../cache/GridCachePreloaderAdapter.java        |   5 +-
 .../processors/cache/GridCacheUtils.java        |  17 --
 .../binary/CacheObjectBinaryProcessorImpl.java  |   3 +-
 .../dht/GridClientPartitionTopology.java        | 120 ++++----
 .../dht/GridDhtPartitionTopology.java           |  28 +-
 .../dht/GridDhtPartitionTopologyImpl.java       | 284 +++++++++++--------
 .../dht/preloader/GridDhtPartitionDemander.java | 230 +++++++++------
 .../dht/preloader/GridDhtPartitionFullMap.java  |  18 +-
 .../GridDhtPartitionsExchangeFuture.java        |  56 +++-
 .../dht/preloader/GridDhtPreloader.java         |   9 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../service/GridServiceProcessor.java           |   4 +-
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  41 ++-
 .../tcp/internal/TcpDiscoveryStatistics.java    |   4 +
 .../AbstractAffinityFunctionSelfTest.java       |   2 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   2 +-
 .../GridCachePartitionedAffinitySpreadTest.java |   7 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  | 232 ---------------
 .../GridCacheRebalancingSyncSelfTest.java       |   2 +
 .../h2/twostep/GridReduceQueryExecutor.java     |  14 +-
 25 files changed, 620 insertions(+), 805 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/858b129e/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/858b129e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/858b129e/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
----------------------------------------------------------------------
diff --cc modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
index ee5b65c,43017db..8f8d78a
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
@@@ -104,24 -104,9 +104,25 @@@ public abstract class AbstractAffinityF
      }
  
      /**
+      * @param backups Number of backups.
       * @throws Exception If failed.
       */
 +    public void testNullKeyForPartitionCalculation() throws Exception {
 +        AffinityFunction aff = affinityFunction();
 +
 +        try {
 +            aff.partition(null);
 +
 +            fail("Should throw IllegalArgumentException due to NULL affinity key.");
 +        } catch (IllegalArgumentException e) {
 +            e.getMessage().contains("Null key is passed for a partition calculation. " +
 +                "Make sure that an affinity key that is used is initialized properly.");
 +        }
 +    }
 +
 +    /**
 +     * @throws Exception If failed.
 +     */
      protected void checkNodeRemoved(int backups) throws Exception {
          checkNodeRemoved(backups, 1, 1);
      }

http://git-wip-us.apache.org/repos/asf/ignite/blob/858b129e/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------


[17/25] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NIO sessio

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/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 1fe437c..b392c07 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
@@ -46,6 +46,7 @@ import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
 import javax.net.ssl.SSLEngine;
 import javax.net.ssl.SSLException;
 import org.apache.ignite.Ignite;
@@ -53,6 +54,7 @@ import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteClientDisconnectedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.IgniteSystemProperties;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.AddressResolver;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -103,6 +105,7 @@ import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteFuture;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteProductVersion;
 import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -179,6 +182,7 @@ import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META
  * <li>Node local IP address (see {@link #setLocalAddress(String)})</li>
  * <li>Node local port number (see {@link #setLocalPort(int)})</li>
  * <li>Local port range (see {@link #setLocalPortRange(int)}</li>
+ * <li>Connections per node (see {@link #setConnectionsPerNode(int)})</li>
  * <li>Connection buffer flush frequency (see {@link #setConnectionBufferFlushFrequency(long)})</li>
  * <li>Connection buffer size (see {@link #setConnectionBufferSize(int)})</li>
  * <li>Idle connection timeout (see {@link #setIdleConnectionTimeout(long)})</li>
@@ -238,6 +242,9 @@ import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.SSL_META
 @IgniteSpiConsistencyChecked(optional = false)
 public class TcpCommunicationSpi extends IgniteSpiAdapter
     implements CommunicationSpi<Message>, TcpCommunicationSpiMBean {
+    /** */
+    private static final IgniteProductVersion MULTIPLE_CONN_SINCE_VER = IgniteProductVersion.fromString("1.8.0");
+
     /** IPC error message. */
     public static final String OUT_OF_RESOURCES_TCP_MSG = "Failed to allocate shared memory segment " +
         "(switching to TCP, may be slower).";
@@ -257,11 +264,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Node attribute that is mapped to node's external addresses (value is <tt>comm.tcp.ext-addrs</tt>). */
     public static final String ATTR_EXT_ADDRS = "comm.tcp.ext-addrs";
 
+    /** */
+    public static final String ATTR_PAIRED_CONN = "comm.tcp.pairedConnection";
+
     /** Default port which node sets listener to (value is <tt>47100</tt>). */
     public static final int DFLT_PORT = 47100;
 
     /** Default port which node sets listener for shared memory connections (value is <tt>48100</tt>). */
-    public static final int DFLT_SHMEM_PORT = 48100;
+    public static final int DFLT_SHMEM_PORT = -1;
 
     /** Default idle connection timeout (value is <tt>30000</tt>ms). */
     public static final long DFLT_IDLE_CONN_TIMEOUT = 30000;
@@ -283,12 +293,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
     /**
      * Default count of selectors for TCP server equals to
-     * {@code "Math.min(4, Runtime.getRuntime().availableProcessors())"}.
+     * {@code "Math.min(8, Runtime.getRuntime().availableProcessors())"}.
      */
-    public static final int DFLT_SELECTORS_CNT = Math.min(4, Runtime.getRuntime().availableProcessors());
+    public static final int DFLT_SELECTORS_CNT = Math.max(4, Runtime.getRuntime().availableProcessors() / 2);
 
-    /** Node ID meta for session. */
-    private static final int NODE_ID_META = GridNioSessionMetaKey.nextUniqueKey();
+    /** Connection index meta for session. */
+    private static final int CONN_IDX_META = GridNioSessionMetaKey.nextUniqueKey();
 
     /** Message tracker meta for session. */
     private static final int TRACKER_META = GridNioSessionMetaKey.nextUniqueKey();
@@ -303,11 +313,14 @@ 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 = 16;
+    public static final int DFLT_ACK_SND_THRESHOLD = 32;
 
     /** Default socket write timeout. */
     public static final long DFLT_SOCK_WRITE_TIMEOUT = 2000;
 
+    /** Default connections per node. */
+    public static final int DFLT_CONN_PER_NODE = 1;
+
     /** No-op runnable. */
     private static final IgniteRunnable NOOP = new IgniteRunnable() {
         @Override public void run() {
@@ -327,11 +340,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** */
     private ConnectGateway connectGate;
 
+    /** */
+    private ConnectionPolicy connPlc;
+
     /** Server listener. */
     private final GridNioServerListener<Message> srvLsnr =
         new GridNioServerListenerAdapter<Message>() {
             @Override public void onSessionWriteTimeout(GridNioSession ses) {
-                LT.warn(log, "Communication SPI Session write timed out (consider increasing " +
+                LT.warn(log,"Communication SPI session write timed out (consider increasing " +
                     "'socketWriteTimeout' " + "configuration property) [remoteAddr=" + ses.remoteAddress() +
                     ", writeTimeout=" + sockWriteTimeout + ']');
 
@@ -347,46 +363,53 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     if (log.isDebugEnabled())
                         log.debug("Sending local node ID to newly accepted session: " + ses);
 
-                    ses.send(nodeIdMessage());
+                    try {
+                        ses.sendNoFuture(nodeIdMessage());
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to send message: " + e, e);
+                    }
                 }
             }
 
             @Override public void onDisconnected(GridNioSession ses, @Nullable Exception e) {
-                UUID id = ses.meta(NODE_ID_META);
+                ConnectionKey connId = ses.meta(CONN_IDX_META);
 
-                if (id != null) {
-                    GridCommunicationClient client = clients.get(id);
+                if (connId != null) {
+                    UUID id = connId.nodeId();
 
-                    if (client instanceof GridTcpNioCommunicationClient &&
-                        ((GridTcpNioCommunicationClient) client).session() == ses) {
-                        client.close();
+                    GridCommunicationClient[] nodeClients = clients.get(id);
 
-                        clients.remove(id, client);
+                    if (nodeClients != null) {
+                        for (GridCommunicationClient client : nodeClients) {
+                            if (client instanceof GridTcpNioCommunicationClient &&
+                                ((GridTcpNioCommunicationClient)client).session() == ses) {
+                                client.close();
+
+                                removeNodeClient(id, client);
+                            }
+                        }
                     }
 
                     if (!stopping) {
-                        boolean reconnect = false;
-
-                        GridNioRecoveryDescriptor recoveryData = ses.recoveryDescriptor();
-
-                        if (recoveryData != null) {
-                            if (recoveryData.nodeAlive(getSpiContext().node(id))) {
-                                if (!recoveryData.messagesFutures().isEmpty()) {
-                                    reconnect = true;
+                        GridNioRecoveryDescriptor outDesc = ses.outRecoveryDescriptor();
 
+                        if (outDesc != null) {
+                            if (outDesc.nodeAlive(getSpiContext().node(id))) {
+                                if (!outDesc.messagesRequests().isEmpty()) {
                                     if (log.isDebugEnabled())
                                         log.debug("Session was closed but there are unacknowledged messages, " +
-                                            "will try to reconnect [rmtNode=" + recoveryData.node().id() + ']');
+                                            "will try to reconnect [rmtNode=" + outDesc.node().id() + ']');
+
+                                    DisconnectedSessionInfo disconnectData =
+                                        new DisconnectedSessionInfo(outDesc, connId.connectionIndex());
+
+                                    commWorker.addProcessDisconnectRequest(disconnectData);
                                 }
                             }
                             else
-                                recoveryData.onNodeLeft();
+                                outDesc.onNodeLeft();
                         }
-
-                        DisconnectedSessionInfo disconnectData = new DisconnectedSessionInfo(recoveryData,
-                            reconnect);
-
-                        commWorker.addProcessDisconnectRequest(disconnectData);
                     }
 
                     CommunicationListener<Message> lsnr0 = lsnr;
@@ -403,21 +426,24 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             private void onFirstMessage(GridNioSession ses, Message msg) {
                 UUID sndId;
 
-                if (msg instanceof NodeIdMessage)
-                    sndId = U.bytesToUuid(((NodeIdMessage)msg).nodeIdBytes, 0);
+                ConnectionKey connKey;
+
+                if (msg instanceof NodeIdMessage) {
+                    sndId = U.bytesToUuid(((NodeIdMessage) msg).nodeIdBytes, 0);
+                    connKey = new ConnectionKey(sndId, 0, -1);
+                }
                 else {
                     assert msg instanceof HandshakeMessage : msg;
 
+                    HandshakeMessage msg0 = (HandshakeMessage)msg;
+
                     sndId = ((HandshakeMessage)msg).nodeId();
+                    connKey = new ConnectionKey(sndId, msg0.connectionIndex(), msg0.connectCount());
                 }
 
                 if (log.isDebugEnabled())
                     log.debug("Remote node ID received: " + sndId);
 
-                final UUID old = ses.addMeta(NODE_ID_META, sndId);
-
-                assert old == null;
-
                 final ClusterNode rmtNode = getSpiContext().node(sndId);
 
                 if (rmtNode == null) {
@@ -429,57 +455,65 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     return;
                 }
 
+                final ConnectionKey old = ses.addMeta(CONN_IDX_META, connKey);
+
+                assert old == null;
+
                 ClusterNode locNode = getSpiContext().localNode();
 
                 if (ses.remoteAddress() == null)
                     return;
 
-                GridCommunicationClient oldClient = clients.get(sndId);
+                assert msg instanceof HandshakeMessage : msg;
 
-                boolean hasShmemClient = false;
+                HandshakeMessage msg0 = (HandshakeMessage)msg;
 
-                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 + ']');
+                if (usePairedConnections(rmtNode)) {
+                    final GridNioRecoveryDescriptor recoveryDesc = inRecoveryDescriptor(rmtNode, connKey);
 
-                        ses.send(new RecoveryLastReceivedMessage(-1));
+                    ConnectClosureNew c = new ConnectClosureNew(ses, recoveryDesc, rmtNode);
 
-                        return;
-                    }
+                    boolean reserve = recoveryDesc.tryReserve(msg0.connectCount(), c);
+
+                    if (reserve)
+                        connectedNew(recoveryDesc, ses, true);
                     else {
-                        assert oldClient instanceof GridShmemCommunicationClient;
+                        if (c.failed) {
+                            ses.send(new RecoveryLastReceivedMessage(-1));
+
+                            for (GridNioSession ses0 : nioSrvr.sessions()) {
+                                ConnectionKey key0 = ses0.meta(CONN_IDX_META);
 
-                        hasShmemClient = true;
+                                if (ses0.accepted() && key0 != null &&
+                                    key0.nodeId().equals(connKey.nodeId()) &&
+                                    key0.connectionIndex() == connKey.connectionIndex() &&
+                                    key0.connectCount() < connKey.connectCount())
+                                    ses0.close();
+                            }
+                        }
                     }
                 }
+                else {
+                    assert connKey.connectionIndex() >= 0 : connKey;
 
-                GridFutureAdapter<GridCommunicationClient> fut = new GridFutureAdapter<>();
-
-                GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(sndId, fut);
-
-                assert msg instanceof HandshakeMessage : msg;
-
-                HandshakeMessage msg0 = (HandshakeMessage)msg;
+                    GridCommunicationClient[] curClients = clients.get(sndId);
 
-                final GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(rmtNode);
+                    GridCommunicationClient oldClient =
+                        curClients != null && connKey.connectionIndex() < curClients.length ?
+                            curClients[connKey.connectionIndex()] :
+                            null;
 
-                if (oldFut == null) {
-                    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 + ']');
+                                    "to this node, rejecting [locNode=" + locNode.id() +
+                                    ", rmtNode=" + sndId + ']');
 
                             ses.send(new RecoveryLastReceivedMessage(-1));
 
-                            fut.onDone(oldClient);
-
                             return;
                         }
                         else {
@@ -489,51 +523,86 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         }
                     }
 
-                    boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
-                            new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
+                    GridFutureAdapter<GridCommunicationClient> fut = new GridFutureAdapter<>();
 
-                    if (log.isDebugEnabled())
-                        log.debug("Received incoming connection from remote node " +
+                    GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(connKey, fut);
+
+                    final GridNioRecoveryDescriptor recoveryDesc = inRecoveryDescriptor(rmtNode, connKey);
+
+                    if (oldFut == null) {
+                        curClients = clients.get(sndId);
+
+                        oldClient = curClients != null && connKey.connectionIndex() < curClients.length ?
+                            curClients[connKey.connectionIndex()] : null;
+
+                        if (oldClient != null) {
+                            if (oldClient instanceof GridTcpNioCommunicationClient) {
+                                assert oldClient.connectionIndex() == connKey.connectionIndex() : oldClient;
+
+                                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));
+
+                                fut.onDone(oldClient);
+
+                                return;
+                            }
+                            else {
+                                assert oldClient instanceof GridShmemCommunicationClient;
+
+                                hasShmemClient = true;
+                            }
+                        }
+
+                        boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+                            new ConnectClosure(ses, recoveryDesc, rmtNode, connKey, msg0, !hasShmemClient, fut));
+
+                        if (log.isDebugEnabled())
+                            log.debug("Received incoming connection from remote node " +
                                 "[rmtNode=" + rmtNode.id() + ", reserved=" + reserved + ']');
 
-                    if (reserved) {
-                        try {
-                            GridTcpNioCommunicationClient client =
+                        if (reserved) {
+                            try {
+                                GridTcpNioCommunicationClient client =
                                     connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
 
-                            fut.onDone(client);
-                        }
-                        finally {
-                            clientFuts.remove(rmtNode.id(), fut);
+                                fut.onDone(client);
+                            }
+                            finally {
+                                clientFuts.remove(connKey, fut);
+                            }
                         }
                     }
-                }
-                else {
-                    if (oldFut instanceof ConnectFuture && locNode.order() < rmtNode.order()) {
-                        if (log.isDebugEnabled()) {
-                            log.debug("Received incoming connection from remote node while " +
+                    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 {
-                        // The code below causes a race condition between shmem and TCP (see IGNITE-1294)
-                        boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
-                                new ConnectClosure(ses, recoveryDesc, rmtNode, msg0, !hasShmemClient, fut));
+                            ses.send(new RecoveryLastReceivedMessage(-1));
+                        }
+                        else {
+                            // The code below causes a race condition between shmem and TCP (see IGNITE-1294)
+                            boolean reserved = recoveryDesc.tryReserve(msg0.connectCount(),
+                                new ConnectClosure(ses, recoveryDesc, rmtNode, connKey, msg0, !hasShmemClient, fut));
 
-                        if (reserved)
-                            connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
+                            if (reserved)
+                                connected(recoveryDesc, ses, rmtNode, msg0.received(), true, !hasShmemClient);
+                        }
                     }
                 }
             }
 
             @Override public void onMessage(GridNioSession ses, Message msg) {
-                UUID sndId = ses.meta(NODE_ID_META);
+                ConnectionKey connKey = ses.meta(CONN_IDX_META);
 
-                if (sndId == null) {
+                if (connKey == null) {
                     assert ses.accepted() : ses;
 
                     if (!connectGate.tryEnter()) {
@@ -555,29 +624,37 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 else {
                     rcvdMsgsCnt.increment();
 
-                    GridNioRecoveryDescriptor recovery = ses.recoveryDescriptor();
+                    if (msg instanceof RecoveryLastReceivedMessage) {
+                        GridNioRecoveryDescriptor recovery = ses.outRecoveryDescriptor();
 
-                    if (recovery != null) {
-                        if (msg instanceof RecoveryLastReceivedMessage) {
+                        if (recovery != null) {
                             RecoveryLastReceivedMessage msg0 = (RecoveryLastReceivedMessage)msg;
 
-                            if (log.isDebugEnabled())
-                                log.debug("Received recovery acknowledgement [rmtNode=" + sndId +
+                            if (log.isDebugEnabled()) {
+                                log.debug("Received recovery acknowledgement [rmtNode=" + connKey.nodeId() +
+                                    ", connIdx=" + connKey.connectionIndex() +
                                     ", rcvCnt=" + msg0.received() + ']');
+                            }
 
                             recovery.ackReceived(msg0.received());
 
                             return;
                         }
-                        else {
+                    }
+                    else {
+                        GridNioRecoveryDescriptor recovery = ses.inRecoveryDescriptor();
+
+                        if (recovery != null) {
                             long rcvCnt = recovery.onReceived();
 
                             if (rcvCnt % ackSndThreshold == 0) {
-                                if (log.isDebugEnabled())
-                                    log.debug("Send recovery acknowledgement [rmtNode=" + sndId +
+                                if (log.isDebugEnabled()) {
+                                    log.debug("Send recovery acknowledgement [rmtNode=" + connKey.nodeId() +
+                                        ", connIdx=" + connKey.connectionIndex() +
                                         ", rcvCnt=" + rcvCnt + ']');
+                                }
 
-                                nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(rcvCnt));
+                                ses.systemMessage(new RecoveryLastReceivedMessage(rcvCnt));
 
                                 recovery.lastAcknowledged(rcvCnt);
                             }
@@ -603,7 +680,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     else
                         c = NOOP;
 
-                    notifyListener(sndId, msg, c);
+                    notifyListener(connKey.nodeId(), msg, c);
                 }
             }
 
@@ -611,7 +688,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
              * @param recovery Recovery descriptor.
              * @param ses Session.
              * @param node Node.
-             * @param rcvCnt Number of received messages..
+             * @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.
@@ -623,32 +700,128 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 long rcvCnt,
                 boolean sndRes,
                 boolean createClient) {
+                ConnectionKey connKey = ses.meta(CONN_IDX_META);
+
+                assert connKey != null && connKey.connectionIndex() >= 0 : connKey;
+                assert !usePairedConnections(node);
+
                 recovery.onHandshake(rcvCnt);
 
-                ses.recoveryDescriptor(recovery);
+                ses.inRecoveryDescriptor(recovery);
+                ses.outRecoveryDescriptor(recovery);
 
                 nioSrvr.resend(ses);
 
-                if (sndRes)
-                    nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.received()));
+                try {
+                    if (sndRes)
+                        nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.received()));
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to send message: " + e, e);
+                }
 
-                recovery.connected();
+                recovery.onConnected();
 
                 GridTcpNioCommunicationClient client = null;
 
                 if (createClient) {
-                    client = new GridTcpNioCommunicationClient(ses, log);
-
-                    GridCommunicationClient oldClient = clients.putIfAbsent(node.id(), client);
+                    client = new GridTcpNioCommunicationClient(connKey.connectionIndex(), ses, log);
 
-                    assert oldClient == null : "Client already created [node=" + node + ", client=" + client +
-                        ", oldClient=" + oldClient + ", recoveryDesc=" + recovery + ']';
+                    addNodeClient(node, connKey.connectionIndex(), client);
                 }
 
                 return client;
             }
 
             /**
+             * @param recovery Recovery descriptor.
+             * @param ses Session.
+             * @param sndRes If {@code true} sends response for recovery handshake.
+             */
+            private void connectedNew(
+                GridNioRecoveryDescriptor recovery,
+                GridNioSession ses,
+                boolean sndRes) {
+                try {
+                    ses.inRecoveryDescriptor(recovery);
+
+                    if (sndRes)
+                        nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recovery.received()));
+
+                    recovery.onConnected();
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to send message: " + e, e);
+                }
+            }
+
+            /**
+             *
+             */
+            class ConnectClosureNew implements IgniteInClosure<Boolean> {
+                /** */
+                private static final long serialVersionUID = 0L;
+
+                /** */
+                private final GridNioSession ses;
+
+                /** */
+                private final GridNioRecoveryDescriptor recoveryDesc;
+
+                /** */
+                private final ClusterNode rmtNode;
+
+                /** */
+                private boolean failed;
+
+                /**
+                 * @param ses Incoming session.
+                 * @param recoveryDesc Recovery descriptor.
+                 * @param rmtNode Remote node.
+                 */
+                ConnectClosureNew(GridNioSession ses,
+                    GridNioRecoveryDescriptor recoveryDesc,
+                    ClusterNode rmtNode) {
+                    this.ses = ses;
+                    this.recoveryDesc = recoveryDesc;
+                    this.rmtNode = rmtNode;
+                }
+
+                /** {@inheritDoc} */
+                @Override public void apply(Boolean success) {
+                    try {
+                        failed = !success;
+
+                        if (success) {
+                            IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
+                                @Override public void apply(IgniteInternalFuture<?> msgFut) {
+                                    try {
+                                        msgFut.get();
+
+                                        connectedNew(recoveryDesc, ses, false);
+                                    }
+                                    catch (IgniteCheckedException e) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Failed to send recovery handshake " +
+                                                    "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
+
+                                        recoveryDesc.release();
+                                    }
+                                }
+                            };
+
+                            nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.received()), lsnr);
+                        }
+                        else
+                            nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(-1));
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to send message: " + e, e);
+                    }
+                }
+            }
+
+            /**
              *
              */
             @SuppressWarnings("PackageVisibleInnerClass")
@@ -674,10 +847,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 /** */
                 private final boolean createClient;
 
+                /** */
+                private final ConnectionKey connKey;
+
                 /**
                  * @param ses Incoming session.
                  * @param recoveryDesc Recovery descriptor.
                  * @param rmtNode Remote node.
+                 * @param connKey Connection key.
                  * @param msg Handshake message.
                  * @param createClient If {@code true} creates NIO communication client..
                  * @param fut Connect future.
@@ -685,12 +862,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ConnectClosure(GridNioSession ses,
                     GridNioRecoveryDescriptor recoveryDesc,
                     ClusterNode rmtNode,
+                    ConnectionKey connKey,
                     HandshakeMessage msg,
                     boolean createClient,
                     GridFutureAdapter<GridCommunicationClient> fut) {
                     this.ses = ses;
                     this.recoveryDesc = recoveryDesc;
                     this.rmtNode = rmtNode;
+                    this.connKey = connKey;
                     this.msg = msg;
                     this.createClient = createClient;
                     this.fut = fut;
@@ -699,39 +878,44 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 /** {@inheritDoc} */
                 @Override public void apply(Boolean success) {
                     if (success) {
-                        IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
-                            @Override public void apply(IgniteInternalFuture<?> msgFut) {
-                                try {
-                                    msgFut.get();
+                        try {
+                            IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
+                                @Override public void apply(IgniteInternalFuture<?> msgFut) {
+                                    try {
+                                        msgFut.get();
 
-                                    GridTcpNioCommunicationClient client =
-                                        connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient);
+                                        GridTcpNioCommunicationClient client =
+                                                connected(recoveryDesc, ses, rmtNode, msg.received(), false, createClient);
 
-                                    fut.onDone(client);
-                                }
-                                catch (IgniteCheckedException e) {
-                                    if (log.isDebugEnabled())
-                                        log.debug("Failed to send recovery handshake " +
-                                            "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
+                                        fut.onDone(client);
+                                    }
+                                    catch (IgniteCheckedException e) {
+                                        if (log.isDebugEnabled())
+                                            log.debug("Failed to send recovery handshake " +
+                                                    "[rmtNode=" + rmtNode.id() + ", err=" + e + ']');
 
-                                    recoveryDesc.release();
+                                        recoveryDesc.release();
 
-                                    fut.onDone();
-                                }
-                                finally {
-                                    clientFuts.remove(rmtNode.id(), fut);
+                                        fut.onDone();
+                                    }
+                                    finally {
+                                        clientFuts.remove(connKey, fut);
+                                    }
                                 }
-                            }
-                        };
+                            };
 
-                        nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.received()), lsnr);
+                            nioSrvr.sendSystem(ses, new RecoveryLastReceivedMessage(recoveryDesc.received()), lsnr);
+                        }
+                        catch (IgniteCheckedException e) {
+                            U.error(log, "Failed to send message: " + e, e);
+                        }
                     }
                     else {
                         try {
                             fut.onDone();
                         }
                         finally {
-                            clientFuts.remove(rmtNode.id(), fut);
+                            clientFuts.remove(connKey, fut);
                         }
                     }
                 }
@@ -794,6 +978,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Shared memory server. */
     private IpcSharedMemoryServerEndpoint shmemSrv;
 
+    /** */
+    private boolean usePairedConnections = true;
+
+    /** */
+    private int connectionsPerNode = DFLT_CONN_PER_NODE;
+
     /** {@code TCP_NODELAY} option value for created sockets. */
     private boolean tcpNoDelay = DFLT_TCP_NODELAY;
 
@@ -816,7 +1006,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     private final Collection<ShmemWorker> shmemWorkers = new ConcurrentLinkedDeque8<>();
 
     /** Clients. */
-    private final ConcurrentMap<UUID, GridCommunicationClient> clients = GridConcurrentFactory.newMap();
+    private final ConcurrentMap<UUID, GridCommunicationClient[]> clients = GridConcurrentFactory.newMap();
 
     /** SPI listener. */
     private volatile CommunicationListener<Message> lsnr;
@@ -830,6 +1020,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /** Count of selectors to use in TCP server. */
     private int selectorsCnt = DFLT_SELECTORS_CNT;
 
+    /**
+     * Defines how many non-blocking {@code selector.selectNow()} should be made before
+     * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+     * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+     */
+    private long selectorSpins = IgniteSystemProperties.getLong("IGNITE_SELECTOR_SPINS", 0L);
+
     /** Address resolver. */
     private AddressResolver addrRslvr;
 
@@ -863,11 +1060,17 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     };
 
     /** Client connect futures. */
-    private final ConcurrentMap<UUID, GridFutureAdapter<GridCommunicationClient>> clientFuts =
+    private final ConcurrentMap<ConnectionKey, GridFutureAdapter<GridCommunicationClient>> clientFuts =
         GridConcurrentFactory.newMap();
 
     /** */
-    private final ConcurrentMap<ClientKey, GridNioRecoveryDescriptor> recoveryDescs = GridConcurrentFactory.newMap();
+    private final ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> recoveryDescs = GridConcurrentFactory.newMap();
+
+    /** */
+    private final ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> outRecDescs = GridConcurrentFactory.newMap();
+
+    /** */
+    private final ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> inRecDescs = GridConcurrentFactory.newMap();
 
     /** Discovery listener. */
     private final GridLocalEventListener discoLsnr = new GridLocalEventListener() {
@@ -976,6 +1179,49 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return locPortRange;
     }
 
+    /** {@inheritDoc} */
+    @Override public boolean isUsePairedConnections() {
+        return usePairedConnections;
+    }
+
+    /**
+     * Set this to {@code true} if {@code TcpCommunicationSpi} should
+     * maintain connection for outgoing and incoming messages separately.
+     * In this case total number of connections between local and each remote node
+     * is {@link #getConnectionsPerNode()} * 2.
+     * <p>
+     * Set this to {@code false} if each connection of {@link #getConnectionsPerNode()}
+     * should be used for outgoing and incoming messages. In this case total number
+     * of connections between local and each remote node is {@link #getConnectionsPerNode()}.
+     * In this case load NIO selectors load
+     * balancing of {@link GridNioServer} will be disabled.
+     * <p>
+     * Default is {@code true}.
+     *
+     * @param usePairedConnections {@code true} to use paired connections and {@code false} otherwise.
+     * @see #getConnectionsPerNode()
+     */
+    public void setUsePairedConnections(boolean usePairedConnections) {
+        this.usePairedConnections = usePairedConnections;
+    }
+
+    /**
+     * Sets number of connections to each remote node. if {@link #isUsePairedConnections()}
+     * is {@code true} then number of connections is doubled and half is used for incoming and
+     * half for outgoing messages.
+     *
+     * @param maxConnectionsPerNode Number of connections per node.
+     * @see #isUsePairedConnections()
+     */
+    public void setConnectionsPerNode(int maxConnectionsPerNode) {
+        this.connectionsPerNode = maxConnectionsPerNode;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int getConnectionsPerNode() {
+        return connectionsPerNode;
+    }
+
     /**
      * Sets local port to accept shared memory connections.
      * <p>
@@ -1222,6 +1468,22 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         return selectorsCnt;
     }
 
+    /** {@inheritDoc} */
+    @Override public long getSelectorSpins() {
+        return selectorSpins;
+    }
+
+    /**
+     * Defines how many non-blocking {@code selector.selectNow()} should be made before
+     * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+     * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+     *
+     * @param selectorSpins Selector thread busy-loop iterations.
+     */
+    public void setSelectorSpins(long selectorSpins) {
+        this.selectorSpins = selectorSpins;
+    }
+
     /**
      * Sets value for {@code TCP_NODELAY} socket option. Each
      * socket will be opened using provided value.
@@ -1396,7 +1658,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         if (log != null) {
             StringBuilder sb = new StringBuilder("Communication SPI recovery descriptors: ").append(U.nl());
 
-            for (Map.Entry<ClientKey, GridNioRecoveryDescriptor> entry : recoveryDescs.entrySet()) {
+            for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> entry : recoveryDescs.entrySet()) {
                 GridNioRecoveryDescriptor desc = entry.getValue();
 
                 sb.append("    [key=").append(entry.getKey())
@@ -1409,14 +1671,48 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     .append(']').append(U.nl());
             }
 
-            sb.append("Communication SPI clients: ").append(U.nl());
+            for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> entry : outRecDescs.entrySet()) {
+                GridNioRecoveryDescriptor desc = entry.getValue();
+
+                sb.append("    [key=").append(entry.getKey())
+                    .append(", msgsSent=").append(desc.sent())
+                    .append(", msgsAckedByRmt=").append(desc.acked())
+                    .append(", reserveCnt=").append(desc.reserveCount())
+                    .append(", connected=").append(desc.connected())
+                    .append(", reserved=").append(desc.reserved())
+                    .append(", descIdHash=").append(System.identityHashCode(desc))
+                    .append(']').append(U.nl());
+            }
+
+            for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> entry : inRecDescs.entrySet()) {
+                GridNioRecoveryDescriptor desc = entry.getValue();
 
-            for (Map.Entry<UUID, GridCommunicationClient> entry : clients.entrySet()) {
-                sb.append("    [node=").append(entry.getKey())
-                    .append(", client=").append(entry.getValue())
+                sb.append("    [key=").append(entry.getKey())
+                    .append(", msgsRcvd=").append(desc.received())
+                    .append(", lastAcked=").append(desc.lastAcknowledged())
+                    .append(", reserveCnt=").append(desc.reserveCount())
+                    .append(", connected=").append(desc.connected())
+                    .append(", reserved=").append(desc.reserved())
+                    .append(", handshakeIdx=").append(desc.handshakeIndex())
+                    .append(", descIdHash=").append(System.identityHashCode(desc))
                     .append(']').append(U.nl());
             }
 
+            sb.append("Communication SPI clients: ").append(U.nl());
+
+            for (Map.Entry<UUID, GridCommunicationClient[]> entry : clients.entrySet()) {
+                UUID nodeId = entry.getKey();
+                GridCommunicationClient[] clients0 = entry.getValue();
+
+                for (GridCommunicationClient client : clients0) {
+                    if (client != null) {
+                        sb.append("    [node=").append(nodeId)
+                            .append(", client=").append(client)
+                            .append(']').append(U.nl());
+                    }
+                }
+            }
+
             U.warn(log, sb.toString());
         }
 
@@ -1426,6 +1722,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             nioSrvr.dumpStats();
     }
 
+    /** */
+    private final ThreadLocal<Integer> threadConnIdx = new ThreadLocal<>();
+
+    /** */
+    private final AtomicInteger connIdx = new AtomicInteger();
+
     /** {@inheritDoc} */
     @Override public Map<String, Object> getNodeAttributes() throws IgniteSpiException {
         initFailureDetectionTimeout();
@@ -1439,6 +1741,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         assertParameter(msgQueueLimit >= 0, "msgQueueLimit >= 0");
         assertParameter(shmemPort > 0 || shmemPort == -1, "shmemPort > 0 || shmemPort == -1");
         assertParameter(selectorsCnt > 0, "selectorsCnt > 0");
+        assertParameter(connectionsPerNode > 0, "connectionsPerNode > 0");
+        assertParameter(connectionsPerNode <= 1024, "connectionsPerNode <= 1024");
 
         if (!failureDetectionTimeoutEnabled()) {
             assertParameter(reconCnt > 0, "reconnectCnt > 0");
@@ -1458,6 +1762,21 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 "Specified 'unackedMsgsBufSize' is too low, it should be at least 'ackSndThreshold * 5'.");
         }
 
+        if (connectionsPerNode > 1) {
+            connPlc = new ConnectionPolicy() {
+                @Override public int connectionIndex() {
+                    return (int)(U.safeAbs(Thread.currentThread().getId()) % connectionsPerNode);
+                }
+            };
+        }
+        else {
+            connPlc = new ConnectionPolicy() {
+                @Override public int connectionIndex() {
+                    return 0;
+                }
+            };
+        }
+
         try {
             locHost = U.resolveLocalHost(locAddr);
         }
@@ -1495,6 +1814,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             res.put(createSpiAttributeName(ATTR_PORT), boundTcpPort);
             res.put(createSpiAttributeName(ATTR_SHMEM_PORT), boundTcpShmemPort >= 0 ? boundTcpShmemPort : null);
             res.put(createSpiAttributeName(ATTR_EXT_ADDRS), extAddrs);
+            res.put(createSpiAttributeName(ATTR_PAIRED_CONN), usePairedConnections);
 
             return res;
         }
@@ -1524,6 +1844,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             log.debug(configInfo("sockRcvBuf", sockRcvBuf));
             log.debug(configInfo("shmemPort", shmemPort));
             log.debug(configInfo("msgQueueLimit", msgQueueLimit));
+            log.debug(configInfo("connectionsPerNode", connectionsPerNode));
 
             if (failureDetectionTimeoutEnabled()) {
                 log.debug(configInfo("connTimeout", connTimeout));
@@ -1548,6 +1869,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 ", slowClientQueueLimit=" + slowClientQueueLimit + ']');
         }
 
+        if (msgQueueLimit == 0)
+            U.quietAndWarn(log, "Message queue limit is set to 0 which may lead to " +
+                "potential OOMEs when running cache operations in FULL_ASYNC or PRIMARY_SYNC modes " +
+                "due to message queues growth on sender and reciever sides.");
+
         registerMBean(gridName, this, TcpCommunicationSpiMBean.class);
 
         connectGate = new ConnectGateway();
@@ -1642,9 +1968,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         assert formatter != null;
 
-                        UUID rmtNodeId = ses.meta(NODE_ID_META);
+                        ConnectionKey key = ses.meta(CONN_IDX_META);
 
-                        return rmtNodeId != null ? formatter.reader(rmtNodeId, msgFactory) : null;
+                        return key != null ? formatter.reader(key.nodeId(), msgFactory) : null;
                     }
                 };
 
@@ -1657,9 +1983,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         assert formatter != null;
 
-                        UUID rmtNodeId = ses.meta(NODE_ID_META);
+                        ConnectionKey key = ses.meta(CONN_IDX_META);
 
-                        return rmtNodeId != null ? formatter.writer(rmtNodeId) : null;
+                        return key != null ? formatter.writer(key.nodeId()) : null;
                     }
                 };
 
@@ -1716,6 +2042,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         .logger(log)
                         .selectorCount(selectorsCnt)
                         .gridName(gridName)
+                        .serverName("tcp-comm")
                         .tcpNoDelay(tcpNoDelay)
                         .directBuffer(directBuf)
                         .byteOrder(ByteOrder.nativeOrder())
@@ -1725,18 +2052,21 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         .directMode(true)
                         .metricsListener(metricsLsnr)
                         .writeTimeout(sockWriteTimeout)
+                        .selectorSpins(selectorSpins)
                         .filters(filters)
                         .writerFactory(writerFactory)
                         .skipRecoveryPredicate(skipRecoveryPred)
                         .messageQueueSizeListener(queueSizeMonitor)
+                        .balancing(usePairedConnections) // Current balancing logic assumes separate in/out connections.
                         .build();
 
                 boundTcpPort = port;
 
                 // Ack Port the TCP server was bound to.
                 if (log.isInfoEnabled())
-                    log.info("Successfully bound to TCP port [port=" + boundTcpPort +
-                        ", locHost=" + locHost + ']');
+                    log.info("Successfully bound communication NIO server to TCP port " +
+                        "[port=" + boundTcpPort + ", locHost=" + locHost + ", selectorsCnt=" + selectorsCnt +
+                        ", selectorSpins=" + srvr.selectorSpins() + ']');
 
                 srvr.idleTimeout(idleConnTimeout);
 
@@ -1837,8 +2167,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         shmemWorkers.clear();
 
         // Force closing on stop (safety).
-        for (GridCommunicationClient client : clients.values())
-            client.forceClose();
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null)
+                    client.forceClose();
+            }
+        }
 
         // Clear resources.
         nioSrvr = null;
@@ -1863,8 +2197,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             connectGate.stopped();
 
         // Force closing.
-        for (GridCommunicationClient client : clients.values())
-            client.forceClose();
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null)
+                    client.forceClose();
+            }
+        }
 
         getSpiContext().deregisterPorts();
 
@@ -1875,8 +2213,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     @Override public void onClientDisconnected(IgniteFuture<?> reconnectFut) {
         connectGate.disconnected(reconnectFut);
 
-        for (GridCommunicationClient client : clients.values())
-            client.forceClose();
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null)
+                    client.forceClose();
+            }
+        }
 
         IgniteClientDisconnectedCheckedException err = new IgniteClientDisconnectedCheckedException(reconnectFut,
             "Failed to connect client node disconnected.");
@@ -1885,6 +2227,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             clientFut.onDone(err);
 
         recoveryDescs.clear();
+        inRecDescs.clear();
+        outRecDescs.clear();
     }
 
     /** {@inheritDoc} */
@@ -1898,16 +2242,18 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     void onNodeLeft(UUID nodeId) {
         assert nodeId != null;
 
-        GridCommunicationClient client = clients.get(nodeId);
-
-        if (client != null) {
-            if (log.isDebugEnabled())
-                log.debug("Forcing NIO client close since node has left [nodeId=" + nodeId +
-                    ", client=" + client + ']');
+        GridCommunicationClient[] clients0 = clients.remove(nodeId);
 
-            client.forceClose();
+        if (clients0 != null) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null) {
+                    if (log.isDebugEnabled())
+                        log.debug("Forcing NIO client close since node has left [nodeId=" + nodeId +
+                            ", client=" + client + ']');
 
-            clients.remove(nodeId, client);
+                    client.forceClose();
+                }
+            }
         }
     }
 
@@ -1982,11 +2328,13 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         else {
             GridCommunicationClient client = null;
 
+            int connIdx = useMultipleConnections(node) ? connPlc.connectionIndex() : 0;
+
             try {
                 boolean retry;
 
                 do {
-                    client = reserveClient(node);
+                    client = reserveClient(node, connIdx);
 
                     UUID nodeId = null;
 
@@ -2000,7 +2348,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     if (!retry)
                         sentMsgsCnt.increment();
                     else {
-                        clients.remove(node.id(), client);
+                        removeNodeClient(node.id(), client);
 
                         ClusterNode node0 = getSpiContext().node(node.id());
 
@@ -2017,26 +2365,94 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 throw new IgniteSpiException("Failed to send message to remote node: " + node, e);
             }
             finally {
-                if (client != null && clients.remove(node.id(), client))
+                if (client != null && removeNodeClient(node.id(), client))
                     client.forceClose();
             }
         }
     }
 
     /**
+     * @param nodeId Node ID.
+     * @param rmvClient Client to remove.
+     * @return {@code True} if client was removed.
+     */
+    private boolean removeNodeClient(UUID nodeId, GridCommunicationClient rmvClient) {
+        for (;;) {
+            GridCommunicationClient[] curClients = clients.get(nodeId);
+
+            if (curClients == null || rmvClient.connectionIndex() >= curClients.length || curClients[rmvClient.connectionIndex()] != rmvClient)
+                return false;
+
+            GridCommunicationClient[] newClients = Arrays.copyOf(curClients, curClients.length);
+
+            newClients[rmvClient.connectionIndex()] = null;
+
+            if (clients.replace(nodeId, curClients, newClients))
+                return true;
+        }
+    }
+
+    /**
+     * @param node Node.
+     * @param connIdx Connection index.
+     * @param addClient Client to add.
+     */
+    private void addNodeClient(ClusterNode node, int connIdx, GridCommunicationClient addClient) {
+        assert connectionsPerNode > 0 : connectionsPerNode;
+        assert connIdx == addClient.connectionIndex() : addClient;
+
+        if (connIdx >= connectionsPerNode) {
+            assert !usePairedConnections(node);
+
+            return;
+        }
+
+        for (;;) {
+            GridCommunicationClient[] curClients = clients.get(node.id());
+
+            assert curClients == null || curClients[connIdx] == null : "Client already created [node=" + node.id() +
+                ", connIdx=" + connIdx +
+                ", client=" + addClient +
+                ", oldClient=" + curClients[connIdx] + ']';
+
+            GridCommunicationClient[] newClients;
+
+            if (curClients == null) {
+                newClients = new GridCommunicationClient[useMultipleConnections(node) ? connectionsPerNode : 1];
+                newClients[connIdx] = addClient;
+
+                if (clients.putIfAbsent(node.id(), newClients) == null)
+                    break;
+            }
+            else {
+                newClients = Arrays.copyOf(curClients, curClients.length);
+                newClients[connIdx] = addClient;
+
+                if (clients.replace(node.id(), curClients, newClients))
+                    break;
+            }
+        }
+    }
+
+    /**
      * Returns existing or just created client to node.
      *
      * @param node Node to which client should be open.
+     * @param connIdx Connection index.
      * @return The existing or just created client.
      * @throws IgniteCheckedException Thrown if any exception occurs.
      */
-    private GridCommunicationClient reserveClient(ClusterNode node) throws IgniteCheckedException {
+    private GridCommunicationClient reserveClient(ClusterNode node, int connIdx) throws IgniteCheckedException {
         assert node != null;
+        assert (connIdx >= 0 && connIdx < connectionsPerNode) || !usePairedConnections(node) : connIdx;
 
         UUID nodeId = node.id();
 
         while (true) {
-            GridCommunicationClient client = clients.get(nodeId);
+            GridCommunicationClient[] curClients = clients.get(nodeId);
+
+            GridCommunicationClient client = curClients != null && connIdx < curClients.length ?
+                curClients[connIdx] : null;
 
             if (client == null) {
                 if (stopping)
@@ -2045,25 +2461,27 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                 // Do not allow concurrent connects.
                 GridFutureAdapter<GridCommunicationClient> fut = new ConnectFuture();
 
-                GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(nodeId, fut);
+                ConnectionKey connKey = new ConnectionKey(nodeId, connIdx, -1);
+
+                GridFutureAdapter<GridCommunicationClient> oldFut = clientFuts.putIfAbsent(connKey, fut);
 
                 if (oldFut == null) {
                     try {
-                        GridCommunicationClient client0 = clients.get(nodeId);
+                        GridCommunicationClient[] curClients0 = clients.get(nodeId);
+
+                        GridCommunicationClient client0 = curClients0 != null && connIdx < curClients0.length ?
+                            curClients0[connIdx] : null;
 
                         if (client0 == null) {
-                            client0 = createNioClient(node);
+                            client0 = createNioClient(node, connIdx);
 
                             if (client0 != null) {
-                                GridCommunicationClient old = clients.put(nodeId, client0);
-
-                                assert old == null : "Client already created " +
-                                    "[node=" + node + ", client=" + client0 + ", oldClient=" + old + ']';
+                                addNodeClient(node, connIdx, client0);
 
                                 if (client0 instanceof GridTcpNioCommunicationClient) {
                                     GridTcpNioCommunicationClient tcpClient = ((GridTcpNioCommunicationClient)client0);
 
-                                    if (tcpClient.session().closeTime() > 0 && clients.remove(nodeId, client0)) {
+                                    if (tcpClient.session().closeTime() > 0 && removeNodeClient(nodeId, client0)) {
                                         if (log.isDebugEnabled())
                                             log.debug("Session was closed after client creation, will retry " +
                                                 "[node=" + node + ", client=" + client0 + ']');
@@ -2085,7 +2503,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             throw (Error)e;
                     }
                     finally {
-                        clientFuts.remove(nodeId, fut);
+                        clientFuts.remove(connKey, fut);
                     }
                 }
                 else
@@ -2097,27 +2515,31 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     continue;
 
                 if (getSpiContext().node(nodeId) == null) {
-                    if (clients.remove(nodeId, client))
+                    if (removeNodeClient(nodeId, client))
                         client.forceClose();
 
                     throw new IgniteSpiException("Destination node is not in topology: " + node.id());
                 }
             }
 
+            assert connIdx == client.connectionIndex() : client;
+
             if (client.reserve())
                 return client;
             else
                 // Client has just been closed by idle worker. Help it and try again.
-                clients.remove(nodeId, client);
+                removeNodeClient(nodeId, client);
         }
     }
 
     /**
      * @param node Node to create client for.
+     * @param connIdx Connection index.
      * @return Client.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable protected GridCommunicationClient createNioClient(ClusterNode node) throws IgniteCheckedException {
+    @Nullable private GridCommunicationClient createNioClient(ClusterNode node, int connIdx)
+        throws IgniteCheckedException {
         assert node != null;
 
         Integer shmemPort = node.attribute(createSpiAttributeName(ATTR_SHMEM_PORT));
@@ -2136,6 +2558,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             try {
                 GridCommunicationClient client = createShmemClient(
                     node,
+                    connIdx,
                     shmemPort);
 
                 if (log.isDebugEnabled())
@@ -2158,7 +2581,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         connectGate.enter();
 
         try {
-            GridCommunicationClient client = createTcpClient(node);
+            GridCommunicationClient client = createTcpClient(node, connIdx);
 
             if (log.isDebugEnabled())
                 log.debug("TCP client created: " + client);
@@ -2173,10 +2596,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     /**
      * @param node Node.
      * @param port Port.
+     * @param connIdx Connection index.
      * @return Client.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable protected GridCommunicationClient createShmemClient(ClusterNode node,
+    @Nullable private GridCommunicationClient createShmemClient(ClusterNode node,
+        int connIdx,
         Integer port) throws IgniteCheckedException {
         int attempt = 1;
 
@@ -2190,7 +2615,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             GridCommunicationClient client;
 
             try {
-                client = new GridShmemCommunicationClient(metricsLsnr,
+                client = new GridShmemCommunicationClient(
+                    connIdx,
+                    metricsLsnr,
                     port,
                     timeoutHelper.nextTimeoutChunk(connTimeout),
                     log,
@@ -2211,7 +2638,12 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             }
 
             try {
-                safeHandshake(client, null, node.id(), timeoutHelper.nextTimeoutChunk(connTimeout0), null);
+                safeHandshake(client,
+                    null,
+                    node.id(),
+                    timeoutHelper.nextTimeoutChunk(connTimeout0),
+                    null,
+                    null);
             }
             catch (HandshakeTimeoutException | IgniteSpiOperationTimeoutException e) {
                 client.forceClose();
@@ -2270,10 +2702,10 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      */
     private void checkClientQueueSize(GridNioSession ses, int msgQueueSize) {
         if (slowClientQueueLimit > 0 && msgQueueSize > slowClientQueueLimit) {
-            UUID id = ses.meta(NODE_ID_META);
+            ConnectionKey id = ses.meta(CONN_IDX_META);
 
             if (id != null) {
-                ClusterNode node = getSpiContext().node(id);
+                ClusterNode node = getSpiContext().node(id.nodeId);
 
                 if (node != null && node.isClient()) {
                     String msg = "Client node outbound message queue size exceeded slowClientQueueLimit, " +
@@ -2283,11 +2715,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         ", clientNode=" + node +
                         ", slowClientQueueLimit=" + slowClientQueueLimit + ']';
 
-                    U.quietAndWarn(
-                        log,
-                        msg);
+                    U.quietAndWarn(log, msg);
 
-                    getSpiContext().failNode(id, msg);
+                    getSpiContext().failNode(id.nodeId(), msg);
                 }
             }
         }
@@ -2297,10 +2727,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * Establish TCP connection to remote node and returns client.
      *
      * @param node Remote node.
+     * @param connIdx Connection index.
      * @return Client.
      * @throws IgniteCheckedException If failed.
      */
-    protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException {
+    protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx) throws IgniteCheckedException {
         Collection<String> rmtAddrs0 = node.attribute(createSpiAttributeName(ATTR_ADDRS));
         Collection<String> rmtHostNames0 = node.attribute(createSpiAttributeName(ATTR_HOST_NAMES));
         Integer boundPort = node.attribute(createSpiAttributeName(ATTR_PORT));
@@ -2368,7 +2799,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             "(node left topology): " + node);
                     }
 
-                    GridNioRecoveryDescriptor recoveryDesc = recoveryDescriptor(node);
+                    ConnectionKey connKey = new ConnectionKey(node.id(), connIdx, -1);
+
+                    GridNioRecoveryDescriptor recoveryDesc = outRecoveryDescriptor(node, connKey);
 
                     if (!recoveryDesc.reserve()) {
                         U.closeQuiet(ch);
@@ -2395,11 +2828,14 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             sslMeta.sslEngine(sslEngine);
                         }
 
+                        Integer handshakeConnIdx = useMultipleConnections(node) ? connIdx : null;
+
                         rcvCnt = safeHandshake(ch,
                             recoveryDesc,
                             node.id(),
                             timeoutHelper.nextTimeoutChunk(connTimeout0),
-                            sslMeta);
+                            sslMeta,
+                            handshakeConnIdx);
 
                         if (rcvCnt == -1)
                             return null;
@@ -2410,7 +2846,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     }
 
                     try {
-                        meta.put(NODE_ID_META, node.id());
+                        meta.put(CONN_IDX_META, connKey);
 
                         if (recoveryDesc != null) {
                             recoveryDesc.onHandshake(rcvCnt);
@@ -2420,7 +2856,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         GridNioSession ses = nioSrvr.createSession(ch, meta).get();
 
-                        client = new GridTcpNioCommunicationClient(ses, log);
+                        client = new GridTcpNioCommunicationClient(connIdx, ses, log);
 
                         conn = true;
                     }
@@ -2564,6 +3000,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
      * @param rmtNodeId Remote node.
      * @param timeout Timeout for handshake.
      * @param sslMeta Session meta.
+     * @param handshakeConnIdx Non null connection index if need send it in handshake.
      * @throws IgniteCheckedException If handshake failed or wasn't completed withing timeout.
      * @return Handshake response.
      */
@@ -2573,7 +3010,8 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         @Nullable GridNioRecoveryDescriptor recovery,
         UUID rmtNodeId,
         long timeout,
-        GridSslMeta sslMeta
+        GridSslMeta sslMeta,
+        @Nullable Integer handshakeConnIdx
     ) throws IgniteCheckedException {
         HandshakeTimeoutObject<T> obj = new HandshakeTimeoutObject<>(client, U.currentTimeMillis() + timeout);
 
@@ -2655,14 +3093,28 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                             "fully initialized [isStopping=" + getSpiContext().isStopping() + ']');
 
                     if (recovery != null) {
-                        HandshakeMessage msg = new HandshakeMessage(locNode.id(),
-                            recovery.incrementConnectCount(),
-                            recovery.received());
+                        HandshakeMessage msg;
+
+                        int msgSize = 33;
+
+                        if (handshakeConnIdx != null) {
+                            msg = new HandshakeMessage2(locNode.id(),
+                                recovery.incrementConnectCount(),
+                                recovery.received(),
+                                handshakeConnIdx);
+
+                            msgSize += 4;
+                        }
+                        else {
+                            msg = new HandshakeMessage(locNode.id(),
+                                recovery.incrementConnectCount(),
+                                recovery.received());
+                        }
 
                         if (log.isDebugEnabled())
                             log.debug("Write handshake message [rmtNode=" + rmtNodeId + ", msg=" + msg + ']');
 
-                        buf = ByteBuffer.allocate(33);
+                        buf = ByteBuffer.allocate(msgSize);
 
                         buf.order(ByteOrder.nativeOrder());
 
@@ -2689,6 +3141,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                         else
                             ch.write(ByteBuffer.wrap(nodeIdMessage().nodeIdBytesWithType));
                     }
+
                     if (recovery != null) {
                         if (log.isDebugEnabled())
                             log.debug("Waiting for handshake [rmtNode=" + rmtNodeId + ']');
@@ -2818,26 +3271,81 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
         U.join(commWorker, log);
 
-        for (GridCommunicationClient client : clients.values())
-            client.forceClose();
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null)
+                    client.forceClose();
+            }
+        }
+    }
+
+    /**
+     * @param node Node.
+     * @param key Connection key.
+     * @return Recovery descriptor for outgoing connection.
+     */
+    private GridNioRecoveryDescriptor outRecoveryDescriptor(ClusterNode node, ConnectionKey key) {
+        if (usePairedConnections(node))
+            return recoveryDescriptor(outRecDescs, true, node, key);
+        else
+            return recoveryDescriptor(recoveryDescs, false, node, key);
     }
 
     /**
      * @param node Node.
-     * @return Recovery receive data for given node.
+     * @param key Connection key.
+     * @return Recovery descriptor for incoming connection.
+     */
+    private GridNioRecoveryDescriptor inRecoveryDescriptor(ClusterNode node, ConnectionKey key) {
+        if (usePairedConnections(node))
+            return recoveryDescriptor(inRecDescs, true, node, key);
+        else
+            return recoveryDescriptor(recoveryDescs, false, node, key);
+    }
+
+    /**
+     * @param node Node.
+     * @return {@code True} if given node supports multiple connections per-node for communication.
+     */
+    private boolean useMultipleConnections(ClusterNode node) {
+        return node.version().compareToIgnoreTimestamp(MULTIPLE_CONN_SINCE_VER) >= 0;
+    }
+
+    /**
+     * @param node Node.
+     * @return {@code True} if can use in/out connection pair for communication.
      */
-    private GridNioRecoveryDescriptor recoveryDescriptor(ClusterNode node) {
-        ClientKey id = new ClientKey(node.id(), node.order());
+    private boolean usePairedConnections(ClusterNode node) {
+        if (usePairedConnections) {
+            Boolean attr = node.attribute(createSpiAttributeName(ATTR_PAIRED_CONN));
+
+            return attr != null && attr;
+        }
+
+        return false;
+    }
 
-        GridNioRecoveryDescriptor recovery = recoveryDescs.get(id);
+    /**
+     * @param recoveryDescs Descriptors map.
+     * @param pairedConnections {@code True} if in/out connections pair is used for communication with node.
+     * @param node Node.
+     * @param key Connection key.
+     * @return Recovery receive data for given node.
+     */
+    private GridNioRecoveryDescriptor recoveryDescriptor(
+        ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> recoveryDescs,
+        boolean pairedConnections,
+        ClusterNode node,
+        ConnectionKey key) {
+        GridNioRecoveryDescriptor recovery = recoveryDescs.get(key);
 
         if (recovery == null) {
             int maxSize = Math.max(msgQueueLimit, ackSndThreshold);
 
-            int queueLimit = unackedMsgsBufSize != 0 ? unackedMsgsBufSize : (maxSize * 5);
+            int queueLimit = unackedMsgsBufSize != 0 ? unackedMsgsBufSize : (maxSize * 128);
 
-            GridNioRecoveryDescriptor old =
-                recoveryDescs.putIfAbsent(id, recovery = new GridNioRecoveryDescriptor(queueLimit, node, log));
+            GridNioRecoveryDescriptor old = recoveryDescs.putIfAbsent(key,
+                recovery = new GridNioRecoveryDescriptor(pairedConnections, queueLimit, node, log));
 
             if (old != null)
                 recovery = old;
@@ -2879,54 +3387,6 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         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 IgniteCheckedException {
         /** */
@@ -3026,9 +3486,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         assert formatter != null;
 
-                        UUID rmtNodeId = ses.meta(NODE_ID_META);
+                        ConnectionKey connKey = ses.meta(CONN_IDX_META);
 
-                        return rmtNodeId != null ? formatter.writer(rmtNodeId) : null;
+                        return connKey != null ? formatter.writer(connKey.nodeId()) : null;
                     }
                 };
 
@@ -3042,9 +3502,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
 
                         assert formatter != null;
 
-                        UUID rmtNodeId = ses.meta(NODE_ID_META);
+                        ConnectionKey connKey = ses.meta(CONN_IDX_META);
 
-                        return rmtNodeId != null ? formatter.reader(rmtNodeId, msgFactory) : null;
+                        return connKey != null ? formatter.reader(connKey.nodeId(), msgFactory) : null;
                     }
                 };
 
@@ -3125,62 +3585,108 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
         private void processIdle() {
             cleanupRecovery();
 
-            for (Map.Entry<UUID, GridCommunicationClient> e : clients.entrySet()) {
+            for (Map.Entry<UUID, GridCommunicationClient[]> e : clients.entrySet()) {
                 UUID nodeId = e.getKey();
 
-                GridCommunicationClient client = e.getValue();
+                for (GridCommunicationClient client : e.getValue()) {
+                    if (client == null)
+                        continue;
 
-                ClusterNode node = getSpiContext().node(nodeId);
+                    ClusterNode node = getSpiContext().node(nodeId);
 
-                if (node == null) {
-                    if (log.isDebugEnabled())
-                        log.debug("Forcing close of non-existent node connection: " + nodeId);
+                    if (node == null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Forcing close of non-existent node connection: " + nodeId);
 
-                    client.forceClose();
+                        client.forceClose();
 
-                    clients.remove(nodeId, client);
+                        removeNodeClient(nodeId, client);
 
-                    continue;
-                }
+                        continue;
+                    }
 
-                GridNioRecoveryDescriptor recovery = null;
+                    GridNioRecoveryDescriptor recovery = null;
 
-                if (client instanceof GridTcpNioCommunicationClient) {
-                    recovery = recoveryDescs.get(new ClientKey(node.id(), node.order()));
+                    if (!usePairedConnections(node) && client instanceof GridTcpNioCommunicationClient) {
+                        recovery = recoveryDescs.get(new ConnectionKey(node.id(), client.connectionIndex(), -1));
 
-                    if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
-                        RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
+                        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() + ']');
+                            if (log.isDebugEnabled())
+                                log.debug("Send recovery acknowledgement on timeout [rmtNode=" + nodeId +
+                                    ", rcvCnt=" + msg.received() + ']');
 
-                        nioSrvr.sendSystem(((GridTcpNioCommunicationClient)client).session(), msg);
+                            try {
+                                nioSrvr.sendSystem(((GridTcpNioCommunicationClient)client).session(), msg);
 
-                        recovery.lastAcknowledged(msg.received());
+                                recovery.lastAcknowledged(msg.received());
+                            }
+                            catch (IgniteCheckedException err) {
+                                U.error(log, "Failed to send message: " + err, err);
+                            }
 
-                        continue;
+                            continue;
+                        }
                     }
-                }
 
-                long idleTime = client.getIdleTime();
+                    long idleTime = client.getIdleTime();
+
+                    if (idleTime >= idleConnTimeout) {
+                        if (recovery == null && usePairedConnections(node))
+                            recovery = outRecDescs.get(new ConnectionKey(node.id(), client.connectionIndex(), -1));
+
+                        if (recovery != null &&
+                            recovery.nodeAlive(getSpiContext().node(nodeId)) &&
+                            !recovery.messagesRequests().isEmpty()) {
+                            if (log.isDebugEnabled())
+                                log.debug("Node connection is idle, but there are unacknowledged messages, " +
+                                    "will wait: " + nodeId);
+
+                            continue;
+                        }
 
-                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);
+                            log.debug("Closing idle node connection: " + nodeId);
 
-                        continue;
+                        if (client.close() || client.closed())
+                            removeNodeClient(nodeId, client);
                     }
+                }
+            }
 
-                    if (log.isDebugEnabled())
-                        log.debug("Closing idle node connection: " + nodeId);
+            for (GridNioSession ses : nioSrvr.sessions()) {
+                GridNioRecoveryDescriptor recovery = ses.inRecoveryDescriptor();
+
+                if (recovery != null && usePairedConnections(recovery.node())) {
+                    assert ses.accepted() : ses;
+
+                    sendAckOnTimeout(recovery, ses);
+                }
+            }
+        }
+
+        /**
+         * @param recovery Recovery descriptor.
+         * @param ses Session.
+         */
+        private void sendAckOnTimeout(GridNioRecoveryDescriptor recovery, GridNioSession ses) {
+            if (recovery != null && recovery.lastAcknowledged() != recovery.received()) {
+                RecoveryLastReceivedMessage msg = new RecoveryLastReceivedMessage(recovery.received());
+
+                if (log.isDebugEnabled()) {
+                    log.debug("Send recovery acknowledgement on timeout [rmtNode=" + recovery.node().id() +
+                        ", rcvCnt=" + msg.received() +
+                        ", lastAcked=" + recovery.lastAcknowledged() + ']');
+                }
+
+                try {
+                    nioSrvr.sendSystem(ses, msg);
 
-                    if (client.close() || client.closed())
-                        clients.remove(nodeId, client);
+                    recovery.lastAcknowledged(msg.received());
+                }
+                catch (IgniteCheckedException e) {
+                    U.error(log, "Failed to send message: " + e, e);
                 }
             }
         }
@@ -3189,15 +3695,24 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
          *
          */
         private void cleanupRecovery() {
-            Set<ClientKey> left = null;
+            cleanupRecovery(recoveryDescs);
+            cleanupRecovery(inRecDescs);
+            cleanupRecovery(outRecDescs);
+        }
+
+        /**
+         * @param recoveryDescs Recovery descriptors to cleanup.
+         */
+        private void cleanupRecovery(ConcurrentMap<ConnectionKey, GridNioRecoveryDescriptor> recoveryDescs) {
+            Set<ConnectionKey> left = null;
 
-            for (Map.Entry<ClientKey, GridNioRecoveryDescriptor> e : recoveryDescs.entrySet()) {
+            for (Map.Entry<ConnectionKey, GridNioRecoveryDescriptor> e : recoveryDescs.entrySet()) {
                 if (left != null && left.contains(e.getKey()))
                     continue;
 
-                GridNioRecoveryDescriptor recoverySnd = e.getValue();
+                GridNioRecoveryDescriptor recoveryDesc = e.getValue();
 
-                if (!recoverySnd.nodeAlive(getSpiContext().node(recoverySnd.node().id()))) {
+                if (!recoveryDesc.nodeAlive(getSpiContext().node(e.getKey().nodeId()))) {
                     if (left == null)
                         left = new HashSet<>();
 
@@ -3208,11 +3723,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             if (left != null) {
                 assert !left.isEmpty();
 
-                for (ClientKey id : left) {
-                    GridNioRecoveryDescriptor recoverySnd = recoveryDescs.get(id);
+                for (ConnectionKey id : left) {
+                    GridNioRecoveryDescriptor recoveryDesc = recoveryDescs.get(id);
 
-                    if (recoverySnd != null && recoverySnd.onNodeLeft())
-                        recoveryDescs.remove(id);
+                    if (recoveryDesc != null && recoveryDesc.onNodeLeft())
+                        recoveryDescs.remove(id, recoveryDesc);
                 }
             }
         }
@@ -3221,45 +3736,43 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
          * @param sesInfo Disconnected session information.
          */
         private void processDisconnect(DisconnectedSessionInfo sesInfo) {
-            if (sesInfo.reconnect) {
-                GridNioRecoveryDescriptor recoveryDesc = sesInfo.recoveryDesc;
-
-                ClusterNode node = recoveryDesc.node();
+            GridNioRecoveryDescriptor recoveryDesc = sesInfo.recoveryDesc;
 
-                if (!recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
-                    return;
+            ClusterNode node = recoveryDesc.node();
 
-                try {
-                    if (log.isDebugEnabled())
-                        log.debug("Recovery reconnect [rmtNode=" + recoveryDesc.node().id() + ']');
+            if (!recoveryDesc.nodeAlive(getSpiContext().node(node.id())))
+                return;
 
-                    GridCommunicationClient client = reserveClient(node);
+            try {
+                if (log.isDebugEnabled())
+                    log.debug("Recovery reconnect [rmtNode=" + recoveryDesc.node().id() + ']');
 
-                    client.release();
-                }
-                catch (IgniteCheckedException | IgniteException e) {
-                    try {
-                        if (recoveryDesc.nodeAlive(getSpiContext().node(node.id())) && getSpiContext().pingNode(node.id())) {
-                            if (log.isDebugEnabled())
-                                log.debug("Recovery reconnect failed, will retry " +
-                                    "[rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+                GridCommunicationClient client = reserveClient(node, sesInfo.connIdx);
 
-                            addProcessDisconnectRequest(sesInfo);
-                        }
-                        else {
-                            if (log.isDebugEnabled())
-                                log.debug("Recovery reconnect failed, " +
-                                    "node left [rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+                client.release();
+            }
+            catch (IgniteCheckedException | IgniteException e) {
+                try {
+                    if (recoveryDesc.nodeAlive(getSpiContext().node(node.id())) && getSpiContext().pingNode(node.id())) {
+                        if (log.isDebugEnabled())
+                            log.debug("Recovery reconnect failed, will retry " +
+                                "[rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
 
-                            onException("Recovery reconnect failed, node left [rmtNode=" + recoveryDesc.node().id() + "]",
-                                e);
-                        }
+                        addProcessDisconnectRequest(sesInfo);
                     }
-                    catch (IgniteClientDisconnectedException e0) {
+                    else {
                         if (log.isDebugEnabled())
-                            log.debug("Failed to ping node, client disconnected.");
+                            log.debug("Recovery reconnect failed, " +
+                                "node left [rmtNode=" + recoveryDesc.node().id() + ", err=" + e + ']');
+
+                        onE

<TRUNCATED>

[10/25] ignite git commit: IGNITE-4027 Extract PlatformTargetProxy interface

Posted by sb...@apache.org.
IGNITE-4027 Extract PlatformTargetProxy interface

This closes #1188


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

Branch: refs/heads/ignite-4371
Commit: 59e6fec0b92c353ee5e128b9343a59f4b99bd468
Parents: 597f3a5
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Dec 8 14:53:16 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Dec 8 14:53:16 2016 +0300

----------------------------------------------------------------------
 .../platform/PlatformAbstractTarget.java        | 268 +++----------------
 .../platform/PlatformAsyncTarget.java           |  44 +++
 .../platform/PlatformNoopProcessor.java         |  41 +--
 .../processors/platform/PlatformProcessor.java  |  42 +--
 .../platform/PlatformProcessorImpl.java         |  87 +++---
 .../processors/platform/PlatformTarget.java     | 103 ++++---
 .../platform/PlatformTargetProxy.java           | 126 +++++++++
 .../platform/PlatformTargetProxyImpl.java       | 222 +++++++++++++++
 .../binary/PlatformBinaryProcessor.java         |   6 +-
 .../platform/cache/PlatformCache.java           |  15 +-
 .../platform/cache/PlatformCacheIterator.java   |   2 +-
 .../cache/affinity/PlatformAffinity.java        |   4 +-
 .../affinity/PlatformAffinityFunction.java      |   7 +-
 .../PlatformAffinityFunctionTarget.java         |   4 +-
 .../query/PlatformAbstractQueryCursor.java      |   4 +-
 .../query/PlatformContinuousQueryProxy.java     |   3 +-
 .../callback/PlatformCallbackGateway.java       |   6 +-
 .../callback/PlatformCallbackUtils.java         |   6 +-
 .../platform/cluster/PlatformClusterGroup.java  |  18 +-
 .../platform/compute/PlatformCompute.java       |  15 +-
 .../datastreamer/PlatformDataStreamer.java      |   4 +-
 .../PlatformStreamReceiverImpl.java             |   8 +-
 .../datastructures/PlatformAtomicLong.java      |   4 +-
 .../datastructures/PlatformAtomicReference.java |   8 +-
 .../datastructures/PlatformAtomicSequence.java  |   2 +-
 .../platform/events/PlatformEvents.java         |  15 +-
 .../platform/messaging/PlatformMessaging.java   |   9 +-
 .../platform/services/PlatformServices.java     |  27 +-
 .../transactions/PlatformTransactions.java      |   8 +-
 .../platform/utils/PlatformFutureUtils.java     |  14 +-
 .../utils/PlatformListenableTarget.java         |  62 +++++
 .../cpp/jni/include/ignite/jni/exports.h        |   3 -
 .../platforms/cpp/jni/include/ignite/jni/java.h |   7 -
 modules/platforms/cpp/jni/project/vs/module.def |   2 -
 modules/platforms/cpp/jni/src/exports.cpp       |   8 -
 modules/platforms/cpp/jni/src/java.cpp          |  76 ++----
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  13 +-
 .../Impl/Common/Listenable.cs                   |  49 ++++
 .../Impl/Compute/ComputeImpl.cs                 |   4 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |   2 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |   8 -
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   5 -
 43 files changed, 817 insertions(+), 545 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
index 2df86ac..506470b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAbstractTarget.java
@@ -24,16 +24,16 @@ import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
-import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
 import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformListenable;
+import org.apache.ignite.internal.processors.platform.utils.PlatformListenableTarget;
 import org.apache.ignite.lang.IgniteFuture;
 import org.jetbrains.annotations.Nullable;
 
 /**
  * Abstract interop target.
  */
-public abstract class PlatformAbstractTarget implements PlatformTarget {
+public abstract class PlatformAbstractTarget implements PlatformTarget, PlatformAsyncTarget {
     /** Constant: TRUE.*/
     protected static final int TRUE = 1;
 
@@ -60,144 +60,6 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
         log = platformCtx.kernalContext().log(PlatformAbstractTarget.class);
     }
 
-    /** {@inheritDoc} */
-    @Override public long inLongOutLong(int type, long val) throws Exception {
-        try {
-            return processInLongOutLong(type, val);
-        }
-        catch (Exception e) {
-            throw convertException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public long inStreamOutLong(int type, long memPtr) throws Exception {
-        try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
-            BinaryRawReaderEx reader = platformCtx.reader(mem);
-
-            return processInStreamOutLong(type, reader, mem);
-        }
-        catch (Exception e) {
-            throw convertException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object inStreamOutObject(int type, long memPtr) throws Exception {
-        try (PlatformMemory mem = memPtr != 0 ? platformCtx.memory().get(memPtr) : null) {
-            BinaryRawReaderEx reader = mem != null ? platformCtx.reader(mem) : null;
-
-            return processInStreamOutObject(type, reader);
-        }
-        catch (Exception e) {
-            throw convertException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void outStream(int type, long memPtr) throws Exception {
-        try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
-            PlatformOutputStream out = mem.output();
-
-            BinaryRawWriterEx writer = platformCtx.writer(out);
-
-            processOutStream(type, writer);
-
-            out.synchronize();
-        }
-        catch (Exception e) {
-            throw convertException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object outObject(int type) throws Exception {
-        try {
-            return processOutObject(type);
-        }
-        catch (Exception e) {
-            throw convertException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception {
-        try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
-            BinaryRawReaderEx reader = platformCtx.reader(inMem);
-
-            try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) {
-                PlatformOutputStream out = outMem.output();
-
-                BinaryRawWriterEx writer = platformCtx.writer(out);
-
-                processInStreamOutStream(type, reader, writer);
-
-                out.synchronize();
-            }
-        }
-        catch (Exception e) {
-            throw convertException(e);
-        }
-    }
-
-    /** {@inheritDoc} */
-    @Override public Object inObjectStreamOutObjectStream(int type, Object arg, long inMemPtr, long outMemPtr)
-        throws Exception {
-        PlatformMemory inMem = null;
-        PlatformMemory outMem = null;
-
-        try {
-            BinaryRawReaderEx reader = null;
-
-            if (inMemPtr != 0) {
-                inMem = platformCtx.memory().get(inMemPtr);
-
-                reader = platformCtx.reader(inMem);
-            }
-
-            PlatformOutputStream out = null;
-            BinaryRawWriterEx writer = null;
-
-            if (outMemPtr != 0) {
-                outMem = platformCtx.memory().get(outMemPtr);
-
-                out = outMem.output();
-
-                writer = platformCtx.writer(out);
-            }
-
-            Object res = processInObjectStreamOutObjectStream(type, arg, reader, writer);
-
-            if (out != null)
-                out.synchronize();
-
-            return res;
-        }
-        catch (Exception e) {
-            throw convertException(e);
-        }
-        finally {
-            try {
-                if (inMem != null)
-                    inMem.close();
-            }
-            finally {
-                if (outMem != null)
-                    outMem.close();
-            }
-        }
-    }
-
-    /**
-     * Convert caught exception.
-     *
-     * @param e Exception to convert.
-     * @return Converted exception.
-     */
-    public Exception convertException(Exception e) {
-        return e;
-    }
-
     /**
      * @return Context.
      */
@@ -206,128 +68,60 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
     }
 
     /** {@inheritDoc} */
-    @Override public void listenFuture(final long futId, int typ) throws Exception {
-        PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, null, this);
+    @Override public Exception convertException(Exception e) {
+        return e;
     }
 
     /** {@inheritDoc} */
-    @Override public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception {
-        PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, futureWriter(opId), this);
-    }
-
-    /**
-     * When overridden in a derived class, gets future for the current operation.
-     *
-     * @return current future.
-     * @throws IgniteCheckedException If failed.
-     */
-    protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+    @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
         throw new IgniteCheckedException("Future listening is not supported in " + getClass());
     }
 
-    /**
-     * When overridden in a derived class, gets a custom future writer.
-     *
-     * @param opId Operation id.
-     * @return A custom writer for given op id.
-     */
-    @Nullable protected PlatformFutureUtils.Writer futureWriter(int opId){
+    /** {@inheritDoc} */
+    @Override @Nullable public PlatformFutureUtils.Writer futureWriter(int opId){
         return null;
     }
 
-    /**
-     * Process IN operation.
-     *
-     * @param type Type.
-     * @param val Value.
-     * @return Result.
-     * @throws IgniteCheckedException In case of exception.
-     */
-    protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         return throwUnsupported(type);
     }
 
-    /**
-     * Process IN operation.
-     *
-     * @param type Type.
-     * @param reader Binary reader.
-     * @return Result.
-     * @throws IgniteCheckedException In case of exception.
-     */
-    protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
         return throwUnsupported(type);
     }
 
-    /**
-     * Process IN operation.
-     *
-     * @param type Type.
-     * @param reader Binary reader.
-     * @return Result.
-     * @throws IgniteCheckedException In case of exception.
-     */
-    protected long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException {
         return processInStreamOutLong(type, reader);
     }
 
-    /**
-     * Process IN-OUT operation.
-     *
-     * @param type Type.
-     * @param reader Binary reader.
-     * @param writer Binary writer.
-     * @throws IgniteCheckedException In case of exception.
-     */
-    protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+    /** {@inheritDoc} */
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
         throws IgniteCheckedException {
         throwUnsupported(type);
     }
 
-    /**
-     * Process IN operation with managed object as result.
-     *
-     * @param type Type.
-     * @param reader Binary reader.
-     * @return Result.
-     * @throws IgniteCheckedException In case of exception.
-     */
-    protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader)
+        throws IgniteCheckedException {
         return throwUnsupported(type);
     }
 
-    /**
-     * Process IN-OUT operation.
-     *
-     * @param type Type.
-     * @param arg Argument.
-     * @param reader Binary reader.
-     * @param writer Binary writer.
-     * @throws IgniteCheckedException In case of exception.
-     */
-    protected Object processInObjectStreamOutObjectStream(int type, @Nullable Object arg, BinaryRawReaderEx reader,
-        BinaryRawWriterEx writer) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public PlatformTarget processInObjectStreamOutObjectStream(int type, @Nullable PlatformTarget arg,
+        BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException {
         return throwUnsupported(type);
     }
 
-    /**
-     * Process OUT operation.
-     *
-     * @param type Type.
-     * @param writer Binary writer.
-     * @throws IgniteCheckedException In case of exception.
-     */
-    protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         throwUnsupported(type);
     }
 
-    /**
-     * Process OUT operation.
-     *
-     * @param type Type.
-     * @throws IgniteCheckedException In case of exception.
-     */
-    protected Object processOutObject(int type) throws IgniteCheckedException {
+    /** {@inheritDoc} */
+    @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
         return throwUnsupported(type);
     }
 
@@ -338,7 +132,7 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
      * @return Dummy value which is never returned.
      * @throws IgniteCheckedException Exception to be thrown.
      */
-    protected <T> T throwUnsupported(int type) throws IgniteCheckedException {
+    private <T> T throwUnsupported(int type) throws IgniteCheckedException {
         throw new IgniteCheckedException("Unsupported operation type: " + type);
     }
 
@@ -411,4 +205,14 @@ public abstract class PlatformAbstractTarget implements PlatformTarget {
 
         return TRUE;
     }
+
+    /**
+     * Wraps a listenable to be returned to platform.
+     *
+     * @param listenable Listenable.
+     * @return Target.
+     */
+    protected PlatformTarget wrapListenable(PlatformListenable listenable) {
+        return new PlatformListenableTarget(listenable, platformCtx);
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAsyncTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAsyncTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAsyncTarget.java
new file mode 100644
index 0000000..a4d35c9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformAsyncTarget.java
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Async target.
+ */
+public interface PlatformAsyncTarget {
+    /**
+     * Gets future for the current operation.
+     *
+     * @return current future.
+     * @throws IgniteCheckedException If failed.
+     */
+    IgniteInternalFuture currentFuture() throws IgniteCheckedException;
+
+    /**
+     * Gets a custom future writer.
+     *
+     * @param opId Operation id.
+     * @return A custom writer for given op id.
+     */
+    @Nullable PlatformFutureUtils.Writer futureWriter(int opId);
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
index fd357ec..2911418 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformNoopProcessor.java
@@ -61,27 +61,27 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget cache(@Nullable String name) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy cache(@Nullable String name) throws IgniteCheckedException {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget createCache(@Nullable String name) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy createCache(@Nullable String name) throws IgniteCheckedException {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget createCacheFromConfig(long memPtr) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy createCacheFromConfig(long memPtr) throws IgniteCheckedException {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
         return null;
     }
 
@@ -91,47 +91,48 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget affinity(@Nullable String name) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy affinity(@Nullable String name) throws IgniteCheckedException {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget dataStreamer(@Nullable String cacheName, boolean keepBinary) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy dataStreamer(@Nullable String cacheName, boolean keepBinary)
+        throws IgniteCheckedException {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget transactions() {
+    @Override public PlatformTargetProxy transactions() {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget projection() throws IgniteCheckedException {
+    @Override public PlatformTargetProxy projection() throws IgniteCheckedException {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget compute(PlatformTarget grp) {
+    @Override public PlatformTargetProxy compute(PlatformTargetProxy grp) {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget message(PlatformTarget grp) {
+    @Override public PlatformTargetProxy message(PlatformTargetProxy grp) {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget events(PlatformTarget grp) {
+    @Override public PlatformTargetProxy events(PlatformTargetProxy grp) {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget services(PlatformTarget grp) {
+    @Override public PlatformTargetProxy services(PlatformTargetProxy grp) {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget extensions() {
+    @Override public PlatformTargetProxy extensions() {
         return null;
     }
 
@@ -142,7 +143,7 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget atomicLong(String name, long initVal, boolean create) throws IgniteException {
+    @Override public PlatformTargetProxy atomicLong(String name, long initVal, boolean create) throws IgniteException {
         return null;
     }
 
@@ -157,22 +158,22 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget atomicSequence(String name, long initVal, boolean create) throws IgniteException {
+    @Override public PlatformTargetProxy atomicSequence(String name, long initVal, boolean create) throws IgniteException {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget atomicReference(String name, long memPtr, boolean create) throws IgniteException {
+    @Override public PlatformTargetProxy atomicReference(String name, long memPtr, boolean create) throws IgniteException {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget createNearCache(@Nullable String cacheName, long memPtr) {
+    @Override public PlatformTargetProxy createNearCache(@Nullable String cacheName, long memPtr) {
         return null;
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
+    @Override public PlatformTargetProxy getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
         return null;
     }
 
@@ -187,7 +188,7 @@ public class PlatformNoopProcessor extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget binaryProcessor() {
+    @Override public PlatformTargetProxy binaryProcessor() {
         return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
index f01175e..e0d94d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessor.java
@@ -26,7 +26,7 @@ import org.jetbrains.annotations.Nullable;
 /**
  * Platform processor.
  */
-@SuppressWarnings("UnusedDeclaration")
+@SuppressWarnings({"UnusedDeclaration", "UnnecessaryInterfaceModifier"})
 public interface PlatformProcessor extends GridProcessor {
     /**
      * Gets owning Ignite instance.
@@ -68,7 +68,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Cache.
      * @throws IgniteCheckedException If failed.
      */
-    public PlatformTarget cache(@Nullable String name) throws IgniteCheckedException;
+    public PlatformTargetProxy cache(@Nullable String name) throws IgniteCheckedException;
 
     /**
      * Create cache.
@@ -77,7 +77,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Cache.
      * @throws IgniteCheckedException If failed.
      */
-    public PlatformTarget createCache(@Nullable String name) throws IgniteCheckedException;
+    public PlatformTargetProxy createCache(@Nullable String name) throws IgniteCheckedException;
 
     /**
      * Get or create cache.
@@ -86,7 +86,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Cache.
      * @throws IgniteCheckedException If failed.
      */
-    public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException;
+    public PlatformTargetProxy getOrCreateCache(@Nullable String name) throws IgniteCheckedException;
 
     /**
      * Create cache.
@@ -95,7 +95,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Cache.
      * @throws IgniteCheckedException If failed.
      */
-    public PlatformTarget createCacheFromConfig(long memPtr) throws IgniteCheckedException;
+    public PlatformTargetProxy createCacheFromConfig(long memPtr) throws IgniteCheckedException;
 
     /**
      * Get or create cache.
@@ -104,7 +104,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Cache.
      * @throws IgniteCheckedException If failed.
      */
-    public PlatformTarget getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException;
+    public PlatformTargetProxy getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException;
 
     /**
      * Destroy dynamically created cache.
@@ -121,7 +121,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Affinity.
      * @throws IgniteCheckedException If failed.
      */
-    public PlatformTarget affinity(@Nullable String name) throws IgniteCheckedException;
+    public PlatformTargetProxy affinity(@Nullable String name) throws IgniteCheckedException;
 
     /**
      * Get data streamer.
@@ -131,14 +131,14 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Data streamer.
      * @throws IgniteCheckedException If failed.
      */
-    public PlatformTarget dataStreamer(@Nullable String cacheName, boolean keepBinary) throws IgniteCheckedException;
+    public PlatformTargetProxy dataStreamer(@Nullable String cacheName, boolean keepBinary) throws IgniteCheckedException;
 
     /**
      * Get transactions.
      *
      * @return Transactions.
      */
-    public PlatformTarget transactions();
+    public PlatformTargetProxy transactions();
 
     /**
      * Get projection.
@@ -146,7 +146,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @return Projection.
      * @throws IgniteCheckedException If failed.
      */
-    public PlatformTarget projection() throws IgniteCheckedException;
+    public PlatformTargetProxy projection() throws IgniteCheckedException;
 
     /**
      * Create interop compute.
@@ -154,7 +154,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @param grp Cluster group.
      * @return Compute instance.
      */
-    public PlatformTarget compute(PlatformTarget grp);
+    public PlatformTargetProxy compute(PlatformTargetProxy grp);
 
     /**
      * Create interop messaging.
@@ -162,7 +162,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @param grp Cluster group.
      * @return Messaging instance.
      */
-    public PlatformTarget message(PlatformTarget grp);
+    public PlatformTargetProxy message(PlatformTargetProxy grp);
 
     /**
      * Create interop events.
@@ -170,7 +170,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @param grp Cluster group.
      * @return Events instance.
      */
-    public PlatformTarget events(PlatformTarget grp);
+    public PlatformTargetProxy events(PlatformTargetProxy grp);
 
     /**
      * Create interop services.
@@ -178,14 +178,14 @@ public interface PlatformProcessor extends GridProcessor {
      * @param grp Cluster group.
      * @return Services instance.
      */
-    public PlatformTarget services(PlatformTarget grp);
+    public PlatformTargetProxy services(PlatformTargetProxy grp);
 
     /**
      * Get platform extensions. Override this method to provide any additional targets and operations you need.
      *
      * @return Platform extensions.
      */
-    public PlatformTarget extensions();
+    public PlatformTargetProxy extensions();
 
     /**
      * Register cache store.
@@ -203,7 +203,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @param create Create flag.
      * @return Platform atomic long.
      */
-    public PlatformTarget atomicLong(String name, long initVal, boolean create);
+    public PlatformTargetProxy atomicLong(String name, long initVal, boolean create);
 
     /**
      * Get or create AtomicSequence.
@@ -212,7 +212,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @param create Create flag.
      * @return Platform atomic long.
      */
-    public PlatformTarget atomicSequence(String name, long initVal, boolean create);
+    public PlatformTargetProxy atomicSequence(String name, long initVal, boolean create);
 
     /**
      * Get or create AtomicReference.
@@ -221,7 +221,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @param create Create flag.
      * @return Platform atomic long.
      */
-    public PlatformTarget atomicReference(String name, long memPtr, boolean create);
+    public PlatformTargetProxy atomicReference(String name, long memPtr, boolean create);
 
     /**
      * Gets the configuration of the current Ignite instance.
@@ -244,7 +244,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @param memPtr Pointer to a stream with near cache config. 0 for default config.
      * @return Cache.
      */
-    public PlatformTarget createNearCache(@Nullable String cacheName, long memPtr);
+    public PlatformTargetProxy createNearCache(@Nullable String cacheName, long memPtr);
 
     /**
      * Gets existing near cache with the given name or creates a new one.
@@ -253,7 +253,7 @@ public interface PlatformProcessor extends GridProcessor {
      * @param memPtr Pointer to a stream with near cache config. 0 for default config.
      * @return Cache.
      */
-    public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr);
+    public PlatformTargetProxy getOrCreateNearCache(@Nullable String cacheName, long memPtr);
 
     /**
      * Gets a value indicating whether Ignite logger has specified level enabled.
@@ -277,5 +277,5 @@ public interface PlatformProcessor extends GridProcessor {
      *
      * @return Binary processor.
      */
-    public PlatformTarget binaryProcessor();
+    public PlatformTargetProxy binaryProcessor();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
index f775987..8c81ebb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformProcessorImpl.java
@@ -220,7 +220,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget cache(@Nullable String name) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy cache(@Nullable String name) throws IgniteCheckedException {
         IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().cache(name);
 
         if (cache == null)
@@ -230,7 +230,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget createCache(@Nullable String name) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy createCache(@Nullable String name) throws IgniteCheckedException {
         IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createCache(name);
 
         assert cache != null;
@@ -239,7 +239,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy getOrCreateCache(@Nullable String name) throws IgniteCheckedException {
         IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateCache(name);
 
         assert cache != null;
@@ -248,7 +248,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget createCacheFromConfig(long memPtr) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy createCacheFromConfig(long memPtr) throws IgniteCheckedException {
         BinaryRawReaderEx reader = platformCtx.reader(platformCtx.memory().get(memPtr));
         CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
 
@@ -260,7 +260,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
+    @Override public PlatformTargetProxy getOrCreateCacheFromConfig(long memPtr) throws IgniteCheckedException {
         BinaryRawReaderEx reader = platformCtx.reader(platformCtx.memory().get(memPtr));
         CacheConfiguration cfg = PlatformConfigurationUtils.readCacheConfiguration(reader);
 
@@ -278,60 +278,60 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget affinity(@Nullable String name) throws IgniteCheckedException {
-        return new PlatformAffinity(platformCtx, ctx, name);
+    @Override public PlatformTargetProxy affinity(@Nullable String name) throws IgniteCheckedException {
+        return proxy(new PlatformAffinity(platformCtx, ctx, name));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget dataStreamer(@Nullable String cacheName, boolean keepBinary)
+    @Override public PlatformTargetProxy dataStreamer(@Nullable String cacheName, boolean keepBinary)
         throws IgniteCheckedException {
         IgniteDataStreamer ldr = ctx.dataStream().dataStreamer(cacheName);
 
         ldr.keepBinary(true);
 
-        return new PlatformDataStreamer(platformCtx, cacheName, (DataStreamerImpl)ldr, keepBinary);
+        return proxy(new PlatformDataStreamer(platformCtx, cacheName, (DataStreamerImpl)ldr, keepBinary));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget transactions() {
-        return new PlatformTransactions(platformCtx);
+    @Override public PlatformTargetProxy transactions() {
+        return proxy(new PlatformTransactions(platformCtx));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget projection() throws IgniteCheckedException {
-        return new PlatformClusterGroup(platformCtx, ctx.grid().cluster());
+    @Override public PlatformTargetProxy projection() throws IgniteCheckedException {
+        return proxy(new PlatformClusterGroup(platformCtx, ctx.grid().cluster()));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget compute(PlatformTarget grp) {
-        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
+    @Override public PlatformTargetProxy compute(PlatformTargetProxy grp) {
+        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
 
-        return new PlatformCompute(platformCtx, grp0.projection(), PlatformUtils.ATTR_PLATFORM);
+        return proxy(new PlatformCompute(platformCtx, grp0.projection(), PlatformUtils.ATTR_PLATFORM));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget message(PlatformTarget grp) {
-        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
+    @Override public PlatformTargetProxy message(PlatformTargetProxy grp) {
+        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
 
-        return new PlatformMessaging(platformCtx, grp0.projection().ignite().message(grp0.projection()));
+        return proxy(new PlatformMessaging(platformCtx, grp0.projection().ignite().message(grp0.projection())));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget events(PlatformTarget grp) {
-        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
+    @Override public PlatformTargetProxy events(PlatformTargetProxy grp) {
+        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
 
-        return new PlatformEvents(platformCtx, grp0.projection().ignite().events(grp0.projection()));
+        return proxy(new PlatformEvents(platformCtx, grp0.projection().ignite().events(grp0.projection())));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget services(PlatformTarget grp) {
-        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp;
+    @Override public PlatformTargetProxy services(PlatformTargetProxy grp) {
+        PlatformClusterGroup grp0 = (PlatformClusterGroup)grp.unwrap();
 
-        return new PlatformServices(platformCtx, grp0.projection().ignite().services(grp0.projection()), false);
+        return proxy(new PlatformServices(platformCtx, grp0.projection().ignite().services(grp0.projection()), false));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget extensions() {
+    @Override public PlatformTargetProxy extensions() {
         return null;
     }
 
@@ -356,28 +356,32 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget atomicLong(String name, long initVal, boolean create) throws IgniteException {
+    @Override public PlatformTargetProxy atomicLong(String name, long initVal, boolean create) throws IgniteException {
         GridCacheAtomicLongImpl atomicLong = (GridCacheAtomicLongImpl)ignite().atomicLong(name, initVal, create);
 
         if (atomicLong == null)
             return null;
 
-        return new PlatformAtomicLong(platformCtx, atomicLong);
+        return proxy(new PlatformAtomicLong(platformCtx, atomicLong));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget atomicSequence(String name, long initVal, boolean create) throws IgniteException {
+    @Override public PlatformTargetProxy atomicSequence(String name, long initVal, boolean create)
+        throws IgniteException {
         IgniteAtomicSequence atomicSeq = ignite().atomicSequence(name, initVal, create);
 
         if (atomicSeq == null)
             return null;
 
-        return new PlatformAtomicSequence(platformCtx, atomicSeq);
+        return proxy(new PlatformAtomicSequence(platformCtx, atomicSeq));
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget atomicReference(String name, long memPtr, boolean create) throws IgniteException {
-        return PlatformAtomicReference.createInstance(platformCtx, name, memPtr, create);
+    @Override public PlatformTargetProxy atomicReference(String name, long memPtr, boolean create)
+        throws IgniteException {
+        PlatformAtomicReference ref = PlatformAtomicReference.createInstance(platformCtx, name, memPtr, create);
+
+        return ref != null ? proxy(ref) : null;
     }
 
     /** {@inheritDoc} */
@@ -427,7 +431,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget createNearCache(@Nullable String cacheName, long memPtr) {
+    @Override public PlatformTargetProxy createNearCache(@Nullable String cacheName, long memPtr) {
         NearCacheConfiguration cfg = getNearCacheConfiguration(memPtr);
 
         IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().createNearCache(cacheName, cfg);
@@ -436,7 +440,7 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
+    @Override public PlatformTargetProxy getOrCreateNearCache(@Nullable String cacheName, long memPtr) {
         NearCacheConfiguration cfg = getNearCacheConfiguration(memPtr);
 
         IgniteCacheProxy cache = (IgniteCacheProxy)ctx.grid().getOrCreateNearCache(cacheName, cfg);
@@ -447,8 +451,8 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     /**
      * Creates new platform cache.
      */
-    private PlatformTarget createPlatformCache(IgniteCacheProxy cache) {
-        return new PlatformCache(platformCtx, cache, false, cacheExts);
+    private PlatformTargetProxy createPlatformCache(IgniteCacheProxy cache) {
+        return proxy(new PlatformCache(platformCtx, cache, false, cacheExts));
     }
 
     /** {@inheritDoc} */
@@ -504,8 +508,8 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /** {@inheritDoc} */
-    @Override public PlatformTarget binaryProcessor() {
-        return new PlatformBinaryProcessor(platformCtx);
+    @Override public PlatformTargetProxy binaryProcessor() {
+        return proxy(new PlatformBinaryProcessor(platformCtx));
     }
 
     /**
@@ -580,6 +584,13 @@ public class PlatformProcessorImpl extends GridProcessorAdapter implements Platf
     }
 
     /**
+     * Wraps target in a proxy.
+     */
+    private PlatformTargetProxy proxy(PlatformTarget target) {
+        return new PlatformTargetProxyImpl(target, platformCtx);
+    }
+
+    /**
      * Store and manager pair.
      */
     private static class StoreInfo {

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
index 805fd5e..5d234dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTarget.java
@@ -18,7 +18,9 @@
 package org.apache.ignite.internal.processors.platform;
 
 import org.apache.ignite.IgniteCheckedException;
-import org.apache.ignite.internal.processors.platform.utils.*;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -27,94 +29,89 @@ import org.jetbrains.annotations.Nullable;
 @SuppressWarnings("UnusedDeclaration")
 public interface PlatformTarget {
     /**
-     * Operation accepting long value and returning long value.
+     * Process IN operation.
      *
-     * @param type Operation type.
+     * @param type Type.
      * @param val Value.
      * @return Result.
-     * @throws Exception If case of failure.
+     * @throws IgniteCheckedException In case of exception.
      */
-    public long inLongOutLong(int type, long val) throws Exception;
+    long processInLongOutLong(int type, long val) throws IgniteCheckedException;
 
     /**
-     * Operation accepting memory stream and returning long value.
+     * Process IN operation.
      *
-     * @param type Operation type.
-     * @param memPtr Memory pointer.
+     * @param type Type.
+     * @param reader Binary reader.
      * @return Result.
-     * @throws Exception If case of failure.
+     * @throws IgniteCheckedException In case of exception.
      */
-    public long inStreamOutLong(int type, long memPtr) throws Exception;
+    long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException;
 
     /**
-     * Operation accepting memory stream and returning object.
+     * Process IN operation.
      *
-     * @param type Operation type.
-     * @param memPtr Memory pointer.
+     * @param type Type.
+     * @param reader Binary reader.
      * @return Result.
-     * @throws Exception If case of failure.
+     * @throws IgniteCheckedException In case of exception.
      */
-    public Object inStreamOutObject(int type, long memPtr) throws Exception;
+    long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem) throws IgniteCheckedException;
 
     /**
-     * Operation accepting one memory stream and returning result to another memory stream.
+     * Process IN-OUT operation.
      *
-     * @param type Operation type.
-     * @param inMemPtr Input memory pointer.
-     * @param outMemPtr Output memory pointer.
-     * @throws Exception In case of failure.
+     * @param type Type.
+     * @param reader Binary reader.
+     * @param writer Binary writer.
+     * @throws IgniteCheckedException In case of exception.
      */
-    public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception;
+    void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+        throws IgniteCheckedException;
 
     /**
-     * Operation accepting an object and a memory stream and returning result to another memory stream and an object.
+     * Process IN-OUT operation.
      *
-     * @param type Operation type.
-     * @param arg Argument (optional).
-     * @param inMemPtr Input memory pointer.
-     * @param outMemPtr Output memory pointer.
-     * @return Result.
-     * @throws Exception In case of failure.
+     * @param type Type.
+     * @param reader Binary reader.
+     * @throws IgniteCheckedException In case of exception.
      */
-    public Object inObjectStreamOutObjectStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr)
-        throws Exception;
+    PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException;
 
     /**
-     * Operation returning result to memory stream.
+     * Process IN-OUT operation.
      *
-     * @param type Operation type.
-     * @param memPtr Memory pointer.
-     * @throws Exception In case of failure.
+     * @param type Type.
+     * @param arg Argument.
+     * @param reader Binary reader.
+     * @param writer Binary writer.
+     * @throws IgniteCheckedException In case of exception.
      */
-    public void outStream(int type, long memPtr) throws Exception;
+    PlatformTarget processInObjectStreamOutObjectStream(int type, @Nullable PlatformTarget arg, BinaryRawReaderEx reader,
+        BinaryRawWriterEx writer) throws IgniteCheckedException;
 
     /**
-     * Operation returning object result.
+     * Process OUT operation.
      *
-     * @param type Operation type.
-     * @return Result.
-     * @throws Exception If failed.
+     * @param type Type.
+     * @param writer Binary writer.
+     * @throws IgniteCheckedException In case of exception.
      */
-    public Object outObject(int type) throws Exception;
+    void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException;
 
     /**
-     * Start listening for the future.
+     * Process OUT operation.
      *
-     * @param futId Future ID.
-     * @param typ Result type.
-     * @throws IgniteCheckedException In case of failure.
+     * @param type Type.
+     * @throws IgniteCheckedException In case of exception.
      */
-    @SuppressWarnings("UnusedDeclaration")
-    public void listenFuture(final long futId, int typ) throws Exception;
+    PlatformTarget processOutObject(int type) throws IgniteCheckedException;
 
     /**
-     * Start listening for the future for specific operation type.
+     * Convert caught exception.
      *
-     * @param futId Future ID.
-     * @param typ Result type.
-     * @param opId Operation ID required to pick correct result writer.
-     * @throws IgniteCheckedException In case of failure.
+     * @param e Exception to convert.
+     * @return Converted exception.
      */
-    @SuppressWarnings("UnusedDeclaration")
-    public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception;
+    Exception convertException(Exception e);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxy.java
new file mode 100644
index 0000000..a4f2a56
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxy.java
@@ -0,0 +1,126 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Platform target that is invoked via JNI and propagates calls to underlying {@link PlatformTarget}.
+ */
+@SuppressWarnings("UnusedDeclaration")
+public interface PlatformTargetProxy {
+    /**
+     * Operation accepting long value and returning long value.
+     *
+     * @param type Operation type.
+     * @param val Value.
+     * @return Result.
+     * @throws Exception If case of failure.
+     */
+    long inLongOutLong(int type, long val) throws Exception;
+
+    /**
+     * Operation accepting memory stream and returning long value.
+     *
+     * @param type Operation type.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     * @throws Exception If case of failure.
+     */
+    long inStreamOutLong(int type, long memPtr) throws Exception;
+
+    /**
+     * Operation accepting memory stream and returning object.
+     *
+     * @param type Operation type.
+     * @param memPtr Memory pointer.
+     * @return Result.
+     * @throws Exception If case of failure.
+     */
+    Object inStreamOutObject(int type, long memPtr) throws Exception;
+
+    /**
+     * Operation accepting one memory stream and returning result to another memory stream.
+     *
+     * @param type Operation type.
+     * @param inMemPtr Input memory pointer.
+     * @param outMemPtr Output memory pointer.
+     * @throws Exception In case of failure.
+     */
+    void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception;
+
+    /**
+     * Operation accepting an object and a memory stream and returning result to another memory stream and an object.
+     *
+     * @param type Operation type.
+     * @param arg Argument (optional).
+     * @param inMemPtr Input memory pointer.
+     * @param outMemPtr Output memory pointer.
+     * @return Result.
+     * @throws Exception In case of failure.
+     */
+    Object inObjectStreamOutObjectStream(int type, @Nullable Object arg, long inMemPtr, long outMemPtr)
+        throws Exception;
+
+    /**
+     * Operation returning result to memory stream.
+     *
+     * @param type Operation type.
+     * @param memPtr Memory pointer.
+     * @throws Exception In case of failure.
+     */
+    void outStream(int type, long memPtr) throws Exception;
+
+    /**
+     * Operation returning object result.
+     *
+     * @param type Operation type.
+     * @return Result.
+     * @throws Exception If failed.
+     */
+    Object outObject(int type) throws Exception;
+
+    /**
+     * Start listening for the future.
+     *
+     * @param futId Future ID.
+     * @param typ Result type.
+     * @throws IgniteCheckedException In case of failure.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    void listenFuture(final long futId, int typ) throws Exception;
+
+    /**
+     * Start listening for the future for specific operation type.
+     *
+     * @param futId Future ID.
+     * @param typ Result type.
+     * @param opId Operation ID required to pick correct result writer.
+     * @throws IgniteCheckedException In case of failure.
+     */
+    @SuppressWarnings("UnusedDeclaration")
+    void listenFutureForOperation(final long futId, int typ, int opId) throws Exception;
+
+    /**
+     * Returns the underlying target.
+     *
+     * @return Underlying target.
+     */
+    PlatformTarget unwrap();
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxyImpl.java
new file mode 100644
index 0000000..25a4de8
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/PlatformTargetProxyImpl.java
@@ -0,0 +1,222 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.binary.BinaryRawReaderEx;
+import org.apache.ignite.internal.binary.BinaryRawWriterEx;
+import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
+import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
+import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
+
+/**
+ * Platform target that is invoked via JNI and propagates calls to underlying {@link PlatformTarget}.
+ */
+public class PlatformTargetProxyImpl implements PlatformTargetProxy {
+    /** Context. */
+    protected final PlatformContext platformCtx;
+
+    /** Underlying target. */
+    private final PlatformTarget target;
+
+    public PlatformTargetProxyImpl(PlatformTarget target, PlatformContext platformCtx) {
+        assert platformCtx != null;
+        assert target != null;
+
+        this.platformCtx = platformCtx;
+        this.target = target;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long inLongOutLong(int type, long val) throws Exception {
+        try {
+            return target.processInLongOutLong(type, val);
+        }
+        catch (Exception e) {
+            throw target.convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public long inStreamOutLong(int type, long memPtr) throws Exception {
+        try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
+            BinaryRawReaderEx reader = platformCtx.reader(mem);
+
+            return target.processInStreamOutLong(type, reader, mem);
+        }
+        catch (Exception e) {
+            throw target.convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object inStreamOutObject(int type, long memPtr) throws Exception {
+        try (PlatformMemory mem = memPtr != 0 ? platformCtx.memory().get(memPtr) : null) {
+            BinaryRawReaderEx reader = mem != null ? platformCtx.reader(mem) : null;
+
+            return wrapProxy(target.processInStreamOutObject(type, reader));
+        }
+        catch (Exception e) {
+            throw target.convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void outStream(int type, long memPtr) throws Exception {
+        try (PlatformMemory mem = platformCtx.memory().get(memPtr)) {
+            PlatformOutputStream out = mem.output();
+
+            BinaryRawWriterEx writer = platformCtx.writer(out);
+
+            target.processOutStream(type, writer);
+
+            out.synchronize();
+        }
+        catch (Exception e) {
+            throw target.convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object outObject(int type) throws Exception {
+        try {
+            return wrapProxy(target.processOutObject(type));
+        }
+        catch (Exception e) {
+            throw target.convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void inStreamOutStream(int type, long inMemPtr, long outMemPtr) throws Exception {
+        try (PlatformMemory inMem = platformCtx.memory().get(inMemPtr)) {
+            BinaryRawReaderEx reader = platformCtx.reader(inMem);
+
+            try (PlatformMemory outMem = platformCtx.memory().get(outMemPtr)) {
+                PlatformOutputStream out = outMem.output();
+
+                BinaryRawWriterEx writer = platformCtx.writer(out);
+
+                target.processInStreamOutStream(type, reader, writer);
+
+                out.synchronize();
+            }
+        }
+        catch (Exception e) {
+            throw target.convertException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public Object inObjectStreamOutObjectStream(int type, Object arg, long inMemPtr, long outMemPtr)
+        throws Exception {
+        PlatformMemory inMem = null;
+        PlatformMemory outMem = null;
+
+        try {
+            BinaryRawReaderEx reader = null;
+
+            if (inMemPtr != 0) {
+                inMem = platformCtx.memory().get(inMemPtr);
+
+                reader = platformCtx.reader(inMem);
+            }
+
+            PlatformOutputStream out = null;
+            BinaryRawWriterEx writer = null;
+
+            if (outMemPtr != 0) {
+                outMem = platformCtx.memory().get(outMemPtr);
+
+                out = outMem.output();
+
+                writer = platformCtx.writer(out);
+            }
+
+            PlatformTarget res = target.processInObjectStreamOutObjectStream(type, unwrapProxy(arg), reader, writer);
+
+            if (out != null)
+                out.synchronize();
+
+            return wrapProxy(res);
+        }
+        catch (Exception e) {
+            throw target.convertException(e);
+        }
+        finally {
+            try {
+                if (inMem != null)
+                    inMem.close();
+            }
+            finally {
+                if (outMem != null)
+                    outMem.close();
+            }
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void listenFuture(final long futId, int typ) throws Exception {
+        PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, null, target);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void listenFutureForOperation(final long futId, int typ, int opId) throws Exception {
+        PlatformFutureUtils.listen(platformCtx, currentFuture(), futId, typ, futureWriter(opId), target);
+    }
+
+    /** {@inheritDoc} */
+    @Override public PlatformTarget unwrap() {
+        return target;
+    }
+
+    /**
+     * @return Future writer.
+     */
+    private PlatformFutureUtils.Writer futureWriter(int opId) {
+        return ((PlatformAsyncTarget)target).futureWriter(opId);
+    }
+
+    /**
+     * @return Current future.
+     */
+    private IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+        return ((PlatformAsyncTarget)target).currentFuture();
+    }
+
+    /**
+     * Wraps an object in a proxy when possible.
+     *
+     * @param obj Object to wrap.
+     * @return Wrapped object.
+     */
+    private Object wrapProxy(PlatformTarget obj) {
+        return obj == null ? null : new PlatformTargetProxyImpl(obj, platformCtx);
+    }
+
+    /**
+     * Unwraps an object from a proxy when possible.
+     *
+     * @param obj Object to unwrap.
+     * @return Unwrapped object.
+     */
+    private PlatformTarget unwrapProxy(Object obj) {
+        return obj == null ? null : ((PlatformTargetProxyImpl)obj).target;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
index 1bb577e..3c00abc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/binary/PlatformBinaryProcessor.java
@@ -49,7 +49,7 @@ public class PlatformBinaryProcessor extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
         if (type == OP_PUT_META) {
             platformCtx.processMetadata(reader);
 
@@ -60,7 +60,7 @@ public class PlatformBinaryProcessor extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         if (type == OP_GET_ALL_META)
             platformCtx.writeAllMetadata(writer);
         else
@@ -68,7 +68,7 @@ public class PlatformBinaryProcessor extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader,
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader,
         BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_META: {

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
index aec3703..aee317e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCache.java
@@ -43,6 +43,7 @@ import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformNativeException;
 import org.apache.ignite.internal.processors.platform.cache.expiry.PlatformExpiryPolicy;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQuery;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformContinuousQueryProxy;
 import org.apache.ignite.internal.processors.platform.cache.query.PlatformFieldsQueryCursor;
@@ -400,7 +401,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem)
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader, PlatformMemory mem)
         throws IgniteCheckedException {
         try {
             switch (type) {
@@ -824,7 +825,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader)
+    @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_QRY_SQL:
@@ -903,7 +904,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_NAME:
                 writer.writeObject(cache.getName());
@@ -940,7 +941,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+    @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
         switch (type) {
             case OP_WITH_ASYNC: {
                 if (cache.isAsync())
@@ -983,7 +984,7 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_SIZE: {
                 CachePeekMode[] modes = PlatformUtils.decodeCachePeekModes((int)val);
@@ -1121,12 +1122,12 @@ public class PlatformCache extends PlatformAbstractTarget {
     }
 
     /** <inheritDoc /> */
-    @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+    @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
         return ((IgniteFutureImpl) cacheAsync.future()).internalFuture();
     }
 
     /** <inheritDoc /> */
-    @Nullable @Override protected PlatformFutureUtils.Writer futureWriter(int opId) {
+    @Nullable @Override public PlatformFutureUtils.Writer futureWriter(int opId) {
         if (opId == OP_GET_ALL)
             return WRITER_GET_ALL;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
index 292caea..4c11cc0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/PlatformCacheIterator.java
@@ -47,7 +47,7 @@ public class PlatformCacheIterator extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_NEXT:
                 if (iter.hasNext()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
index 12df188..e24345c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinity.java
@@ -117,7 +117,7 @@ public class PlatformAffinity extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_PARTITION:
                 return aff.partition(reader.readObjectDetached());
@@ -168,7 +168,7 @@ public class PlatformAffinity extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions"})
-    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_PRIMARY_PARTITIONS: {

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
index 8076a19..2d3cada 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunction.java
@@ -26,6 +26,7 @@ import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxyImpl;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
@@ -279,7 +280,11 @@ public class PlatformAffinityFunction implements AffinityFunction, Externalizabl
                 ? new PlatformAffinityFunctionTarget(ctx, baseFunc)
                 : null;
 
-            ptr = ctx.gateway().affinityFunctionInit(mem.pointer(), baseTarget);
+            PlatformTargetProxyImpl baseTargetProxy = baseTarget != null
+                    ? new PlatformTargetProxyImpl(baseTarget, ctx)
+                    : null;
+
+            ptr = ctx.gateway().affinityFunctionInit(mem.pointer(), baseTargetProxy);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java
index 8a07b33..342e726 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget.java
@@ -71,7 +71,7 @@ public class PlatformAffinityFunctionTarget extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
         if (type == OP_PARTITION)
             return baseFunc.partition(reader.readObjectDetached());
         else if (type == OP_REMOVE_NODE) {
@@ -84,7 +84,7 @@ public class PlatformAffinityFunctionTarget extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         if (type == OP_ASSIGN_PARTITIONS) {
             AffinityFunctionContext affCtx = currentAffCtx.get();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
index 6a259ca..f201425 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformAbstractQueryCursor.java
@@ -71,7 +71,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, final BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, final BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_BATCH: {
                 assert iter != null : "iterator() has not been called";
@@ -136,7 +136,7 @@ public abstract class PlatformAbstractQueryCursor<T> extends PlatformAbstractTar
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_ITERATOR:
                 iter = cursor.iterator();

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java
index 04f17ff..27d784a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cache/query/PlatformContinuousQueryProxy.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.platform.cache.query;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
 
 /**
  * Proxy that implements PlatformTarget.
@@ -41,7 +42,7 @@ public class PlatformContinuousQueryProxy extends PlatformAbstractTarget  {
     }
 
     /** {@inheritDoc} */
-    @Override public Object outObject(int type) throws Exception {
+    @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
         return qry.getInitialQueryCursor();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
index f21861e..c77f501 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackGateway.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.platform.callback;
 
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
-import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunctionTarget;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxy;
 import org.apache.ignite.internal.util.GridStripedSpinBusyLock;
 
 /**
@@ -429,7 +429,7 @@ public class PlatformCallbackGateway {
      * @param memPtr Stream pointer.
      * @param keepBinary Binary flag.
      */
-    public void dataStreamerStreamReceiverInvoke(long ptr, Object cache, long memPtr, boolean keepBinary) {
+    public void dataStreamerStreamReceiverInvoke(long ptr, PlatformTargetProxy cache, long memPtr, boolean keepBinary) {
         enter();
 
         try {
@@ -995,7 +995,7 @@ public class PlatformCallbackGateway {
      * @param baseFunc Optional func for base calls.
      * @return Affinity function pointer.
      */
-    public long affinityFunctionInit(long memPtr, PlatformAffinityFunctionTarget baseFunc) {
+    public long affinityFunctionInit(long memPtr, PlatformTargetProxy baseFunc) {
         enter();
 
         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
index 50c4c28..9d60ec0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/callback/PlatformCallbackUtils.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.callback;
 
-import org.apache.ignite.internal.processors.platform.cache.affinity.PlatformAffinityFunctionTarget;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxy;
 
 /**
  * Platform callback utility methods. Implemented in target platform. All methods in this class must be
@@ -226,7 +226,7 @@ public class PlatformCallbackUtils {
      * @param memPtr Stream pointer.
      * @param keepBinary Binary flag.
      */
-    static native void dataStreamerStreamReceiverInvoke(long envPtr, long ptr, Object cache, long memPtr,
+    static native void dataStreamerStreamReceiverInvoke(long envPtr, long ptr, PlatformTargetProxy cache, long memPtr,
         boolean keepBinary);
 
     /**
@@ -504,7 +504,7 @@ public class PlatformCallbackUtils {
      * @param baseFunc Optional func for base calls.
      * @return Affinity function pointer.
      */
-    static native long affinityFunctionInit(long envPtr, long memPtr, PlatformAffinityFunctionTarget baseFunc);
+    static native long affinityFunctionInit(long envPtr, long memPtr, PlatformTargetProxy baseFunc);
 
     /**
      * Gets the partition from affinity function.

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
index dc73468..f49f477 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/cluster/PlatformClusterGroup.java
@@ -30,6 +30,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
@@ -120,7 +121,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings("deprecation")
-    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_METRICS:
                 platformCtx.writeClusterMetrics(writer, prj.metrics());
@@ -134,7 +135,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"ConstantConditions", "deprecation"})
-    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_METRICS_FILTERED: {
@@ -217,7 +218,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_PING_NODE:
                 return pingNode(reader.readUuid()) ? TRUE : FALSE;
@@ -228,7 +229,8 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader)
+        throws IgniteCheckedException {
         switch (type) {
             case OP_FOR_NODE_IDS: {
                 Collection<UUID> ids = PlatformUtils.readCollection(reader);
@@ -272,8 +274,8 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInObjectStreamOutObjectStream(
-            int type, @Nullable Object arg, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+    @Override public PlatformTarget processInObjectStreamOutObjectStream(
+            int type, @Nullable PlatformTarget arg, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
             throws IgniteCheckedException {
         switch (type) {
             case OP_FOR_OTHERS: {
@@ -289,7 +291,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+    @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
         switch (type) {
             case OP_FOR_REMOTES:
                 return new PlatformClusterGroup(platformCtx, (ClusterGroupEx)prj.forRemotes());
@@ -314,7 +316,7 @@ public class PlatformClusterGroup extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_RESET_METRICS: {
                 assert prj instanceof IgniteCluster; // Can only be invoked on top-level cluster group.


[23/25] ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-4371
Commit: fa358cc1b81310cf5ffe5a3da17a3dad842de33f
Parents: b65802b 10ade28
Author: devozerov <vo...@gridgain.com>
Authored: Fri Dec 9 13:10:43 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 9 13:10:43 2016 +0300

----------------------------------------------------------------------
 .../ignite/examples/ExampleNodeStartup.java     |    2 +-
 .../examples/datagrid/CachePutGetExample.java   |    2 +-
 .../scalar/examples/ScalarJvmCloudExample.scala |    2 +-
 .../rest/ClientMemcachedProtocolSelfTest.java   |    4 +-
 .../rest/protocols/tcp/MockNioSession.java      |   25 +-
 .../apache/ignite/IgniteSystemProperties.java   |    3 +
 .../cache/store/CacheLoadOnlyStoreAdapter.java  |    6 +-
 .../configuration/IgniteConfiguration.java      |   50 +-
 .../internal/GridEventConsumeHandler.java       |    2 +-
 .../ignite/internal/GridJobContextImpl.java     |    4 +-
 .../ignite/internal/GridKernalContext.java      |    9 +
 .../ignite/internal/GridKernalContextImpl.java  |   16 +-
 .../internal/GridPerformanceSuggestions.java    |    2 +-
 .../org/apache/ignite/internal/GridTopic.java   |    5 +-
 .../ignite/internal/IgniteInternalFuture.java   |   11 +
 .../apache/ignite/internal/IgniteKernal.java    |   85 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   32 +-
 .../GridClientConnectionManagerAdapter.java     |    1 +
 .../client/router/impl/GridTcpRouterImpl.java   |    1 +
 .../managers/communication/GridIoManager.java   |  207 ++-
 .../managers/communication/GridIoMessage.java   |   13 +
 .../communication/GridIoMessageFactory.java     |   12 +-
 .../communication/IgniteIoTestMessage.java      |  235 +++
 .../processors/cache/GridCacheAdapter.java      |   26 +-
 .../processors/cache/GridCacheMessage.java      |    7 +
 .../processors/cache/GridCacheUtils.java        |   35 +
 .../processors/cache/IgniteCacheProxy.java      |    8 +
 .../distributed/GridDistributedLockRequest.java |    5 +
 .../GridDistributedTxFinishResponse.java        |    6 +
 .../GridDistributedUnlockRequest.java           |    5 +
 .../distributed/dht/GridDhtCacheAdapter.java    |    3 +-
 .../distributed/dht/GridDhtLockResponse.java    |    9 +-
 .../dht/atomic/GridDhtAtomicCache.java          |    5 +-
 .../GridDhtAtomicSingleUpdateRequest.java       |    5 +
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |    5 +
 .../atomic/GridNearAtomicFullUpdateRequest.java |    5 +
 .../GridNearAtomicSingleUpdateRequest.java      |    5 +
 .../distributed/near/GridNearGetRequest.java    |    5 +
 .../local/atomic/GridLocalAtomicCache.java      |    3 +
 .../query/GridCacheDistributedQueryManager.java |    2 +-
 .../cache/query/GridCacheQueryRequest.java      |    6 +-
 .../transactions/IgniteTxLocalAdapter.java      |    8 +-
 .../datastreamer/DataStreamProcessor.java       |   22 +-
 .../internal/processors/igfs/IgfsContext.java   |    4 +-
 .../processors/igfs/IgfsDataManager.java        |    6 +-
 .../internal/processors/igfs/IgfsImpl.java      |    2 +-
 .../internal/processors/odbc/OdbcProcessor.java |    1 +
 .../platform/compute/PlatformCompute.java       |    6 +
 .../tcp/GridTcpMemcachedNioListener.java        |   15 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |    2 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |    1 +
 .../service/GridServiceProcessor.java           |    6 +-
 .../ignite/internal/util/IgniteUtils.java       |   62 +-
 .../ignite/internal/util/StripedExecutor.java   |  667 +++++++++
 .../util/future/GridFinishedFuture.java         |   24 +
 .../internal/util/future/GridFutureAdapter.java |   15 +-
 .../util/future/GridFutureChainListener.java    |   30 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |    2 +-
 .../nio/GridAbstractCommunicationClient.java    |   12 +-
 .../util/nio/GridCommunicationClient.java       |    9 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |   15 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   10 +-
 .../internal/util/nio/GridNioCodecFilter.java   |   17 +-
 .../ignite/internal/util/nio/GridNioFilter.java |   16 +-
 .../internal/util/nio/GridNioFilterAdapter.java |   10 +-
 .../internal/util/nio/GridNioFilterChain.java   |   14 +-
 .../ignite/internal/util/nio/GridNioFuture.java |    4 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |  124 +-
 .../ignite/internal/util/nio/GridNioServer.java | 1404 +++++++++++++++---
 .../internal/util/nio/GridNioSession.java       |   25 +-
 .../internal/util/nio/GridNioSessionImpl.java   |   65 +-
 .../ignite/internal/util/nio/GridNioWorker.java |   48 +
 .../util/nio/GridSelectorNioSessionImpl.java    |  221 ++-
 .../util/nio/GridShmemCommunicationClient.java  |    7 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   55 +-
 .../internal/util/nio/SessionWriteRequest.java  |   85 ++
 .../internal/util/nio/ssl/GridNioSslFilter.java |   10 +-
 .../util/nio/ssl/GridNioSslHandler.java         |    4 +-
 .../util/tostring/GridToStringBuilder.java      |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 1340 ++++++++++++-----
 .../tcp/TcpCommunicationSpiMBean.java           |   40 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   14 +-
 .../ignite/stream/socket/SocketStreamer.java    |    1 +
 .../ignite/thread/IgniteThreadFactory.java      |    8 +-
 .../IgniteSlowClientDetectionSelfTest.java      |    1 +
 ...unicationBalanceMultipleConnectionsTest.java |   28 +
 .../IgniteCommunicationBalanceTest.java         |  339 +++++
 .../communication/IgniteIoTestMessagesTest.java |   95 ++
 .../IgniteVariousConnectionNumberTest.java      |  166 +++
 .../cache/CrossCacheTxRandomOperationsTest.java |   30 +-
 ...idAbstractCacheInterceptorRebalanceTest.java |    4 +-
 ...CacheOffHeapMultiThreadedUpdateSelfTest.java |    6 +-
 ...eAtomicMessageRecovery10ConnectionsTest.java |   28 +
 ...cMessageRecoveryNoPairedConnectionsTest.java |   47 +
 ...acheConnectionRecovery10ConnectionsTest.java |   35 +
 .../distributed/IgniteCacheCreatePutTest.java   |    2 +-
 .../IgniteCacheMessageRecoveryAbstractTest.java |   24 +-
 .../IgniteCacheMessageWriteTimeoutTest.java     |   17 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    6 +-
 ...erNoStripedPoolMultiNodeFullApiSelfTest.java |   35 +
 ...edNoStripedPoolMultiNodeFullApiSelfTest.java |   35 +
 .../TxDeadlockDetectionNoHangsTest.java         |    2 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   29 +-
 .../GridServiceProcessorProxySelfTest.java      |    2 +-
 .../util/future/GridFutureAdapterSelfTest.java  |  122 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   32 +-
 .../loadtests/nio/GridNioBenchmarkClient.java   |    4 +-
 .../p2p/GridP2PRecursionTaskSelfTest.java       |    2 +-
 .../spi/GridTcpSpiForwardingSelfTest.java       |   18 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   28 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   82 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    5 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |   23 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    9 +-
 ...tionSpiRecoveryFailureDetectionSelfTest.java |    1 +
 ...ationSpiRecoveryNoPairedConnectionsTest.java |   28 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   67 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |    9 +-
 .../junits/GridTestKernalContext.java           |    4 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |    6 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   17 +-
 .../IgniteSpiCommunicationSelfTestSuite.java    |    2 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |    4 +-
 .../HadoopExternalCommunication.java            |    5 +-
 .../communication/HadoopIpcToNioAdapter.java    |    2 +-
 .../communication/HadoopMarshallerFilter.java   |    6 +-
 .../ignite/stream/kafka/KafkaStreamer.java      |    2 +-
 .../ignite/tools/classgen/ClassesGenerator.java |    8 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |    6 +-
 .../yardstick/cache/CacheEntryEventProbe.java   |    2 +-
 .../yardstick/cache/IgniteIoTestBenchmark.java  |   73 +
 .../io/IgniteIoTestAbstractBenchmark.java       |   61 +
 .../io/IgniteIoTestSendAllBenchmark.java        |   32 +
 .../io/IgniteIoTestSendRandomBenchmark.java     |   35 +
 134 files changed, 5935 insertions(+), 998 deletions(-)
----------------------------------------------------------------------



[04/25] ignite git commit: Improved exception handling for failed queries.

Posted by sb...@apache.org.
Improved exception handling for failed queries.


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

Branch: refs/heads/ignite-4371
Commit: 0d4a1b7381fece47ee480f3a06bff7c51a7fead4
Parents: 6ba1711
Author: Alexey Kuznetsov <ak...@apache.org>
Authored: Wed Dec 7 18:02:49 2016 +0700
Committer: Alexey Kuznetsov <ak...@apache.org>
Committed: Wed Dec 7 18:02:49 2016 +0700

----------------------------------------------------------------------
 .../java/org/apache/ignite/internal/visor/query/VisorQueryJob.java | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/0d4a1b73/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
index 6d1de6a..c66b2dd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryJob.java
@@ -197,7 +197,7 @@ public class VisorQueryJob extends VisorJob<VisorQueryArg, IgniteBiTuple<? exten
                 }
             }
         }
-        catch (Exception e) {
+        catch (Throwable e) {
             return new IgniteBiTuple<>(new VisorExceptionWrapper(e), null);
         }
     }


[03/25] ignite git commit: IGNITE-4242 ExchangeManager should wait for cache rebalancing in async way

Posted by sb...@apache.org.
IGNITE-4242 ExchangeManager should wait for cache rebalancing in async way


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

Branch: refs/heads/ignite-4371
Commit: 6ba1711a1fa10d8276974227491136070c3ed43a
Parents: acf20b3
Author: Anton Vinogradov <av...@apache.org>
Authored: Tue Dec 6 12:55:41 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Tue Dec 6 12:55:41 2016 +0300

----------------------------------------------------------------------
 .../GridCachePartitionExchangeManager.java      | 128 +++--------
 .../processors/cache/GridCachePreloader.java    |  11 +-
 .../cache/GridCachePreloaderAdapter.java        |   5 +-
 .../dht/preloader/GridDhtPartitionDemander.java | 230 +++++++++++--------
 .../dht/preloader/GridDhtPreloader.java         |   9 +-
 .../GridCacheRebalancingSyncSelfTest.java       |   2 +
 6 files changed, 183 insertions(+), 202 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 7a24aa1..f04a6ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -21,18 +21,18 @@ import java.text.DateFormat;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
 import java.util.Comparator;
 import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
-import java.util.Queue;
 import java.util.TreeMap;
 import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.LinkedBlockingDeque;
@@ -87,7 +87,6 @@ import org.apache.ignite.internal.util.typedef.CI2;
 import org.apache.ignite.internal.util.typedef.F;
 import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
-import org.apache.ignite.internal.util.typedef.internal.GPC;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.internal.util.worker.GridWorker;
@@ -97,13 +96,11 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
-import org.jsr166.ConcurrentLinkedDeque8;
 
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_PRELOAD_RESEND_TIMEOUT;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_THREAD_DUMP_ON_EXCHANGE_TIMEOUT;
 import static org.apache.ignite.IgniteSystemProperties.getLong;
-import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STARTED;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
@@ -156,9 +153,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
     /** */
     private GridFutureAdapter<?> reconnectExchangeFut;
 
-    /** */
-    private final Queue<Callable<Boolean>> rebalanceQ = new ConcurrentLinkedDeque8<>();
-
     /**
      * Partition map futures.
      * This set also contains already completed exchange futures to address race conditions when coordinator
@@ -1596,12 +1590,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
         @Override protected void body() throws InterruptedException, IgniteInterruptedCheckedException {
             long timeout = cctx.gridConfig().getNetworkTimeout();
 
-            boolean startEvtFired = false;
-
             int cnt = 0;
 
-            IgniteInternalFuture asyncStartFut = null;
-
             while (!isCancelled()) {
                 GridDhtPartitionsExchangeFuture exchFut = null;
 
@@ -1703,20 +1693,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                     continue;
 
                                 changed |= cacheCtx.topology().afterExchange(exchFut);
-
-                                // Preload event notification.
-                                if (!exchFut.skipPreload() && cacheCtx.events().isRecordable(EVT_CACHE_REBALANCE_STARTED)) {
-                                    if (!cacheCtx.isReplicated() || !startEvtFired) {
-                                        DiscoveryEvent discoEvt = exchFut.discoveryEvent();
-
-                                        cacheCtx.events().addPreloadEvent(-1, EVT_CACHE_REBALANCE_STARTED,
-                                            discoEvt.eventNode(), discoEvt.type(), discoEvt.timestamp());
-                                    }
-                                }
                             }
 
-                            startEvtFired = true;
-
                             if (!cctx.kernalContext().clientNode() && changed && futQ.isEmpty())
                                 refreshPartitions();
                         }
@@ -1755,8 +1733,6 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                     if (assignsMap != null) {
                         int size = assignsMap.size();
 
-                        rebalanceQ.clear();
-
                         NavigableMap<Integer, List<Integer>> orderMap = new TreeMap<>();
 
                         for (Map.Entry<Integer, GridDhtPreloaderAssignments> e : assignsMap.entrySet()) {
@@ -1772,101 +1748,65 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                             orderMap.get(order).add(cacheId);
                         }
 
-                        Callable<Boolean> marshR = null;
-                        List<Callable<Boolean>> orderedRs = new ArrayList<>(size);
+                        Runnable r = null;
+
+                        List<String> rebList = new LinkedList<>();
+
+                        boolean assignsCancelled = false;
 
-                        //Ordered rebalance scheduling.
-                        for (Integer order : orderMap.keySet()) {
+                        for (Integer order : orderMap.descendingKeySet()) {
                             for (Integer cacheId : orderMap.get(order)) {
                                 GridCacheContext<K, V> cacheCtx = cctx.cacheContext(cacheId);
 
-                                List<String> waitList = new ArrayList<>(size - 1);
+                                GridDhtPreloaderAssignments assigns = assignsMap.get(cacheId);
 
-                                for (List<Integer> cIds : orderMap.headMap(order).values()) {
-                                    for (Integer cId : cIds)
-                                        waitList.add(cctx.cacheContext(cId).name());
-                                }
+                                if (assigns != null)
+                                    assignsCancelled |= assigns.cancelled();
 
-                                Callable<Boolean> r = cacheCtx.preloader().addAssignments(assignsMap.get(cacheId),
+                                // Cancels previous rebalance future (in case it's not done yet).
+                                // Sends previous rebalance stopped event (if necessary).
+                                // Creates new rebalance future.
+                                // Sends current rebalance started event (if necessary).
+                                // Finishes cache sync future (on empty assignments).
+                                Runnable cur = cacheCtx.preloader().addAssignments(assigns,
                                     forcePreload,
-                                    waitList,
-                                    cnt);
+                                    cnt,
+                                    r);
 
-                                if (r != null) {
-                                    U.log(log, "Cache rebalancing scheduled: [cache=" + cacheCtx.name() +
-                                        ", waitList=" + waitList.toString() + "]");
+                                if (cur != null) {
+                                    rebList.add(U.maskName(cacheCtx.name()));
 
-                                    if (cacheId == CU.cacheId(GridCacheUtils.MARSH_CACHE_NAME))
-                                        marshR = r;
-                                    else
-                                        orderedRs.add(r);
+                                    r = cur;
                                 }
                             }
                         }
 
-                        if (asyncStartFut != null)
-                            asyncStartFut.get(); // Wait for thread stop.
+                        if (assignsCancelled) { // Pending exchange.
+                            U.log(log, "Skipping rebalancing (obsolete exchange ID) " +
+                                "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
+                                ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
+                        }
+                        else if (r != null) {
+                            Collections.reverse(rebList);
 
-                        rebalanceQ.addAll(orderedRs);
+                            U.log(log, "Rebalancing scheduled [order=" + rebList + "]");
 
-                        if (marshR != null || !rebalanceQ.isEmpty()) {
                             if (futQ.isEmpty()) {
-                                U.log(log, "Rebalancing required " +
+                                U.log(log, "Rebalancing started " +
                                     "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
                                     ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
 
-                                if (marshR != null) {
-                                    try {
-                                        marshR.call(); //Marshaller cache rebalancing launches in sync way.
-                                    }
-                                    catch (Exception ex) {
-                                        if (log.isDebugEnabled())
-                                            log.debug("Failed to send initial demand request to node");
-
-                                        continue;
-                                    }
-                                }
-
-                                final GridFutureAdapter fut = new GridFutureAdapter();
-
-                                asyncStartFut = fut;
-
-                                cctx.kernalContext().closure().callLocalSafe(new GPC<Boolean>() {
-                                    @Override public Boolean call() {
-                                        try {
-                                            while (true) {
-                                                Callable<Boolean> r = rebalanceQ.poll();
-
-                                                if (r == null)
-                                                    return false;
-
-                                                if (!r.call())
-                                                    return false;
-                                            }
-                                        }
-                                        catch (Exception ex) {
-                                            if (log.isDebugEnabled())
-                                                log.debug("Failed to send initial demand request to node");
-
-                                            return false;
-                                        }
-                                        finally {
-                                            fut.onDone();
-                                        }
-                                    }
-                                }, /*system pool*/true);
+                                r.run(); // Starts rebalancing routine.
                             }
-                            else {
+                            else
                                 U.log(log, "Skipping rebalancing (obsolete exchange ID) " +
                                     "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
                                     ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
-                            }
                         }
-                        else {
+                        else
                             U.log(log, "Skipping rebalancing (nothing scheduled) " +
                                 "[top=" + exchFut.topologyVersion() + ", evt=" + exchFut.discoveryEvent().name() +
                                 ", node=" + exchFut.discoveryEvent().eventNode().id() + ']');
-                        }
                     }
                 }
                 catch (IgniteInterruptedCheckedException e) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 1d1cfab..3c4456d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache;
 
 import java.util.Collection;
 import java.util.UUID;
-import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
@@ -84,14 +83,14 @@ public interface GridCachePreloader {
      *
      * @param assignments Assignments to add.
      * @param forcePreload Force preload flag.
-     * @param caches Rebalancing of these caches will be finished before this started.
      * @param cnt Counter.
-     * @return Rebalancing closure.
+     * @param next Runnable responsible for cache rebalancing start.
+     * @return Rebalancing runnable.
      */
-    public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments,
+    public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
         boolean forcePreload,
-        Collection<String> caches,
-        int cnt);
+        int cnt,
+        Runnable next);
 
     /**
      * @param p Preload predicate.

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index b15ebc5..656a960 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache;
 
 import java.util.Collection;
 import java.util.UUID;
-import java.util.concurrent.Callable;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cache.affinity.AffinityFunction;
@@ -166,8 +165,8 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
-    @Override public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
-        Collection<String> caches, int cnt) {
+    @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
+        int cnt, Runnable next) {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index 57d5229..a6808c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -28,8 +28,8 @@ import java.util.Map;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.Callable;
 import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -72,6 +72,7 @@ import org.jetbrains.annotations.Nullable;
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_OBJECT_LOADED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_PART_LOADED;
+import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STARTED;
 import static org.apache.ignite.events.EventType.EVT_CACHE_REBALANCE_STOPPED;
 import static org.apache.ignite.internal.GridTopic.TOPIC_CACHE;
 import static org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState.MOVING;
@@ -122,6 +123,18 @@ public class GridDhtPartitionDemander {
     private final Map<Integer, Object> rebalanceTopics;
 
     /**
+     * Started event sent.
+     * Make sense for replicated cache only.
+     */
+    private final AtomicBoolean startedEvtSent = new AtomicBoolean();
+
+    /**
+     * Stopped event sent.
+     * Make sense for replicated cache only.
+     */
+    private final AtomicBoolean stoppedEvtSent = new AtomicBoolean();
+
+    /**
      * @param cctx Cctx.
      * @param demandLock Demand lock.
      */
@@ -249,45 +262,25 @@ public class GridDhtPartitionDemander {
     }
 
     /**
-     * @param name Cache name.
-     * @param fut Future.
-     * @throws IgniteCheckedException If failed.
+     * Sets last exchange future.
+     *
+     * @param lastFut Last future to set.
      */
-    private boolean waitForCacheRebalancing(String name, RebalanceFuture fut) throws IgniteCheckedException {
-        if (log.isDebugEnabled())
-            log.debug("Waiting for another cache to start rebalancing [cacheName=" + cctx.name() +
-                ", waitCache=" + name + ']');
-
-        RebalanceFuture wFut = (RebalanceFuture)cctx.kernalContext().cache().internalCache(name)
-            .preloader().rebalanceFuture();
-
-        if (!topologyChanged(fut) && wFut.updateSeq == fut.updateSeq) {
-            if (!wFut.get()) {
-                U.log(log, "Skipping waiting of " + name + " cache [top=" + fut.topologyVersion() +
-                    "] (cache rebalanced with missed partitions)");
-
-                return false;
-            }
-
-            return true;
-        }
-        else {
-            U.log(log, "Skipping waiting of " + name + " cache [top=" + fut.topologyVersion() +
-                "] (topology already changed)");
-
-            return false;
-        }
+    void onTopologyChanged(GridDhtPartitionsExchangeFuture lastFut) {
+        lastExchangeFut = lastFut;
     }
 
     /**
      * @param assigns Assignments.
      * @param force {@code True} if dummy reassign.
-     * @param caches Rebalancing of these caches will be finished before this started.
      * @param cnt Counter.
-     * @return Rebalancing closure.
+     * @param next Runnable responsible for cache rebalancing start.
+     * @return Rebalancing runnable.
      */
-    Callable<Boolean> addAssignments(final GridDhtPreloaderAssignments assigns, boolean force,
-        final Collection<String> caches, int cnt) {
+    Runnable addAssignments(final GridDhtPreloaderAssignments assigns,
+        boolean force,
+        int cnt,
+        final Runnable next) {
         if (log.isDebugEnabled())
             log.debug("Adding partition assignments: " + assigns);
 
@@ -296,7 +289,7 @@ public class GridDhtPartitionDemander {
         if (delay == 0 || force) {
             final RebalanceFuture oldFut = rebalanceFut;
 
-            final RebalanceFuture fut = new RebalanceFuture(assigns, cctx, log, oldFut.isInitial(), cnt);
+            final RebalanceFuture fut = new RebalanceFuture(assigns, cctx, log, startedEvtSent, stoppedEvtSent, cnt);
 
             if (!oldFut.isInitial())
                 oldFut.cancel();
@@ -310,20 +303,69 @@ public class GridDhtPartitionDemander {
 
             rebalanceFut = fut;
 
-            if (assigns.isEmpty()) {
-                fut.doneIfEmpty(assigns.cancelled());
+            fut.sendRebalanceStartedEvent();
+
+            if (assigns.cancelled()) { // Pending exchange.
+                if (log.isDebugEnabled())
+                    log.debug("Rebalancing skipped due to cancelled assignments.");
+
+                fut.onDone(false);
+
+                fut.sendRebalanceFinishedEvent();
+
+                return null;
+            }
+
+            if (assigns.isEmpty()) { // Nothing to rebalance.
+                if (log.isDebugEnabled())
+                    log.debug("Rebalancing skipped due to empty assignments.");
+
+                fut.onDone(true);
+
+                ((GridFutureAdapter)cctx.preloader().syncFuture()).onDone();
+
+                fut.sendRebalanceFinishedEvent();
 
                 return null;
             }
 
-            return new Callable<Boolean>() {
-                @Override public Boolean call() throws Exception {
-                    for (String c : caches) {
-                        if (!waitForCacheRebalancing(c, fut))
-                            return false;
+            return new Runnable() {
+                @Override public void run() {
+                    try {
+                        if (next != null)
+                            fut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+                                @Override public void apply(IgniteInternalFuture<Boolean> f) {
+                                    try {
+                                        if (f.get()) // Not cancelled.
+                                            next.run(); // Starts next cache rebalancing (according to the order).
+                                    }
+                                    catch (IgniteCheckedException ignored) {
+                                        if (log.isDebugEnabled())
+                                            log.debug(ignored.getMessage());
+                                    }
+                                }
+                            });
+
+                        requestPartitions(fut, assigns);
                     }
+                    catch (IgniteCheckedException e) {
+                        ClusterTopologyCheckedException cause = e.getCause(ClusterTopologyCheckedException.class);
 
-                    return requestPartitions(fut, assigns);
+                        if (cause != null)
+                            log.warning("Failed to send initial demand request to node. " + e.getMessage());
+                        else
+                            log.error("Failed to send initial demand request to node.", e);
+
+                        fut.cancel();
+                    }
+                    catch (Throwable th) {
+                        log.error("Runtime error caught during initial demand request sending.", th);
+
+                        fut.cancel();
+
+                        if (th instanceof Error)
+                            throw th;
+                    }
                 }
             };
         }
@@ -361,14 +403,17 @@ public class GridDhtPartitionDemander {
      * @throws IgniteCheckedException If failed.
      * @return Partitions were requested.
      */
-    private boolean requestPartitions(
+    private void requestPartitions(
         RebalanceFuture fut,
         GridDhtPreloaderAssignments assigns
     ) throws IgniteCheckedException {
-        for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
-            if (topologyChanged(fut))
-                return false;
+        if (topologyChanged(fut)) {
+            fut.cancel();
+
+            return;
+        }
 
+        for (Map.Entry<ClusterNode, GridDhtPartitionDemandMessage> e : assigns.entrySet()) {
             final ClusterNode node = e.getKey();
 
             GridDhtPartitionDemandMessage d = e.getValue();
@@ -387,7 +432,7 @@ public class GridDhtPartitionDemander {
 
             //Check remote node rebalancing API version.
             if (node.version().compareTo(GridDhtPreloader.REBALANCING_VER_2_SINCE) >= 0) {
-                U.log(log, "Starting rebalancing [cache=" + cctx.name() + ", mode=" + cfg.getRebalanceMode() +
+                U.log(log, "Starting rebalancing [mode=" + cfg.getRebalanceMode() +
                     ", fromNode=" + node.id() + ", partitionsCount=" + parts.size() +
                     ", topology=" + fut.topologyVersion() + ", updateSeq=" + fut.updateSeq + "]");
 
@@ -446,8 +491,6 @@ public class GridDhtPartitionDemander {
                 worker.run(node, d);
             }
         }
-
-        return true;
     }
 
     /**
@@ -739,23 +782,17 @@ public class GridDhtPartitionDemander {
     }
 
     /**
-     * Sets last exchange future.
-     *
-     * @param lastFut Last future to set.
-     */
-    void updateLastExchangeFuture(GridDhtPartitionsExchangeFuture lastFut) {
-        lastExchangeFut = lastFut;
-    }
-
-    /**
      *
      */
     public static class RebalanceFuture extends GridFutureAdapter<Boolean> {
         /** */
         private static final long serialVersionUID = 1L;
 
-        /** Should EVT_CACHE_REBALANCE_STOPPED event be sent of not. */
-        private final boolean sndStoppedEvnt;
+        /** Should EVT_CACHE_REBALANCE_STARTED event be sent or not. */
+        private final AtomicBoolean startedEvtSent;
+
+        /** Should EVT_CACHE_REBALANCE_STOPPED event be sent or not. */
+        private final AtomicBoolean stoppedEvtSent;
 
         /** */
         private final GridCacheContext<?, ?> cctx;
@@ -783,13 +820,15 @@ public class GridDhtPartitionDemander {
          * @param assigns Assigns.
          * @param cctx Context.
          * @param log Logger.
-         * @param sentStopEvnt Stop event flag.
+         * @param startedEvtSent Start event sent flag.
+         * @param stoppedEvtSent Stop event sent flag.
          * @param updateSeq Update sequence.
          */
         RebalanceFuture(GridDhtPreloaderAssignments assigns,
             GridCacheContext<?, ?> cctx,
             IgniteLogger log,
-            boolean sentStopEvnt,
+            AtomicBoolean startedEvtSent,
+            AtomicBoolean stoppedEvtSent,
             long updateSeq) {
             assert assigns != null;
 
@@ -797,7 +836,8 @@ public class GridDhtPartitionDemander {
             this.topVer = assigns.topologyVersion();
             this.cctx = cctx;
             this.log = log;
-            this.sndStoppedEvnt = sentStopEvnt;
+            this.startedEvtSent = startedEvtSent;
+            this.stoppedEvtSent = stoppedEvtSent;
             this.updateSeq = updateSeq;
         }
 
@@ -809,7 +849,8 @@ public class GridDhtPartitionDemander {
             this.topVer = null;
             this.cctx = null;
             this.log = null;
-            this.sndStoppedEvnt = false;
+            this.startedEvtSent = null;
+            this.stoppedEvtSent = null;
             this.updateSeq = -1;
         }
 
@@ -848,24 +889,6 @@ public class GridDhtPartitionDemander {
         }
 
         /**
-         * @param cancelled Is cancelled.
-         */
-        private void doneIfEmpty(boolean cancelled) {
-            synchronized (this) {
-                if (isDone())
-                    return;
-
-                assert remaining.isEmpty();
-
-                if (log.isDebugEnabled())
-                    log.debug("Rebalancing is not required [cache=" + cctx.name() +
-                        ", topology=" + topVer + "]");
-
-                checkIsDone(cancelled, true);
-            }
-        }
-
-        /**
          * Cancels this future.
          *
          * @return {@code True}.
@@ -875,8 +898,7 @@ public class GridDhtPartitionDemander {
                 if (isDone())
                     return true;
 
-                U.log(log, "Cancelled rebalancing from all nodes [cache=" + cctx.name()
-                    + ", topology=" + topologyVersion() + ']');
+                U.log(log, "Cancelled rebalancing from all nodes [topology=" + topologyVersion() + ']');
 
                 if (!cctx.kernalContext().isStopping()) {
                     for (UUID nodeId : remaining.keySet())
@@ -885,7 +907,7 @@ public class GridDhtPartitionDemander {
 
                 remaining.clear();
 
-                checkIsDone(true /* cancelled */, false);
+                checkIsDone(true /* cancelled */);
             }
 
             return true;
@@ -907,7 +929,7 @@ public class GridDhtPartitionDemander {
 
                 remaining.remove(nodeId);
 
-                onDone(false); // Finishing rebalance future a non completed.
+                onDone(false); // Finishing rebalance future as non completed.
 
                 checkIsDone(); // But will finish syncFuture only when other nodes are preloaded or rebalancing cancelled.
             }
@@ -988,8 +1010,7 @@ public class GridDhtPartitionDemander {
 
                 if (parts.isEmpty()) {
                     U.log(log, "Completed " + ((remaining.size() == 1 ? "(final) " : "") +
-                        "rebalancing [cache=" + cctx.name() +
-                        ", fromNode=" + nodeId + ", topology=" + topologyVersion() +
+                        "rebalancing [fromNode=" + nodeId + ", topology=" + topologyVersion() +
                         ", time=" + (U.currentTimeMillis() - t.get1()) + " ms]"));
 
                     remaining.remove(nodeId);
@@ -1022,23 +1043,20 @@ public class GridDhtPartitionDemander {
          *
          */
         private void checkIsDone() {
-            checkIsDone(false, false);
+            checkIsDone(false);
         }
 
         /**
          * @param cancelled Is cancelled.
-         * @param wasEmpty {@code True} if future was created without assignments.
          */
-        private void checkIsDone(boolean cancelled, boolean wasEmpty) {
+        private void checkIsDone(boolean cancelled) {
             if (remaining.isEmpty()) {
-                if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STOPPED) && (!cctx.isReplicated() || sndStoppedEvnt))
-                    preloadEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent());
+                sendRebalanceFinishedEvent();
 
                 if (log.isDebugEnabled())
                     log.debug("Completed rebalance future: " + this);
 
-                if (!wasEmpty)
-                    cctx.shared().exchange().scheduleResendPartitions();
+                cctx.shared().exchange().scheduleResendPartitions();
 
                 Collection<Integer> m = new HashSet<>();
 
@@ -1064,6 +1082,30 @@ public class GridDhtPartitionDemander {
             }
         }
 
+        /**
+         *
+         */
+        private void sendRebalanceStartedEvent() {
+            if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STARTED) &&
+                (!cctx.isReplicated() || !startedEvtSent.get())) {
+                preloadEvent(EVT_CACHE_REBALANCE_STARTED, exchFut.discoveryEvent());
+
+                startedEvtSent.set(true);
+            }
+        }
+
+        /**
+         *
+         */
+        private void sendRebalanceFinishedEvent() {
+            if (cctx.events().isRecordable(EVT_CACHE_REBALANCE_STOPPED) &&
+                (!cctx.isReplicated() || !stoppedEvtSent.get())) {
+                preloadEvent(EVT_CACHE_REBALANCE_STOPPED, exchFut.discoveryEvent());
+
+                stoppedEvtSent.set(true);
+            }
+        }
+
         /** {@inheritDoc} */
         public String toString() {
             return S.toString(RebalanceFuture.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 0865d9f..692e7c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -22,7 +22,6 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.UUID;
-import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.atomic.AtomicInteger;
 import java.util.concurrent.locks.ReadWriteLock;
@@ -255,7 +254,7 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     @Override public void onTopologyChanged(GridDhtPartitionsExchangeFuture lastFut) {
         supplier.onTopologyChanged(lastFut.topologyVersion());
 
-        demander.updateLastExchangeFuture(lastFut);
+        demander.onTopologyChanged(lastFut);
     }
 
     /** {@inheritDoc} */
@@ -413,9 +412,9 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public Callable<Boolean> addAssignments(GridDhtPreloaderAssignments assignments,
-        boolean forcePreload, Collection<String> caches, int cnt) {
-        return demander.addAssignments(assignments, forcePreload, caches, cnt);
+    @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
+        boolean forcePreload, int cnt, Runnable next) {
+        return demander.addAssignments(assignments, forcePreload, cnt, next);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/6ba1711a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
index de38952..3dfcd85 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/rebalancing/GridCacheRebalancingSyncSelfTest.java
@@ -501,6 +501,8 @@ public class GridCacheRebalancingSyncSelfTest extends GridCommonAbstractTest {
 
         record = true;
 
+        log.info("Checking GridDhtPartitions*Message absent (it will take 30 SECONDS) ... ");
+
         U.sleep(30_000);
 
         record = false;


[13/25] ignite git commit: Merge remote-tracking branch 'origin/master'

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


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

Branch: refs/heads/ignite-4371
Commit: 2f51b4ac0966d6786f4a172023e2f238ac20aba0
Parents: 9a691c4 c14561c
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 8 18:04:11 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 8 18:04:11 2016 +0300

----------------------------------------------------------------------

----------------------------------------------------------------------



[08/25] ignite git commit: IGNITE-4367 .NET: Fix ServicesTest

Posted by sb...@apache.org.
IGNITE-4367 .NET: Fix ServicesTest


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

Branch: refs/heads/ignite-4371
Commit: 597f3a581a0cc5e6061fd49071c13856be9d3627
Parents: 858b129
Author: Pavel Tupitsyn <pt...@apache.org>
Authored: Thu Dec 8 13:18:31 2016 +0300
Committer: Pavel Tupitsyn <pt...@apache.org>
Committed: Thu Dec 8 13:18:31 2016 +0300

----------------------------------------------------------------------
 .../Apache.Ignite.Core.Tests/Services/ServicesTest.cs       | 9 +++------
 1 file changed, 3 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/597f3a58/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
index 38a96bd..dcddfdb 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core.Tests/Services/ServicesTest.cs
@@ -762,9 +762,6 @@ namespace Apache.Ignite.Core.Tests.Services
             [InstanceResource]
             private IIgnite _grid;
 
-            /** */
-            private readonly object _syncRoot = new object();
-
             /** <inheritdoc /> */
             public int TestProperty { get; set; }
 
@@ -810,7 +807,7 @@ namespace Apache.Ignite.Core.Tests.Services
             /** <inheritdoc /> */
             public void Init(IServiceContext context)
             {
-                lock (_syncRoot)
+                lock (this)
                 {
                     if (ThrowInit)
                         throw new Exception("Expected exception");
@@ -825,7 +822,7 @@ namespace Apache.Ignite.Core.Tests.Services
             /** <inheritdoc /> */
             public void Execute(IServiceContext context)
             {
-                lock (_syncRoot)
+                lock (this)
                 {
                     if (ThrowExecute)
                         throw new Exception("Expected exception");
@@ -843,7 +840,7 @@ namespace Apache.Ignite.Core.Tests.Services
             /** <inheritdoc /> */
             public void Cancel(IServiceContext context)
             {
-                lock (_syncRoot)
+                lock (this)
                 {
                     if (ThrowCancel)
                         throw new Exception("Expected exception");


[14/25] ignite git commit: IGNITE-4270: Hadoop: implemented striped mapper output. This closes #1334.

Posted by sb...@apache.org.
IGNITE-4270: Hadoop: implemented striped mapper output. This closes #1334.


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

Branch: refs/heads/ignite-4371
Commit: 065ca4a75a0765409a27d87c781efb215c0a6c48
Parents: 2f51b4a
Author: devozerov <vo...@gridgain.com>
Authored: Fri Dec 9 12:01:40 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 9 12:01:40 2016 +0300

----------------------------------------------------------------------
 .../communication/GridIoMessageFactory.java     |   6 +
 .../processors/hadoop/HadoopJobProperty.java    |   7 +
 .../hadoop/HadoopMapperAwareTaskOutput.java     |  32 ++
 .../processors/hadoop/HadoopTaskInfo.java       |  43 ++
 .../shuffle/HadoopDirectShuffleMessage.java     | 243 ++++++++++++
 .../processors/hadoop/HadoopMapperUtils.java    |  56 +++
 .../hadoop/impl/v2/HadoopV2Context.java         |  11 +
 .../hadoop/impl/v2/HadoopV2MapTask.java         |  10 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |   4 +
 .../hadoop/shuffle/HadoopShuffle.java           |  23 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        | 389 ++++++++++++++-----
 .../shuffle/HadoopShuffleRemoteState.java       |   5 +-
 .../shuffle/direct/HadoopDirectDataInput.java   | 166 ++++++++
 .../shuffle/direct/HadoopDirectDataOutput.java  | 221 +++++++++++
 .../direct/HadoopDirectDataOutputContext.java   | 100 +++++
 .../direct/HadoopDirectDataOutputState.java     |  54 +++
 .../child/HadoopChildProcessRunner.java         |   2 +-
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |  22 +-
 18 files changed, 1287 insertions(+), 107 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 4ffb220..504e683 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -122,6 +122,7 @@ import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleAck;
 import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishRequest;
 import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleFinishResponse;
 import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopShuffleMessage;
+import org.apache.ignite.internal.processors.hadoop.shuffle.HadoopDirectShuffleMessage;
 import org.apache.ignite.internal.processors.igfs.IgfsAckMessage;
 import org.apache.ignite.internal.processors.igfs.IgfsBlockKey;
 import org.apache.ignite.internal.processors.igfs.IgfsBlocksMessage;
@@ -170,6 +171,11 @@ public class GridIoMessageFactory implements MessageFactory {
         Message msg = null;
 
         switch (type) {
+            case -42:
+                msg = new HadoopDirectShuffleMessage();
+
+                break;
+
             case -41:
                 msg = new HadoopShuffleFinishResponse();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
index e713caa..1f0ef1b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopJobProperty.java
@@ -91,6 +91,13 @@ public enum HadoopJobProperty {
     SHUFFLE_MSG_SIZE("ignite.shuffle.message.size"),
 
     /**
+     * Whether to stripe mapper output for remote reducers.
+     * <p>
+     * Defaults to {@code false}.
+     */
+    SHUFFLE_MAPPER_STRIPED_OUTPUT("ignite.shuffle.mapper.striped.output"),
+
+    /**
      * Shuffle job throttle in milliseconds. When job is executed with separate shuffle thread, this parameter
      * controls sleep duration between iterations through intermediate reducer maps.
      * <p>

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperAwareTaskOutput.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperAwareTaskOutput.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperAwareTaskOutput.java
new file mode 100644
index 0000000..1d6637c
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperAwareTaskOutput.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop;
+
+import org.apache.ignite.IgniteCheckedException;
+
+/**
+ * Special output type with callback invoked when mapper finished writing data.
+ */
+public interface HadoopMapperAwareTaskOutput extends HadoopTaskOutput {
+    /**
+     * Callback invoked when mapper finished writing data.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onMapperFinished() throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
index b76fb85..3509367 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopTaskInfo.java
@@ -46,6 +46,12 @@ public class HadoopTaskInfo implements Externalizable {
     /** */
     private HadoopInputSplit inputSplit;
 
+    /** Whether mapper index is set. */
+    private boolean mapperIdxSet;
+
+    /** Current mapper index. */
+    private int mapperIdx;
+
     /**
      * For {@link Externalizable}.
      */
@@ -78,6 +84,13 @@ public class HadoopTaskInfo implements Externalizable {
         out.writeInt(taskNum);
         out.writeInt(attempt);
         out.writeObject(inputSplit);
+
+        if (mapperIdxSet) {
+            out.writeBoolean(true);
+            out.writeInt(mapperIdx);
+        }
+        else
+            out.writeBoolean(false);
     }
 
     /** {@inheritDoc} */
@@ -87,6 +100,13 @@ public class HadoopTaskInfo implements Externalizable {
         taskNum = in.readInt();
         attempt = in.readInt();
         inputSplit = (HadoopInputSplit)in.readObject();
+
+        if (in.readBoolean()) {
+            mapperIdxSet = true;
+            mapperIdx = in.readInt();
+        }
+        else
+            mapperIdxSet = false;
     }
 
     /**
@@ -118,6 +138,29 @@ public class HadoopTaskInfo implements Externalizable {
     }
 
     /**
+     * @param mapperIdx Current mapper index.
+     */
+    public void mapperIndex(int mapperIdx) {
+        this.mapperIdx = mapperIdx;
+
+        mapperIdxSet = true;
+    }
+
+    /**
+     * @return Current mapper index or {@code null}
+     */
+    public int mapperIndex() {
+        return mapperIdx;
+    }
+
+    /**
+     * @return {@code True} if mapped index is set.
+     */
+    public boolean hasMapperIndex() {
+        return mapperIdxSet;
+    }
+
+    /**
      * @return Input split.
      */
     @Nullable public HadoopInputSplit inputSplit() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
new file mode 100644
index 0000000..e81dc5f
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopDirectShuffleMessage.java
@@ -0,0 +1,243 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.shuffle;
+
+import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
+import org.apache.ignite.internal.util.tostring.GridToStringInclude;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+import java.io.IOException;
+import java.io.ObjectInput;
+import java.io.ObjectOutput;
+import java.nio.ByteBuffer;
+
+/**
+ * Direct shuffle message.
+ */
+public class HadoopDirectShuffleMessage implements Message, HadoopMessage {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    @GridToStringInclude
+    private HadoopJobId jobId;
+
+    /** */
+    @GridToStringInclude
+    private int reducer;
+
+    /** Count. */
+    private int cnt;
+
+    /** Buffer. */
+    private byte[] buf;
+
+    /** Buffer length (equal or less than buf.length). */
+    @GridDirectTransient
+    private transient int bufLen;
+
+    /**
+     * Default constructor.
+     */
+    public HadoopDirectShuffleMessage() {
+        // No-op.
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param jobId Job ID.
+     * @param reducer Reducer.
+     * @param cnt Count.
+     * @param buf Buffer.
+     * @param bufLen Buffer length.
+     */
+    public HadoopDirectShuffleMessage(HadoopJobId jobId, int reducer, int cnt, byte[] buf, int bufLen) {
+        assert jobId != null;
+
+        this.jobId = jobId;
+        this.reducer = reducer;
+        this.cnt = cnt;
+        this.buf = buf;
+        this.bufLen = bufLen;
+    }
+
+    /**
+     * @return Job ID.
+     */
+    public HadoopJobId jobId() {
+        return jobId;
+    }
+
+    /**
+     * @return Reducer.
+     */
+    public int reducer() {
+        return reducer;
+    }
+
+    /**
+     * @return Count.
+     */
+    public int count() {
+        return cnt;
+    }
+
+    /**
+     * @return Buffer.
+     */
+    public byte[] buffer() {
+        return buf;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeMessage("jobId", jobId))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeInt("reducer", reducer))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeInt("cnt", cnt))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
+                if (!writer.writeByteArray("buf", this.buf, 0, bufLen))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                jobId = reader.readMessage("jobId");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                reducer = reader.readInt("reducer");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                cnt = reader.readInt("cnt");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
+                this.buf = reader.readByteArray("buf");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                bufLen = this.buf != null ? this.buf.length : 0;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(HadoopDirectShuffleMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -42;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeExternal(ObjectOutput out) throws IOException {
+        jobId.writeExternal(out);
+
+        out.writeInt(reducer);
+        out.writeInt(cnt);
+
+        U.writeByteArray(out, buf);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException {
+        jobId = new HadoopJobId();
+        jobId.readExternal(in);
+
+        reducer = in.readInt();
+        cnt = in.readInt();
+
+        buf = U.readByteArray(in);
+        bufLen = buf != null ? buf.length : 0;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(HadoopDirectShuffleMessage.class, this);
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperUtils.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperUtils.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperUtils.java
new file mode 100644
index 0000000..87adcb7
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/HadoopMapperUtils.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop;
+
+/**
+ * Set of mapper utility methods.
+ */
+public class HadoopMapperUtils {
+    /** Thread-local mapper index. */
+    private static final ThreadLocal<Integer> MAP_IDX = new ThreadLocal<>();
+
+    /**
+     * @return Current mapper index.
+     */
+    public static int mapperIndex() {
+        Integer res = MAP_IDX.get();
+
+        return res != null ? res : -1;
+    }
+
+    /**
+     * @param idx Current mapper index.
+     */
+    public static void mapperIndex(Integer idx) {
+        MAP_IDX.set(idx);
+    }
+
+    /**
+     * Clear mapper index.
+     */
+    public static void clearMapperIndex() {
+        MAP_IDX.remove();
+    }
+
+    /**
+     * Constructor.
+     */
+    private HadoopMapperUtils() {
+        // No-op.
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
index 90a1bad..eec0636 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2Context.java
@@ -31,6 +31,7 @@ import org.apache.hadoop.mapreduce.task.JobContextImpl;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopFileBlock;
 import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
+import org.apache.ignite.internal.processors.hadoop.HadoopMapperAwareTaskOutput;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskCancelledException;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
@@ -153,6 +154,16 @@ public class HadoopV2Context extends JobContextImpl implements MapContext, Reduc
         }
     }
 
+    /**
+     * Callback invoked from mapper thread when map is finished.
+     *
+     * @throws IgniteCheckedException If failed.
+     */
+    public void onMapperFinished() throws IgniteCheckedException {
+        if (output instanceof HadoopMapperAwareTaskOutput)
+            ((HadoopMapperAwareTaskOutput)output).onMapperFinished();
+    }
+
     /** {@inheritDoc} */
     @Override public OutputCommitter getOutputCommitter() {
         throw new UnsupportedOperationException();

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java
index 418df4e..eb3b935 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/impl/v2/HadoopV2MapTask.java
@@ -28,6 +28,7 @@ import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobInfo;
+import org.apache.ignite.internal.processors.hadoop.HadoopMapperUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
 
 /**
@@ -49,6 +50,11 @@ public class HadoopV2MapTask extends HadoopV2Task {
 
         JobContextImpl jobCtx = taskCtx.jobContext();
 
+        if (taskCtx.taskInfo().hasMapperIndex())
+            HadoopMapperUtils.mapperIndex(taskCtx.taskInfo().mapperIndex());
+        else
+            HadoopMapperUtils.clearMapperIndex();
+
         try {
             InputSplit nativeSplit = hadoopContext().getInputSplit();
 
@@ -72,6 +78,8 @@ public class HadoopV2MapTask extends HadoopV2Task {
 
             try {
                 mapper.run(new WrappedMapper().getMapContext(hadoopContext()));
+
+                hadoopContext().onMapperFinished();
             }
             finally {
                 closeWriter();
@@ -92,6 +100,8 @@ public class HadoopV2MapTask extends HadoopV2Task {
             throw new IgniteCheckedException(e);
         }
         finally {
+            HadoopMapperUtils.clearMapperIndex();
+
             if (err != null)
                 abort(outputFormat);
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index 36782bf..a725534 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -1018,6 +1018,8 @@ public class HadoopJobTracker extends HadoopComponent {
             if (state == null)
                 state = initState(jobId);
 
+            int mapperIdx = 0;
+
             for (HadoopInputSplit split : mappers) {
                 if (state.addMapper(split)) {
                     if (log.isDebugEnabled())
@@ -1026,6 +1028,8 @@ public class HadoopJobTracker extends HadoopComponent {
 
                     HadoopTaskInfo taskInfo = new HadoopTaskInfo(MAP, jobId, meta.taskNumber(split), 0, split);
 
+                    taskInfo.mapperIndex(mapperIdx++);
+
                     if (tasks == null)
                         tasks = new ArrayList<>();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
index 82bbd32..8ffea8c 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffle.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.processors.hadoop.HadoopComponent;
 import org.apache.ignite.internal.processors.hadoop.HadoopContext;
+import org.apache.ignite.internal.processors.hadoop.HadoopInputSplit;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
 import org.apache.ignite.internal.processors.hadoop.HadoopMapReducePlan;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
@@ -39,6 +40,7 @@ import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteBiPredicate;
 import org.apache.ignite.plugin.extensions.communication.Message;
 
+import java.util.Collection;
 import java.util.UUID;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -102,8 +104,8 @@ public class HadoopShuffle extends HadoopComponent {
     private HadoopShuffleJob<UUID> newJob(HadoopJobId jobId) throws IgniteCheckedException {
         HadoopMapReducePlan plan = ctx.jobTracker().plan(jobId);
 
-        HadoopShuffleJob<UUID> job = new HadoopShuffleJob<>(ctx.localNodeId(), log,
-            ctx.jobTracker().job(jobId, null), mem, plan.reducers(), plan.reducers(ctx.localNodeId()), true);
+        HadoopShuffleJob<UUID> job = new HadoopShuffleJob<>(ctx.localNodeId(), log, ctx.jobTracker().job(jobId, null),
+            mem, plan.reducers(), plan.reducers(ctx.localNodeId()), localMappersCount(plan), true);
 
         UUID[] rdcAddrs = new UUID[plan.reducers()];
 
@@ -123,6 +125,18 @@ public class HadoopShuffle extends HadoopComponent {
     }
 
     /**
+     * Get number of local mappers.
+     *
+     * @param plan Plan.
+     * @return Number of local mappers.
+     */
+    private int localMappersCount(HadoopMapReducePlan plan) {
+        Collection<HadoopInputSplit> locMappers = plan.mappers(ctx.localNodeId());
+
+        return F.isEmpty(locMappers) ? 0 : locMappers.size();
+    }
+
+    /**
      * @param nodeId Node ID to send message to.
      * @param msg Message to send.
      * @throws IgniteCheckedException If send failed.
@@ -195,6 +209,11 @@ public class HadoopShuffle extends HadoopComponent {
 
                 job(m.jobId()).onShuffleMessage(src, m);
             }
+            else if (msg instanceof HadoopDirectShuffleMessage) {
+                HadoopDirectShuffleMessage m = (HadoopDirectShuffleMessage)msg;
+
+                job(m.jobId()).onDirectShuffleMessage(src, m);
+            }
             else if (msg instanceof HadoopShuffleAck) {
                 HadoopShuffleAck m = (HadoopShuffleAck)msg;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
index 0a3a0ae..214a335 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleJob.java
@@ -17,20 +17,16 @@
 
 package org.apache.ignite.internal.processors.hadoop.shuffle;
 
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicReferenceArray;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.processors.hadoop.HadoopJob;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+import org.apache.ignite.internal.processors.hadoop.HadoopMapperAwareTaskOutput;
+import org.apache.ignite.internal.processors.hadoop.HadoopMapperUtils;
 import org.apache.ignite.internal.processors.hadoop.HadoopPartitioner;
+import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInfo;
 import org.apache.ignite.internal.processors.hadoop.HadoopTaskInput;
@@ -41,6 +37,9 @@ import org.apache.ignite.internal.processors.hadoop.message.HadoopMessage;
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopConcurrentHashMultimap;
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopMultimap;
 import org.apache.ignite.internal.processors.hadoop.shuffle.collections.HadoopSkipList;
+import org.apache.ignite.internal.processors.hadoop.shuffle.direct.HadoopDirectDataInput;
+import org.apache.ignite.internal.processors.hadoop.shuffle.direct.HadoopDirectDataOutputContext;
+import org.apache.ignite.internal.processors.hadoop.shuffle.direct.HadoopDirectDataOutputState;
 import org.apache.ignite.internal.util.GridUnsafe;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
@@ -55,9 +54,19 @@ import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteBiTuple;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.Nullable;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicReferenceArray;
 
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.PARTITION_HASHMAP_SIZE;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_JOB_THROTTLE;
+import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_MAPPER_STRIPED_OUTPUT;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_MSG_SIZE;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.SHUFFLE_REDUCER_NO_SORTING;
 import static org.apache.ignite.internal.processors.hadoop.HadoopJobProperty.get;
@@ -121,6 +130,9 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
     /** Message size. */
     private final int msgSize;
 
+    /** Whether to strip mappers for remote execution. */
+    private final boolean stripeMappers;
+
     /** Local shuffle states. */
     private volatile HashMap<T, HadoopShuffleLocalState> locShuffleStates = new HashMap<>();
 
@@ -143,11 +155,12 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
      * @param mem Memory.
      * @param totalReducerCnt Amount of reducers in the Job.
      * @param locReducers Reducers will work on current node.
+     * @param locMappersCnt Number of mappers running on the given node.
      * @param embedded Whether shuffle is running in embedded mode.
      * @throws IgniteCheckedException If error.
      */
     public HadoopShuffleJob(T locReduceAddr, IgniteLogger log, HadoopJob job, GridUnsafeMemory mem,
-        int totalReducerCnt, int[] locReducers, boolean embedded) throws IgniteCheckedException {
+        int totalReducerCnt, int[] locReducers, int locMappersCnt, boolean embedded) throws IgniteCheckedException {
         this.locReduceAddr = locReduceAddr;
         this.totalReducerCnt = totalReducerCnt;
         this.job = job;
@@ -155,6 +168,27 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
         this.log = log.getLogger(HadoopShuffleJob.class);
         this.embedded = embedded;
 
+        // No stripes for combiner.
+        boolean stripeMappers0 = get(job.info(), SHUFFLE_MAPPER_STRIPED_OUTPUT, false);
+
+        if (stripeMappers0) {
+            if (job.info().hasCombiner()) {
+                log.info("Striped mapper output is disabled because it cannot be used together with combiner [jobId=" +
+                    job.id() + ']');
+
+                stripeMappers0 = false;
+            }
+
+            if (!embedded) {
+                log.info("Striped mapper output is disabled becuase it cannot be used in external mode [jobId=" +
+                    job.id() + ']');
+
+                stripeMappers0 = false;
+            }
+        }
+
+        stripeMappers = stripeMappers0;
+
         msgSize = get(job.info(), SHUFFLE_MSG_SIZE, DFLT_SHUFFLE_MSG_SIZE);
 
         locReducersCtx = new AtomicReferenceArray<>(totalReducerCnt);
@@ -169,9 +203,20 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
 
         needPartitioner = totalReducerCnt > 1;
 
+        // Size of local map is always equal to total reducer number to allow index-based lookup.
         locMaps = new AtomicReferenceArray<>(totalReducerCnt);
-        rmtMaps = new AtomicReferenceArray<>(totalReducerCnt);
-        msgs = new HadoopShuffleMessage[totalReducerCnt];
+
+        // Size of remote map:
+        // - If there are no local mappers, then we will not send anything, so set to 0;
+        // - If output is not striped, then match it to total reducer count, the same way as for local maps.
+        // - If output is striped, then multiply previous value by number of local mappers.
+        int rmtMapsSize = locMappersCnt == 0 ? 0 : totalReducerCnt;
+
+        if (stripeMappers)
+            rmtMapsSize *= locMappersCnt;
+
+        rmtMaps = new AtomicReferenceArray<>(rmtMapsSize);
+        msgs = new HadoopShuffleMessage[rmtMapsSize];
 
         throttle = get(job.info(), SHUFFLE_JOB_THROTTLE, 0);
     }
@@ -208,24 +253,26 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
 
         this.io = io;
 
-        if (!flushed) {
-            snd = new GridWorker(gridName, "hadoop-shuffle-" + job.id(), log) {
-                @Override protected void body() throws InterruptedException {
-                    try {
-                        while (!isCancelled()) {
-                            if (throttle > 0)
-                                Thread.sleep(throttle);
-
-                            collectUpdatesAndSend(false);
+        if (!stripeMappers) {
+            if (!flushed) {
+                snd = new GridWorker(gridName, "hadoop-shuffle-" + job.id(), log) {
+                    @Override protected void body() throws InterruptedException {
+                        try {
+                            while (!isCancelled()) {
+                                if (throttle > 0)
+                                    Thread.sleep(throttle);
+
+                                collectUpdatesAndSend(false);
+                            }
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw new IllegalStateException(e);
                         }
                     }
-                    catch (IgniteCheckedException e) {
-                        throw new IllegalStateException(e);
-                    }
-                }
-            };
+                };
 
-            new IgniteThread(snd).start();
+                new IgniteThread(snd).start();
+            }
         }
 
         ioInitLatch.countDown();
@@ -306,6 +353,46 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
     }
 
     /**
+     * Process shuffle message.
+     *
+     * @param src Source.
+     * @param msg Message.
+     * @throws IgniteCheckedException Exception.
+     */
+    public void onDirectShuffleMessage(T src, HadoopDirectShuffleMessage msg) throws IgniteCheckedException {
+        assert msg.buffer() != null;
+
+        HadoopTaskContext taskCtx = locReducersCtx.get(msg.reducer()).get();
+
+        HadoopPerformanceCounter perfCntr = HadoopPerformanceCounter.getCounter(taskCtx.counters(), null);
+
+        perfCntr.onShuffleMessage(msg.reducer(), U.currentTimeMillis());
+
+        HadoopMultimap map = getOrCreateMap(locMaps, msg.reducer());
+
+        HadoopSerialization keySer = taskCtx.keySerialization();
+        HadoopSerialization valSer = taskCtx.valueSerialization();
+
+        // Add data from message to the map.
+        try (HadoopMultimap.Adder adder = map.startAdding(taskCtx)) {
+            HadoopDirectDataInput in = new HadoopDirectDataInput(msg.buffer());
+
+            Object key = null;
+            Object val = null;
+
+            for (int i = 0; i < msg.count(); i++) {
+                key = keySer.read(in, key);
+                val = valSer.read(in, val);
+
+                adder.write(key, val);
+            }
+        }
+
+        if (localShuffleState(src).onShuffleMessage())
+            sendFinishResponse(src, msg.jobId());
+    }
+
+    /**
      * @param ack Shuffle ack.
      */
     @SuppressWarnings("ConstantConditions")
@@ -467,88 +554,149 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
     }
 
     /**
-     * Sends map updates to remote reducers.
+     * Send updates to remote reducers.
+     *
+     * @param flush Flush flag.
+     * @throws IgniteCheckedException If failed.
      */
     private void collectUpdatesAndSend(boolean flush) throws IgniteCheckedException {
-        for (int i = 0; i < rmtMaps.length(); i++) {
-            HadoopMultimap map = rmtMaps.get(i);
+        for (int i = 0; i < rmtMaps.length(); i++)
+            collectUpdatesAndSend(i, flush);
+    }
+
+    /**
+     * Send updates to concrete remote reducer.
+     *
+     * @param rmtMapIdx Remote map index.
+     * @param flush Flush flag.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void collectUpdatesAndSend(int rmtMapIdx, boolean flush) throws IgniteCheckedException {
+        final int rmtRdcIdx = stripeMappers ? rmtMapIdx % totalReducerCnt : rmtMapIdx;
 
-            if (map == null)
-                continue; // Skip empty map and local node.
+        HadoopMultimap map = rmtMaps.get(rmtMapIdx);
 
-            if (msgs[i] == null)
-                msgs[i] = new HadoopShuffleMessage(job.id(), i, msgSize);
+        if (map == null)
+            return;
 
-            final int idx = i;
+        if (msgs[rmtMapIdx] == null)
+            msgs[rmtMapIdx] = new HadoopShuffleMessage(job.id(), rmtRdcIdx, msgSize);
 
-            map.visit(false, new HadoopMultimap.Visitor() {
-                /** */
-                private long keyPtr;
+        visit(map, rmtMapIdx, rmtRdcIdx);
 
-                /** */
-                private int keySize;
+        if (flush && msgs[rmtMapIdx].offset() != 0)
+            send(rmtMapIdx, rmtRdcIdx, 0);
+    }
 
-                /** */
-                private boolean keyAdded;
+    /**
+     * Flush remote direct context.
+     *
+     * @param rmtMapIdx Remote map index.
+     * @param rmtDirectCtx Remote direct context.
+     * @param reset Whether to perform reset.
+     */
+    private void sendShuffleMessage(int rmtMapIdx, @Nullable HadoopDirectDataOutputContext rmtDirectCtx, boolean reset) {
+        if (rmtDirectCtx == null)
+            return;
 
-                /** {@inheritDoc} */
-                @Override public void onKey(long keyPtr, int keySize) {
-                    this.keyPtr = keyPtr;
-                    this.keySize = keySize;
+        int cnt = rmtDirectCtx.count();
 
-                    keyAdded = false;
-                }
+        if (cnt == 0)
+            return;
 
-                private boolean tryAdd(long valPtr, int valSize) {
-                    HadoopShuffleMessage msg = msgs[idx];
+        int rmtRdcIdx = stripeMappers ? rmtMapIdx % totalReducerCnt : rmtMapIdx;
 
-                    if (!keyAdded) { // Add key and value.
-                        int size = keySize + valSize;
+        HadoopDirectDataOutputState state = rmtDirectCtx.state();
 
-                        if (!msg.available(size, false))
-                            return false;
+        if (reset)
+            rmtDirectCtx.reset();
 
-                        msg.addKey(keyPtr, keySize);
-                        msg.addValue(valPtr, valSize);
+        HadoopDirectShuffleMessage msg = new HadoopDirectShuffleMessage(job.id(), rmtRdcIdx, cnt,
+            state.buffer(), state.bufferLength());
 
-                        keyAdded = true;
+        T nodeId = reduceAddrs[rmtRdcIdx];
 
-                        return true;
-                    }
+        io.apply(nodeId, msg);
+
+        remoteShuffleState(nodeId).onShuffleMessage();
+    }
+
+    /**
+     * Visit output map.
+     *
+     * @param map Map.
+     * @param rmtMapIdx Remote map index.
+     * @param rmtRdcIdx Remote reducer index.
+     * @throws IgniteCheckedException If failed.
+     */
+    private void visit(HadoopMultimap map, final int rmtMapIdx, final int rmtRdcIdx) throws IgniteCheckedException {
+        map.visit(false, new HadoopMultimap.Visitor() {
+            /** */
+            private long keyPtr;
 
-                    if (!msg.available(valSize, true))
+            /** */
+            private int keySize;
+
+            /** */
+            private boolean keyAdded;
+
+            /** {@inheritDoc} */
+            @Override public void onKey(long keyPtr, int keySize) {
+                this.keyPtr = keyPtr;
+                this.keySize = keySize;
+
+                keyAdded = false;
+            }
+
+            private boolean tryAdd(long valPtr, int valSize) {
+                HadoopShuffleMessage msg = msgs[rmtMapIdx];
+
+                if (!keyAdded) { // Add key and value.
+                    int size = keySize + valSize;
+
+                    if (!msg.available(size, false))
                         return false;
 
+                    msg.addKey(keyPtr, keySize);
                     msg.addValue(valPtr, valSize);
 
+                    keyAdded = true;
+
                     return true;
                 }
 
-                /** {@inheritDoc} */
-                @Override public void onValue(long valPtr, int valSize) {
-                    if (tryAdd(valPtr, valSize))
-                        return;
+                if (!msg.available(valSize, true))
+                    return false;
 
-                    send(idx, keySize + valSize);
+                msg.addValue(valPtr, valSize);
 
-                    keyAdded = false;
+                return true;
+            }
 
-                    if (!tryAdd(valPtr, valSize))
-                        throw new IllegalStateException();
-                }
-            });
+            /** {@inheritDoc} */
+            @Override public void onValue(long valPtr, int valSize) {
+                if (tryAdd(valPtr, valSize))
+                    return;
 
-            if (flush && msgs[i].offset() != 0)
-                send(i, 0);
-        }
+                send(rmtMapIdx, rmtRdcIdx, keySize + valSize);
+
+                keyAdded = false;
+
+                if (!tryAdd(valPtr, valSize))
+                    throw new IllegalStateException();
+            }
+        });
     }
 
     /**
-     * @param idx Index of message.
+     * Send message.
+     *
+     * @param rmtMapIdx Remote map index.
+     * @param rmtRdcIdx Remote reducer index.
      * @param newBufMinSize Min new buffer size.
      */
-    private void send(final int idx, int newBufMinSize) {
-        HadoopShuffleMessage msg = msgs[idx];
+    private void send(int rmtMapIdx, int rmtRdcIdx, int newBufMinSize) {
+        HadoopShuffleMessage msg = msgs[rmtMapIdx];
 
         final long msgId = msg.id();
 
@@ -566,10 +714,10 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
         }
 
         try {
-            io.apply(reduceAddrs[idx], msg);
+            io.apply(reduceAddrs[rmtRdcIdx], msg);
 
             if (embedded)
-                remoteShuffleState(reduceAddrs[idx]).onShuffleMessage();
+                remoteShuffleState(reduceAddrs[rmtRdcIdx]).onShuffleMessage();
         }
         catch (GridClosureException e) {
             if (fut != null)
@@ -593,7 +741,7 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
             });
         }
 
-        msgs[idx] = newBufMinSize == 0 ? null : new HadoopShuffleMessage(job.id(), idx,
+        msgs[rmtMapIdx] = newBufMinSize == 0 ? null : new HadoopShuffleMessage(job.id(), rmtRdcIdx,
             Math.max(msgSize, newBufMinSize));
     }
 
@@ -639,31 +787,33 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
         if (totalReducerCnt == 0)
             return new GridFinishedFuture<>();
 
-        U.await(ioInitLatch);
+        if (!stripeMappers) {
+            U.await(ioInitLatch);
 
-        GridWorker snd0 = snd;
+            GridWorker snd0 = snd;
 
-        if (snd0 != null) {
-            if (log.isDebugEnabled())
-                log.debug("Cancelling sender thread.");
+            if (snd0 != null) {
+                if (log.isDebugEnabled())
+                    log.debug("Cancelling sender thread.");
 
-            snd0.cancel();
+                snd0.cancel();
 
-            try {
-                snd0.join();
+                try {
+                    snd0.join();
 
-                if (log.isDebugEnabled())
-                    log.debug("Finished waiting for sending thread to complete on shuffle job flush: " + job.id());
-            }
-            catch (InterruptedException e) {
-                throw new IgniteInterruptedCheckedException(e);
+                    if (log.isDebugEnabled())
+                        log.debug("Finished waiting for sending thread to complete on shuffle job flush: " + job.id());
+                }
+                catch (InterruptedException e) {
+                    throw new IgniteInterruptedCheckedException(e);
+                }
             }
-        }
 
-        collectUpdatesAndSend(true); // With flush.
+            collectUpdatesAndSend(true); // With flush.
 
-        if (log.isDebugEnabled())
-            log.debug("Finished sending collected updates to remote reducers: " + job.id());
+            if (log.isDebugEnabled())
+                log.debug("Finished sending collected updates to remote reducers: " + job.id());
+        }
 
         GridCompoundFuture fut = new GridCompoundFuture<>();
 
@@ -700,8 +850,8 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
 
             if (log.isDebugEnabled())
                 log.debug("Collected futures to compound futures for flush: " + sentMsgs.size());
-
         }
+
         return fut;
     }
 
@@ -775,13 +925,17 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
     /**
      * Partitioned output.
      */
-    private class PartitionedOutput implements HadoopTaskOutput {
+    public class PartitionedOutput implements HadoopMapperAwareTaskOutput {
         /** */
         private final HadoopTaskOutput[] locAdders = new HadoopTaskOutput[locMaps.length()];
 
         /** */
         private final HadoopTaskOutput[] rmtAdders = new HadoopTaskOutput[rmtMaps.length()];
 
+        /** Remote direct contexts. */
+        private final HadoopDirectDataOutputContext[] rmtDirectCtxs =
+            new HadoopDirectDataOutputContext[rmtMaps.length()];
+
         /** */
         private HadoopPartitioner partitioner;
 
@@ -819,16 +973,53 @@ public class HadoopShuffleJob<T> implements AutoCloseable {
                     locAdders[part] = out = getOrCreateMap(locMaps, part).startAdding(taskCtx);
             }
             else {
-                out = rmtAdders[part];
+                if (stripeMappers) {
+                    int mapperIdx = HadoopMapperUtils.mapperIndex();
 
-                if (out == null)
-                    rmtAdders[part] = out = getOrCreateMap(rmtMaps, part).startAdding(taskCtx);
+                    assert mapperIdx >= 0;
+
+                    int idx = totalReducerCnt * mapperIdx + part;
+
+                    HadoopDirectDataOutputContext rmtDirectCtx = rmtDirectCtxs[idx];
+
+                    if (rmtDirectCtx == null) {
+                        rmtDirectCtx = new HadoopDirectDataOutputContext(msgSize, taskCtx);
+
+                        rmtDirectCtxs[idx] = rmtDirectCtx;
+                    }
+
+                    if (rmtDirectCtx.write(key, val))
+                        sendShuffleMessage(idx, rmtDirectCtx, true);
+
+                    return;
+                }
+                else {
+                    out = rmtAdders[part];
+
+                    if (out == null)
+                        rmtAdders[part] = out = getOrCreateMap(rmtMaps, part).startAdding(taskCtx);
+                }
             }
 
             out.write(key, val);
         }
 
         /** {@inheritDoc} */
+        @Override public void onMapperFinished() throws IgniteCheckedException {
+            if (stripeMappers) {
+                int mapperIdx = HadoopMapperUtils.mapperIndex();
+
+                assert mapperIdx >= 0;
+
+                for (int i = 0; i < totalReducerCnt; i++) {
+                    int idx = totalReducerCnt * mapperIdx + i;
+
+                    sendShuffleMessage(idx, rmtDirectCtxs[idx], false);
+                }
+            }
+        }
+
+        /** {@inheritDoc} */
         @Override public void close() throws IgniteCheckedException {
             for (HadoopTaskOutput adder : locAdders) {
                 if (adder != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java
index 5ffaa55..4331124 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/HadoopShuffleRemoteState.java
@@ -17,17 +17,14 @@
 
 package org.apache.ignite.internal.processors.hadoop.shuffle;
 
-import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
-import org.apache.ignite.lang.IgniteInClosure;
 
-import java.util.UUID;
 import java.util.concurrent.atomic.AtomicLong;
 
 /**
  * Remote shuffle state.
  */
-class HadoopShuffleRemoteState<T> {
+class HadoopShuffleRemoteState {
     /** Message count. */
     private final AtomicLong msgCnt = new AtomicLong();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
new file mode 100644
index 0000000..e3a713a
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataInput.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.shuffle.direct;
+
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.DataInput;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+
+import static org.apache.ignite.internal.util.GridUnsafe.BYTE_ARR_OFF;
+
+/**
+ * Hadoop data input used for direct communication.
+ */
+public class HadoopDirectDataInput extends InputStream implements DataInput {
+    /** Data buffer. */
+    private final byte[] buf;
+
+    /** Position. */
+    private int pos;
+
+    /**
+     * Constructor.
+     *
+     * @param buf Buffer.
+     */
+    public HadoopDirectDataInput(byte[] buf) {
+        this.buf = buf;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int read() throws IOException {
+        return readByte();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readFully(@NotNull byte[] b) throws IOException {
+        readFully(b, 0, b.length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void readFully(@NotNull byte[] b, int off, int len) throws IOException {
+        System.arraycopy(buf, pos, b, off, len);
+
+        pos += len;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int skipBytes(int n) throws IOException {
+        pos += n;
+
+        return n;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readBoolean() throws IOException {
+        return readByte() == 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte readByte() throws IOException {
+        byte res = GridUnsafe.getByte(buf, BYTE_ARR_OFF + pos);
+
+        pos += 1;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readUnsignedByte() throws IOException {
+        return readByte() & 0xff;
+    }
+
+    /** {@inheritDoc} */
+    @Override public short readShort() throws IOException {
+        short res = GridUnsafe.getShort(buf, BYTE_ARR_OFF + pos);
+
+        pos += 2;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readUnsignedShort() throws IOException {
+        return readShort() & 0xffff;
+    }
+
+    /** {@inheritDoc} */
+    @Override public char readChar() throws IOException {
+        char res = GridUnsafe.getChar(buf, BYTE_ARR_OFF + pos);
+
+        pos += 2;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public int readInt() throws IOException {
+        int res = GridUnsafe.getInt(buf, BYTE_ARR_OFF + pos);
+
+        pos += 4;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long readLong() throws IOException {
+        long res = GridUnsafe.getLong(buf, BYTE_ARR_OFF + pos);
+
+        pos += 8;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public float readFloat() throws IOException {
+        float res = GridUnsafe.getFloat(buf, BYTE_ARR_OFF + pos);
+
+        pos += 4;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public double readDouble() throws IOException {
+        double res = GridUnsafe.getDouble(buf, BYTE_ARR_OFF + pos);
+
+        pos += 8;
+
+        return res;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String readLine() throws IOException {
+        // TODO: Create ticket!
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @NotNull @Override public String readUTF() throws IOException {
+        byte[] bytes = new byte[readShort()];
+
+        if (bytes.length != 0)
+            readFully(bytes);
+
+        return new String(bytes, StandardCharsets.UTF_8);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java
new file mode 100644
index 0000000..151e552
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutput.java
@@ -0,0 +1,221 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.shuffle.direct;
+
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.jetbrains.annotations.NotNull;
+
+import java.io.DataOutput;
+import java.io.IOException;
+import java.io.OutputStream;
+import java.io.UTFDataFormatException;
+import java.nio.charset.StandardCharsets;
+
+import static org.apache.ignite.internal.util.GridUnsafe.BYTE_ARR_OFF;
+
+/**
+ * Hadoop data output for direct communication.
+ */
+public class HadoopDirectDataOutput extends OutputStream implements DataOutput {
+    /** Flush size. */
+    private final int flushSize;
+
+    /** Data buffer. */
+    private byte[] buf;
+
+    /** Buffer size. */
+    private int bufSize;
+
+    /** Position. */
+    private int pos;
+
+    /**
+     * Constructor.
+     *
+     * @param flushSize Flush size.
+     */
+    public HadoopDirectDataOutput(int flushSize) {
+        this(flushSize, flushSize);
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param flushSize Flush size.
+     * @param allocSize Allocation size.
+     */
+    public HadoopDirectDataOutput(int flushSize, int allocSize) {
+        this.flushSize = flushSize;
+
+        buf = new byte[allocSize];
+        bufSize = allocSize;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(@NotNull byte[] b) throws IOException {
+        write(b, 0, b.length);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(@NotNull byte[] b, int off, int len) throws IOException {
+        int writePos = ensure(len);
+
+        System.arraycopy(b, off, buf, writePos, len);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void write(int val) throws IOException {
+        writeByte(val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBoolean(boolean val) throws IOException {
+        writeByte(val ? (byte)1 : (byte)0);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeByte(int val) throws IOException {
+        int writePos = ensure(1);
+
+        buf[writePos] = (byte)val;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeShort(int val) throws IOException {
+        int writePos = ensure(2);
+
+        GridUnsafe.putShort(buf, BYTE_ARR_OFF + writePos, (short)val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChar(int val) throws IOException {
+        int writePos = ensure(2);
+
+        GridUnsafe.putChar(buf, BYTE_ARR_OFF + writePos, (char)val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeInt(int val) throws IOException {
+        int writePos = ensure(4);
+
+        GridUnsafe.putInt(buf, BYTE_ARR_OFF + writePos, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeLong(long val) throws IOException {
+        int writePos = ensure(8);
+
+        GridUnsafe.putLong(buf, BYTE_ARR_OFF + writePos, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeFloat(float val) throws IOException {
+        int writePos = ensure(4);
+
+        GridUnsafe.putFloat(buf, BYTE_ARR_OFF + writePos, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeDouble(double val) throws IOException {
+        int writePos = ensure(8);
+
+        GridUnsafe.putDouble(buf, BYTE_ARR_OFF + writePos, val);
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeBytes(@NotNull String str) throws IOException {
+        for(int i = 0; i < str.length(); ++i)
+            write((byte)str.charAt(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeChars(@NotNull String str) throws IOException {
+        for (int i = 0; i < str.length(); ++i)
+            writeChar(str.charAt(i));
+    }
+
+    /** {@inheritDoc} */
+    @Override public void writeUTF(@NotNull String str) throws IOException {
+        byte[] bytes = str.getBytes(StandardCharsets.UTF_8);
+
+        int len = bytes.length;
+
+        if (len > 65535)
+            throw new UTFDataFormatException("UTF8 form of string is longer than 65535 bytes: " + str);
+
+        writeShort((short)len);
+        write(bytes);
+    }
+
+    /**
+     * @return Buffer.
+     */
+    public byte[] buffer() {
+        return buf;
+    }
+
+    /**
+     * @return Position.
+     */
+    public int position() {
+        return pos;
+    }
+
+    /**
+     * @return Whether buffer is ready for flush.
+     */
+    public boolean readyForFlush() {
+        return pos >= flushSize;
+    }
+
+    /**
+     * Ensure that the given amount of bytes is available within the stream, then shift the position.
+     *
+     * @param cnt Count.
+     * @return Position
+     */
+    private int ensure(int cnt) {
+        int pos0 = pos;
+
+        if (pos0 + cnt > bufSize)
+            grow(pos0 + cnt);
+
+        pos += cnt;
+
+        return pos0;
+    }
+
+    /**
+     * Grow array up to the given count.
+     *
+     * @param cnt Count.
+     */
+    private void grow(int cnt) {
+        int bufSize0 = (int)(bufSize * 1.1);
+
+        if (bufSize0 < cnt)
+            bufSize0 = cnt;
+
+        byte[] buf0 = new byte[bufSize0];
+
+        System.arraycopy(buf, 0, buf0, 0, pos);
+
+        buf = buf0;
+        bufSize = bufSize0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
new file mode 100644
index 0000000..bc70ef3
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputContext.java
@@ -0,0 +1,100 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.shuffle.direct;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.hadoop.HadoopSerialization;
+import org.apache.ignite.internal.processors.hadoop.HadoopTaskContext;
+
+/**
+ * Hadoop data output context for direct communication.
+ */
+public class HadoopDirectDataOutputContext {
+    /** Flush size. */
+    private final int flushSize;
+
+    /** Key serialization. */
+    private final HadoopSerialization keySer;
+
+    /** Value serialization. */
+    private final HadoopSerialization valSer;
+
+    /** Data output. */
+    private HadoopDirectDataOutput out;
+
+    /** Number of keys written. */
+    private int cnt;
+
+    /**
+     * Constructor.
+     *
+     * @param flushSize Flush size.
+     * @param taskCtx Task context.
+     * @throws IgniteCheckedException If failed.
+     */
+    public HadoopDirectDataOutputContext(int flushSize, HadoopTaskContext taskCtx)
+        throws IgniteCheckedException {
+        this.flushSize = flushSize;
+
+        keySer = taskCtx.keySerialization();
+        valSer = taskCtx.valueSerialization();
+
+        out = new HadoopDirectDataOutput(flushSize);
+    }
+
+    /**
+     * Write key-value pair.
+     *
+     * @param key Key.
+     * @param val Value.
+     * @return Whether flush is needed.
+     * @throws IgniteCheckedException If failed.
+     */
+    public boolean write(Object key, Object val) throws IgniteCheckedException {
+        keySer.write(out, key);
+        valSer.write(out, val);
+
+        cnt++;
+
+        return out.readyForFlush();
+    }
+
+    /**
+     * @return Key-value pairs count.
+     */
+    public int count() {
+        return cnt;
+    }
+
+    /**
+     * @return State.
+     */
+    public HadoopDirectDataOutputState state() {
+        return new HadoopDirectDataOutputState(out.buffer(), out.position());
+    }
+
+    /**
+     * Reset buffer.
+     */
+    public void reset() {
+        int allocSize = Math.max(flushSize, out.position());
+
+        out = new HadoopDirectDataOutput(flushSize, allocSize);
+        cnt = 0;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
new file mode 100644
index 0000000..a9c12e3
--- /dev/null
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/direct/HadoopDirectDataOutputState.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.hadoop.shuffle.direct;
+
+/**
+ * Hadoop data output state for direct communication.
+ */
+public class HadoopDirectDataOutputState {
+    /** Buffer. */
+    private final byte[] buf;
+
+    /** Buffer length. */
+    private final int bufLen;
+
+    /**
+     * Constructor.
+     *
+     * @param buf Buffer.
+     * @param bufLen Buffer length.
+     */
+    public HadoopDirectDataOutputState(byte[] buf, int bufLen) {
+        this.buf = buf;
+        this.bufLen = bufLen;
+    }
+
+    /**
+     * @return Buffer.
+     */
+    public byte[] buffer() {
+        return buf;
+    }
+
+    /**
+     * @return Length.
+     */
+    public int bufferLength() {
+        return bufLen;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
index cb08c00..3336120 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/HadoopChildProcessRunner.java
@@ -151,7 +151,7 @@ public class HadoopChildProcessRunner {
                 job.initialize(true, nodeDesc.processId());
 
                 shuffleJob = new HadoopShuffleJob<>(comm.localProcessDescriptor(), log, job, mem,
-                    req.totalReducerCount(), req.localReducers(), false);
+                    req.totalReducerCount(), req.localReducers(), 0, false);
 
                 initializeExecutors(req);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/065ca4a7/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java
index b04deeb..8897a38 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/impl/HadoopMapReduceEmbeddedSelfTest.java
@@ -34,6 +34,7 @@ import org.apache.ignite.configuration.HadoopConfiguration;
 import org.apache.ignite.igfs.IgfsPath;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.hadoop.HadoopJobId;
+import org.apache.ignite.internal.processors.hadoop.HadoopJobProperty;
 import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount1;
 import org.apache.ignite.internal.processors.hadoop.impl.examples.HadoopWordCount2;
 
@@ -54,12 +55,28 @@ public class HadoopMapReduceEmbeddedSelfTest extends HadoopMapReduceTest {
         return cfg;
     }
 
+    /*
+     * @throws Exception If fails.
+     */
+    public void testMultiReducerWholeMapReduceExecution() throws Exception {
+        checkMultiReducerWholeMapReduceExecution(false);
+    }
+
+    /*
+     * @throws Exception If fails.
+     */
+    public void testMultiReducerWholeMapReduceExecutionStriped() throws Exception {
+        checkMultiReducerWholeMapReduceExecution(true);
+    }
+
     /**
      * Tests whole job execution with all phases in old and new versions of API with definition of custom
      * Serialization, Partitioner and IO formats.
+     *
+     * @param striped Whether output should be striped or not.
      * @throws Exception If fails.
      */
-    public void testMultiReducerWholeMapReduceExecution() throws Exception {
+    public void checkMultiReducerWholeMapReduceExecution(boolean striped) throws Exception {
         IgfsPath inDir = new IgfsPath(PATH_INPUT);
 
         igfs.mkdirs(inDir);
@@ -81,6 +98,9 @@ public class HadoopMapReduceEmbeddedSelfTest extends HadoopMapReduceTest {
 
             JobConf jobConf = new JobConf();
 
+            if (striped)
+                jobConf.set(HadoopJobProperty.SHUFFLE_MAPPER_STRIPED_OUTPUT.propertyName(), "true");
+
             jobConf.set(CommonConfigurationKeys.IO_SERIALIZATIONS_KEY, CustomSerialization.class.getName());
 
             //To split into about 6-7 items for v2


[16/25] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NIO sessio

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/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 482e2ef..c7a1a53 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
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.spi.communication.tcp;
 
+import org.apache.ignite.internal.util.nio.GridNioServer;
 import org.apache.ignite.mxbean.MXBeanDescription;
 import org.apache.ignite.spi.IgniteSpiManagementMBean;
 
@@ -44,6 +45,35 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
     public int getLocalPort();
 
     /**
+     * Returns {@code true} if {@code TcpCommunicationSpi} should
+     * maintain connection for outgoing and incoming messages separately.
+     * In this case total number of connections between local and some remote node
+     * is {@link #getConnectionsPerNode()} * 2.
+     * <p>
+     * Returns {@code false} if each connection of {@link #getConnectionsPerNode()}
+     * should be used for outgoing and incoming messages. In this case load NIO selectors load
+     * balancing of {@link GridNioServer} will be disabled.
+     * <p>
+     * Default is {@code true}.
+     *
+     * @return {@code true} to use paired connections and {@code false} otherwise.
+     * @see #getConnectionsPerNode()
+     */
+    @MXBeanDescription("Paired connections used.")
+    public boolean isUsePairedConnections();
+
+    /**
+     * Gets number of connections to each remote node. if {@link #isUsePairedConnections()}
+     * is {@code true} then number of connections is doubled and half is used for incoming and
+     * half for outgoing messages.
+     *
+     * @return Number of connections per node.
+     * @see #isUsePairedConnections()
+     */
+    @MXBeanDescription("Connections per node.")
+    public int getConnectionsPerNode();
+
+    /**
      * Gets local port for shared memory communication.
      *
      * @return Port number.
@@ -153,6 +183,16 @@ public interface TcpCommunicationSpiMBean extends IgniteSpiManagementMBean {
     public int getReconnectCount();
 
     /**
+     * Defines how many non-blocking {@code selector.selectNow()} should be made before
+     * falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+     * Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+     *
+     * @return Selector thread busy-loop iterations.
+     */
+    @MXBeanDescription("Selector thread busy-loop iterations.")
+    public long getSelectorSpins();
+
+    /**
      * Gets value for {@code TCP_NODELAY} socket option.
      *
      * @return {@code True} if TCP delay is disabled.

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 204b685..50fa3bd 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -3424,7 +3424,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     if (log.isDebugEnabled())
                         log.debug("Node validation failed [res=" + err + ", node=" + node + ']');
 
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 boolean ping = node.id().equals(err.nodeId()) ? pingNode(node) : pingNode(err.nodeId());
@@ -3469,7 +3469,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 final String rmtMarsh = node.attribute(ATTR_MARSHALLER);
 
                 if (!F.eq(locMarsh, rmtMarsh)) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Local node's marshaller differs from remote node's marshaller " +
@@ -3526,7 +3526,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 boolean locLateAssignBool = locLateAssign != null ? locLateAssign : false;
 
                 if (locMarshUseDfltSuidBool != rmtMarshUseDfltSuidBool) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Local node's " + IGNITE_OPTIMIZED_MARSHALLER_USE_DEFAULT_SUID +
@@ -3568,7 +3568,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 final boolean rmtMarshCompactFooterBool = rmtMarshCompactFooter != null ? rmtMarshCompactFooter : false;
 
                 if (locMarshCompactFooterBool != rmtMarshCompactFooterBool) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Local node's binary marshaller \"compactFooter\" property differs from " +
@@ -3606,7 +3606,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 final boolean rmtMarshStrSerialVer2Bool = rmtMarshStrSerialVer2 != null ? rmtMarshStrSerialVer2 : false;
 
                 if (locMarshStrSerialVer2Bool != rmtMarshStrSerialVer2Bool) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Local node's " + IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2 +
@@ -3679,7 +3679,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     final Boolean rmtSrvcCompatibilityEnabled = node.attribute(ATTR_SERVICES_COMPATIBILITY_MODE);
 
                     if (!F.eq(locSrvcCompatibilityEnabled, rmtSrvcCompatibilityEnabled)) {
-                        utilityPool.submit(
+                        utilityPool.execute(
                             new Runnable() {
                                 @Override public void run() {
                                     String errMsg = "Local node's " + IGNITE_SERVICES_COMPATIBILITY_MODE +
@@ -3714,7 +3714,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     }
                 }
                 else if (Boolean.FALSE.equals(locSrvcCompatibilityEnabled)) {
-                    utilityPool.submit(
+                    utilityPool.execute(
                         new Runnable() {
                             @Override public void run() {
                                 String errMsg = "Remote node doesn't support lazy services configuration and " +

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
index d1c8d19..127778b 100644
--- a/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/stream/socket/SocketStreamer.java
@@ -184,6 +184,7 @@ public class SocketStreamer<T, K, V> extends StreamAdapter<T, K, V> {
         try {
             srv = new GridNioServer.Builder<byte[]>()
                 .address(addr == null ? InetAddress.getLocalHost() : addr)
+                .serverName("sock-streamer")
                 .port(port)
                 .listener(lsnr)
                 .logger(log)

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
index 55557dd..d173594 100644
--- a/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/thread/IgniteThreadFactory.java
@@ -20,6 +20,7 @@ package org.apache.ignite.thread;
 import java.util.concurrent.ThreadFactory;
 import java.util.concurrent.atomic.AtomicInteger;
 
+import org.apache.ignite.internal.util.typedef.internal.S;
 import org.jetbrains.annotations.NotNull;
 
 /**
@@ -62,4 +63,9 @@ public class IgniteThreadFactory implements ThreadFactory {
     @Override public Thread newThread(@NotNull Runnable r) {
         return new IgniteThread(gridName, threadName, r, idxGen.incrementAndGet());
     }
-}
\ No newline at end of file
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgniteThreadFactory.class, this, super.toString());
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
index 760313b..5721887 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteSlowClientDetectionSelfTest.java
@@ -75,6 +75,7 @@ public class IgniteSlowClientDetectionSelfTest extends GridCommonAbstractTest {
         commSpi.setSlowClientQueueLimit(50);
         commSpi.setSharedMemoryPort(-1);
         commSpi.setIdleConnectionTimeout(300_000);
+        commSpi.setConnectionsPerNode(1);
 
         cfg.setCommunicationSpi(commSpi);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
new file mode 100644
index 0000000..e95b1ec
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceMultipleConnectionsTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+/**
+ *
+ */
+public class IgniteCommunicationBalanceMultipleConnectionsTest extends IgniteCommunicationBalanceTest {
+    /** {@inheritDoc} */
+    @Override protected int connectionsPerNode() {
+        return 5;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
new file mode 100644
index 0000000..e142aef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteCommunicationBalanceTest.java
@@ -0,0 +1,339 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.IgniteCompute;
+import org.apache.ignite.IgniteSystemProperties;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.lang.GridAbsPredicate;
+import org.apache.ignite.internal.util.lang.GridAbsPredicateX;
+import org.apache.ignite.internal.util.nio.GridNioServer;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteCallable;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteCommunicationBalanceTest extends GridCommonAbstractTest {
+    /** */
+    private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** */
+    private int selectors;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpCommunicationSpi commSpi = ((TcpCommunicationSpi)cfg.getCommunicationSpi());
+
+        commSpi.setSharedMemoryPort(-1);
+        commSpi.setConnectionsPerNode(connectionsPerNode());
+
+        if (selectors > 0)
+            commSpi.setSelectorsCount(selectors);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(IP_FINDER);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /**
+     * @return Connections per node.
+     */
+    protected int connectionsPerNode() {
+        return 1;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTest() throws Exception {
+        stopAllGrids();
+
+        super.afterTest();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBalance1() throws Exception {
+        System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "5000");
+
+        try {
+            selectors = 4;
+
+            final int SRVS = 4;
+
+            startGridsMultiThreaded(SRVS);
+
+            client = true;
+
+            final Ignite client = startGrid(SRVS);
+
+            for (int i = 0; i < 4; i++) {
+                ClusterNode node = client.cluster().node(ignite(i).cluster().localNode().id());
+
+                client.compute(client.cluster().forNode(node)).call(new DummyCallable(null));
+            }
+
+            waitNioBalanceStop(Collections.singletonList(client), 10_000);
+
+            final GridNioServer srv = GridTestUtils.getFieldValue(client.configuration().getCommunicationSpi(), "nioSrvr");
+
+            ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+            long readMoveCnt1 = srv.readerMoveCount();
+            long writeMoveCnt1 = srv.writerMoveCount();
+
+            int prevNodeIdx = -1;
+
+            for (int iter = 0; iter < 10; iter++) {
+                int nodeIdx = rnd.nextInt(SRVS);
+
+                while (prevNodeIdx == nodeIdx)
+                    nodeIdx = rnd.nextInt(SRVS);
+
+                prevNodeIdx = nodeIdx;
+
+                log.info("Iteration [iter=" + iter + ", node=" + nodeIdx + ']');
+
+                final long readMoveCnt = readMoveCnt1;
+                final long writeMoveCnt = writeMoveCnt1;
+
+                final int nodeIdx0 = nodeIdx;
+
+                GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                    @Override public boolean apply() {
+                        byte[] data = new byte[100_000];
+
+                        for (int j = 0; j < 10; j++) {
+                            for (int i = 0; i < SRVS; i++) {
+                                ClusterNode node = client.cluster().node(ignite(i).cluster().localNode().id());
+
+                                IgniteCompute compute = client.compute(client.cluster().forNode(node));
+
+                                compute.call(new DummyCallable(i == nodeIdx0 ? data : null));
+                            }
+                        }
+
+                        return srv.readerMoveCount() > readMoveCnt && srv.writerMoveCount() > writeMoveCnt;
+                    }
+                }, 30_000);
+
+                waitNioBalanceStop(Collections.singletonList(client), 30_000);
+
+                long readMoveCnt2 = srv.readerMoveCount();
+                long writeMoveCnt2 = srv.writerMoveCount();
+
+                log.info("Move counts [rc1=" + readMoveCnt1 +
+                    ", wc1=" + writeMoveCnt1 +
+                    ", rc2=" + readMoveCnt2 +
+                    ", wc2=" + writeMoveCnt2 + ']');
+
+                assertTrue(readMoveCnt2 > readMoveCnt1);
+                assertTrue(writeMoveCnt2 > writeMoveCnt1);
+
+                readMoveCnt1 = readMoveCnt2;
+                writeMoveCnt1 = writeMoveCnt2;
+            }
+
+            waitNioBalanceStop(G.allGrids(), 10_000);
+        }
+        finally {
+            System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "");
+        }
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testBalance2() throws Exception {
+        System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "1000");
+
+        try {
+            startGridsMultiThreaded(5);
+
+            client = true;
+
+            startGridsMultiThreaded(5, 5);
+
+            for (int i = 0; i < 5; i++) {
+                log.info("Iteration: " + i);
+
+                final AtomicInteger idx = new AtomicInteger();
+
+                GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                    @Override public Void call() throws Exception {
+                        Ignite node = ignite(idx.incrementAndGet() % 10);
+
+                        ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                        int msgs = rnd.nextInt(500, 600);
+
+                        for (int i = 0; i < msgs; i++) {
+                            int sndTo = rnd.nextInt(10);
+
+                            ClusterNode sntToNode = node.cluster().node(ignite(sndTo).cluster().localNode().id());
+
+                            IgniteCompute compute = node.compute(node.cluster().forNode(sntToNode));
+
+                            compute.call(new DummyCallable(new byte[rnd.nextInt(rnd.nextInt(256, 1024))]));
+                        }
+
+                        return null;
+                    }
+                }, 30, "test-thread");
+
+                waitNioBalanceStop(G.allGrids(), 10_000);
+            }
+        }
+        finally {
+            System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "");
+        }
+    }
+
+    /**
+     * @param nodes Node.
+     * @param timeout Timeout.
+     * @throws Exception If failed.
+     */
+    private void waitNioBalanceStop(List<Ignite> nodes, long timeout) throws Exception {
+        final List<GridNioServer> srvs = new ArrayList<>();
+
+        for (Ignite node : nodes) {
+            TcpCommunicationSpi spi = (TcpCommunicationSpi) node.configuration().getCommunicationSpi();
+
+            GridNioServer srv = GridTestUtils.getFieldValue(spi, "nioSrvr");
+
+            srvs.add(srv);
+        }
+
+        assertTrue(GridTestUtils.waitForCondition(new GridAbsPredicateX() {
+            @Override public boolean applyx() throws IgniteCheckedException {
+                List<Long> rCnts = new ArrayList<>();
+                List<Long> wCnts = new ArrayList<>();
+
+                for (GridNioServer srv : srvs) {
+                    long readerMovCnt1 = srv.readerMoveCount();
+                    long writerMovCnt1 = srv.writerMoveCount();
+
+                    rCnts.add(readerMovCnt1);
+                    wCnts.add(writerMovCnt1);
+                }
+
+                U.sleep(2000);
+
+                for (int i = 0; i < srvs.size(); i++) {
+                    GridNioServer srv = srvs.get(i);
+
+                    long readerMovCnt1 = rCnts.get(i);
+                    long writerMovCnt1 = wCnts.get(i);
+
+                    long readerMovCnt2 = srv.readerMoveCount();
+                    long writerMovCnt2 = srv.writerMoveCount();
+
+                    if (readerMovCnt1 != readerMovCnt2) {
+                        log.info("Readers balance is in progress [node=" + i + ", cnt1=" + readerMovCnt1 +
+                            ", cnt2=" + readerMovCnt2 + ']');
+
+                        return false;
+                    }
+                    if (writerMovCnt1 != writerMovCnt2) {
+                        log.info("Writers balance is in progress [node=" + i + ", cnt1=" + writerMovCnt1 +
+                            ", cnt2=" + writerMovCnt2 + ']');
+
+                        return false;
+                    }
+                }
+
+                return true;
+            }
+        }, timeout));
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRandomBalance() throws Exception {
+        System.setProperty(GridNioServer.IGNITE_IO_BALANCE_RANDOM_BALANCE, "true");
+        System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "500");
+
+        try {
+            final int NODES = 10;
+
+            startGridsMultiThreaded(NODES);
+
+            final long stopTime = System.currentTimeMillis() + 60_000;
+
+            GridTestUtils.runMultiThreaded(new Callable<Object>() {
+                @Override public Object call() throws Exception {
+                    ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                    while (System.currentTimeMillis() < stopTime)
+                        ignite(rnd.nextInt(NODES)).compute().broadcast(new DummyCallable(null));
+
+                    return null;
+                }
+            }, 20, "test-thread");
+        }
+        finally {
+            System.setProperty(GridNioServer.IGNITE_IO_BALANCE_RANDOM_BALANCE, "");
+            System.setProperty(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, "");
+        }
+    }
+
+    /**
+     *
+     */
+    private static class DummyCallable implements IgniteCallable<Object> {
+        /** */
+        private byte[] data;
+
+        /**
+         * @param data Data.
+         */
+        DummyCallable(byte[] data) {
+            this.data = data;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object call() throws Exception {
+            return data;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java
new file mode 100644
index 0000000..b644878
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessagesTest.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.G;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ *
+ */
+public class IgniteIoTestMessagesTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        startGrids(3);
+
+        client = true;
+
+        startGrid(3);
+
+        startGrid(4);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testIoTestMessages() throws Exception {
+        for (Ignite node : G.allGrids()) {
+            IgniteKernal ignite = (IgniteKernal)node;
+
+            List<ClusterNode> rmts = new ArrayList<>(ignite.cluster().forRemotes().nodes());
+
+            assertEquals(4, rmts.size());
+
+            for (ClusterNode rmt : rmts) {
+                ignite.sendIoTest(rmt, new byte[1024], false);
+
+                ignite.sendIoTest(rmt, new byte[1024], true);
+
+                ignite.sendIoTest(rmts, new byte[1024], false);
+
+                ignite.sendIoTest(rmts, new byte[1024], true);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java
new file mode 100644
index 0000000..510751e
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/IgniteVariousConnectionNumberTest.java
@@ -0,0 +1,166 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import java.util.Random;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.ignite.Ignite;
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteRunnable;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.TcpDiscoveryIpFinder;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
+import org.apache.ignite.testframework.GridTestUtils;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+import static org.apache.ignite.cache.CacheMode.REPLICATED;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.FULL_SYNC;
+
+/**
+ *
+ */
+public class IgniteVariousConnectionNumberTest extends GridCommonAbstractTest {
+    /** */
+    private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
+
+    /** */
+    private static final int NODES = 6;
+
+    /** */
+    private static Random rnd = new Random();
+
+    /** */
+    private boolean client;
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpDiscoverySpi)cfg.getDiscoverySpi()).setIpFinder(ipFinder);
+
+        int connections = rnd.nextInt(10) + 1;
+
+        log.info("Node connections [name=" + gridName + ", connections=" + connections + ']');
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setConnectionsPerNode(connections);
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setUsePairedConnections(rnd.nextBoolean());
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setSharedMemoryPort(-1);
+
+        cfg.setClientMode(client);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void beforeTestsStarted() throws Exception {
+        super.beforeTestsStarted();
+
+        long seed = U.currentTimeMillis();
+
+        rnd.setSeed(seed);
+
+        log.info("Random seed: " + seed);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected void afterTestsStopped() throws Exception {
+        stopAllGrids();
+
+        super.afterTestsStopped();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testVariousConnectionNumber() throws Exception {
+        startGridsMultiThreaded(3);
+
+        client = true;
+
+        startGridsMultiThreaded(3, 3);
+
+        CacheConfiguration ccfg = new CacheConfiguration();
+
+        ccfg.setCacheMode(REPLICATED);
+        ccfg.setWriteSynchronizationMode(FULL_SYNC);
+
+        ignite(0).createCache(ccfg);
+
+        for (int i = 0; i < 10; i++) {
+            log.info("Iteration: " + i);
+
+            runOperations(5000);
+
+            awaitPartitionMapExchange();
+
+            int idx = ThreadLocalRandom.current().nextInt(NODES);
+
+            Ignite node = ignite(idx);
+
+            client = node.configuration().isClientMode();
+
+            stopGrid(idx);
+
+            startGrid(idx);
+        }
+    }
+
+    /**
+     * @param time Execution time.
+     * @throws Exception If failed.
+     */
+    private void runOperations(final long time) throws Exception {
+        final AtomicInteger idx = new AtomicInteger();
+
+        GridTestUtils.runMultiThreaded(new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                Ignite node = ignite(idx.getAndIncrement() % NODES);
+
+                IgniteCache cache = node.cache(null);
+
+                long stopTime = U.currentTimeMillis() + time;
+
+                ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+                while (U.currentTimeMillis() < stopTime) {
+                    cache.put(rnd.nextInt(10_000), 0);
+
+                    node.compute().broadcast(new DummyJob());
+                }
+
+                return null;
+            }
+        }, NODES * 10, "test-thread");
+    }
+
+    /**
+     *
+     */
+    private static class DummyJob implements IgniteRunnable {
+        /** {@inheritDoc} */
+        @Override public void run() {
+            // No-op.
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
index 67ec371..eaa9923 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CrossCacheTxRandomOperationsTest.java
@@ -86,6 +86,11 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
     }
 
     /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 6 * 60 * 1000;
+    }
+
+    /** {@inheritDoc} */
     @Override protected void beforeTestsStarted() throws Exception {
         super.beforeTestsStarted();
 
@@ -170,9 +175,17 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
     }
 
     /**
+     * @param cacheMode Cache mode.
+     * @param writeSync Write synchronization mode.
+     * @param fairAff Fair affinity flag.
+     * @param ignite Node to use.
+     * @param name Cache name.
      */
-    protected void createCache(CacheMode cacheMode, CacheWriteSynchronizationMode writeSync, boolean fairAff,
-        Ignite ignite, String name) {
+    protected void createCache(CacheMode cacheMode,
+        CacheWriteSynchronizationMode writeSync,
+        boolean fairAff,
+        Ignite ignite,
+        String name) {
         ignite.createCache(cacheConfiguration(name, cacheMode, writeSync, fairAff));
     }
 
@@ -269,9 +282,18 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
 
             boolean checkData = fullSync && !optimistic;
 
+            long stopTime = System.currentTimeMillis() + 10_000;
+
             for (int i = 0; i < 10_000; i++) {
-                if (i % 100 == 0)
+                if (i % 100 == 0) {
+                    if (System.currentTimeMillis() > stopTime) {
+                        log.info("Stop on timeout, iteration: " + i);
+
+                        break;
+                    }
+
                     log.info("Iteration: " + i);
+                }
 
                 boolean rollback = i % 10 == 0;
 
@@ -557,4 +579,4 @@ public class CrossCacheTxRandomOperationsTest extends GridCommonAbstractTest {
             return old;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
index 9405a19..3a2bc81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridAbstractCacheInterceptorRebalanceTest.java
@@ -200,7 +200,9 @@ public abstract class GridAbstractCacheInterceptorRebalanceTest extends GridComm
     private void testRebalance(final Operation operation) throws Exception {
         interceptor = new RebalanceUpdateInterceptor();
 
-        for (int iter = 0; iter < TEST_ITERATIONS; iter++) {
+        long stopTime = System.currentTimeMillis() + 2 * 60_000;
+
+        for (int iter = 0; iter < TEST_ITERATIONS && System.currentTimeMillis() < stopTime; iter++) {
             log.info("Iteration: " + iter);
 
             failed = false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
index 9458a63..6e2e91f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateSelfTest.java
@@ -115,10 +115,10 @@ public class GridCacheOffHeapMultiThreadedUpdateSelfTest extends GridCacheOffHea
         if (gridCount() > 1)
             testPutTx(keyForNode(1), PESSIMISTIC);
     }
-    
+
     /**
      * TODO: IGNITE-592.
-     *  
+     *
      * @throws Exception If failed.
      */
     public void testPutTxOptimistic() throws Exception {
@@ -227,4 +227,4 @@ public class GridCacheOffHeapMultiThreadedUpdateSelfTest extends GridCacheOffHea
 
         assertFalse(failed);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecovery10ConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecovery10ConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecovery10ConnectionsTest.java
new file mode 100644
index 0000000..30fc9ef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecovery10ConnectionsTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicMessageRecovery10ConnectionsTest extends IgniteCacheAtomicMessageRecoveryTest {
+    /** {@inheritDoc} */
+    @Override protected int connectionsPerNode() {
+        return 10;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
new file mode 100644
index 0000000..71772ef
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.java
@@ -0,0 +1,47 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.cache.CacheAtomicityMode;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+
+import static org.apache.ignite.cache.CacheAtomicityMode.ATOMIC;
+
+/**
+ *
+ */
+public class IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest extends IgniteCacheAtomicMessageRecoveryTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        TcpCommunicationSpi commSpi = (TcpCommunicationSpi)cfg.getCommunicationSpi();
+
+        assertTrue(commSpi.isUsePairedConnections());
+
+        commSpi.setUsePairedConnections(false);
+
+        return cfg;
+    }
+
+    /** {@inheritDoc} */
+    @Override protected CacheAtomicityMode atomicityMode() {
+        return ATOMIC;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java
new file mode 100644
index 0000000..919aea6
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheConnectionRecovery10ConnectionsTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi;
+
+/**
+ *
+ */
+public class IgniteCacheConnectionRecovery10ConnectionsTest extends IgniteCacheConnectionRecoveryTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        ((TcpCommunicationSpi)cfg.getCommunicationSpi()).setConnectionsPerNode(10);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
index 2f700f3..a91de67 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheCreatePutTest.java
@@ -107,7 +107,7 @@ public class IgniteCacheCreatePutTest extends GridCommonAbstractTest {
         try {
             int iter = 0;
 
-            while (System.currentTimeMillis() < stopTime) {
+            while (System.currentTimeMillis() < stopTime && iter < 5) {
                 log.info("Iteration: " + iter++);
 
                 try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
index 0460a8f..1bfd727 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageRecoveryAbstractTest.java
@@ -58,6 +58,7 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
 
         commSpi.setSocketWriteTimeout(1000);
         commSpi.setSharedMemoryPort(-1);
+        commSpi.setConnectionsPerNode(connectionsPerNode());
 
         cfg.setCommunicationSpi(commSpi);
 
@@ -76,6 +77,13 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
     }
 
     /**
+     * @return Value for {@link TcpCommunicationSpi#setConnectionsPerNode(int)}.
+     */
+    protected int connectionsPerNode() {
+        return TcpCommunicationSpi.DFLT_CONN_PER_NODE;
+    }
+
+    /**
      * @return Cache atomicity mode.
      */
     protected abstract CacheAtomicityMode atomicityMode();
@@ -174,18 +182,22 @@ public abstract class IgniteCacheMessageRecoveryAbstractTest extends GridCommonA
     static boolean closeSessions(Ignite ignite) throws Exception {
         TcpCommunicationSpi commSpi = (TcpCommunicationSpi)ignite.configuration().getCommunicationSpi();
 
-        Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients");
+        Map<UUID, GridCommunicationClient[]> clients = U.field(commSpi, "clients");
 
         boolean closed = false;
 
-        for (GridCommunicationClient client : clients.values()) {
-            GridTcpNioCommunicationClient client0 = (GridTcpNioCommunicationClient)client;
+        for (GridCommunicationClient[] clients0 : clients.values()) {
+            for (GridCommunicationClient client : clients0) {
+                if (client != null) {
+                    GridTcpNioCommunicationClient client0 = (GridTcpNioCommunicationClient)client;
 
-            GridNioSession ses = client0.session();
+                    GridNioSession ses = client0.session();
 
-            ses.close();
+                    ses.close();
 
-            closed = true;
+                    closed = true;
+                }
+            }
         }
 
         return closed;

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
index 6256225..0dd4079 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteCacheMessageWriteTimeoutTest.java
@@ -50,8 +50,8 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest {
         // Try provoke connection close on socket writeTimeout.
         commSpi.setSharedMemoryPort(-1);
         commSpi.setMessageQueueLimit(10);
-        commSpi.setSocketReceiveBuffer(32);
-        commSpi.setSocketSendBuffer(32);
+        commSpi.setSocketReceiveBuffer(40);
+        commSpi.setSocketSendBuffer(40);
         commSpi.setSocketWriteTimeout(100);
         commSpi.setUnacknowledgedMessagesBufferSize(1000);
         commSpi.setConnectTimeout(10_000);
@@ -66,15 +66,20 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest {
         super.afterTest();
     }
 
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 10 * 60_000;
+    }
+
     /**
      * @throws Exception If failed.
      */
     public void testMessageQueueLimit() throws Exception {
-        startGridsMultiThreaded(3);
-
-        for (int i = 0; i < 15; i++) {
+        for (int i = 0; i < 3; i++) {
             log.info("Iteration: " + i);
 
+            startGridsMultiThreaded(3);
+
             IgniteInternalFuture<?> fut1 = startJobThreads(50);
 
             U.sleep(100);
@@ -83,6 +88,8 @@ public class IgniteCacheMessageWriteTimeoutTest extends GridCommonAbstractTest {
 
             fut1.get();
             fut2.get();
+
+            stopAllGrids();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
index 3fca826..322690c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/IgniteCacheMultiTxLockSelfTest.java
@@ -86,7 +86,6 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
         plc.setMaxSize(100000);
 
         ccfg.setEvictionPolicy(plc);
-        ccfg.setEvictSynchronized(true);
 
         c.setCacheConfiguration(ccfg);
 
@@ -95,6 +94,11 @@ public class IgniteCacheMultiTxLockSelfTest extends GridCommonAbstractTest {
         return c;
     }
 
+    /** {@inheritDoc} */
+    @Override protected long getTestTimeout() {
+        return 60_000;
+    }
+
     /**
      * @throws Exception If failed.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..e8175e5
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest extends
+    GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setStripedPoolSize(-1);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java
new file mode 100644
index 0000000..05fe85f
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.distributed.near;
+
+import org.apache.ignite.configuration.IgniteConfiguration;
+
+/**
+ *
+ */
+public class GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest extends
+    GridCachePartitionedMultiNodeFullApiSelfTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setStripedPoolSize(-1);
+
+        return cfg;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
index c9d18eb..e9d74ff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxDeadlockDetectionNoHangsTest.java
@@ -211,7 +211,7 @@ public class TxDeadlockDetectionNoHangsTest extends GridCommonAbstractTest {
                             tx.commit();
                         }
                         catch (Exception e) {
-                            e.printStackTrace();
+                            log.info("Ignore error: " + e);
                         }
                     }
                 }, NODES_CNT * 3, "tx-thread");

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
index aa240aa..f6a06c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/transactions/TxOptimisticDeadlockDetectionTest.java
@@ -111,6 +111,9 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         cfg.setClientMode(client);
 
+        // Test spi blocks message send, this can cause hang with striped pool.
+        cfg.setStripedPoolSize(-1);
+
         return cfg;
     }
 
@@ -274,8 +277,8 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
                     Object k;
 
-                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
-                        ", tx=" + tx + ", key=" + transformer.apply(key) + ']');
+                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode().id() +
+                        ", tx=" + tx.xid() + ", key=" + transformer.apply(key) + ']');
 
                     cache.put(transformer.apply(key), 0);
 
@@ -309,23 +312,27 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
                         entries.put(k, 2);
                     }
 
-                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode() +
-                        ", tx=" + tx + ", entries=" + entries + ']');
+                    log.info(">>> Performs put [node=" + ((IgniteKernal)ignite).localNode().id() +
+                        ", tx=" + tx.xid() + ", entries=" + entries + ']');
 
                     cache.putAll(entries);
 
                     tx.commit();
                 }
                 catch (Throwable e) {
-                    U.error(log, "Expected exception: ", e);
+                    log.info("Expected exception: " + e);
+
+                    e.printStackTrace(System.out);
 
                     // At least one stack trace should contain TransactionDeadlockException.
                     if (hasCause(e, TransactionTimeoutException.class) &&
-                        hasCause(e, TransactionDeadlockException.class)
-                        ) {
-                        if (deadlockErr.compareAndSet(null, cause(e, TransactionDeadlockException.class)))
-                            U.error(log, "At least one stack trace should contain " +
-                                TransactionDeadlockException.class.getSimpleName(), e);
+                        hasCause(e, TransactionDeadlockException.class)) {
+                        if (deadlockErr.compareAndSet(null, cause(e, TransactionDeadlockException.class))) {
+                            log.info("At least one stack trace should contain " +
+                                TransactionDeadlockException.class.getSimpleName());
+
+                            e.printStackTrace(System.out);
+                        }
                     }
                 }
             }
@@ -344,7 +351,7 @@ public class TxOptimisticDeadlockDetectionTest extends GridCommonAbstractTest {
 
         TransactionDeadlockException deadlockE = deadlockErr.get();
 
-        assertNotNull(deadlockE);
+        assertNotNull("Failed to detect deadlock", deadlockE);
 
         boolean fail = false;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
index 6fc7e02..7b5abf5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorProxySelfTest.java
@@ -372,4 +372,4 @@ public class GridServiceProcessorProxySelfTest extends GridServiceProcessorAbstr
             X.println("Executing cache service: " + ctx.name());
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
index adcd144..4bc9f01 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.util.future;
 
 import java.util.concurrent.Callable;
 import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicReference;
@@ -227,87 +228,98 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
      *
      * @throws Exception In case of any exception.
      */
-    @SuppressWarnings("ErrorNotRethrown")
     public void testChaining() throws Exception {
+        checkChaining(null);
+
+        ExecutorService exec = Executors.newFixedThreadPool(1);
+
+        try {
+            checkChaining(exec);
+
+            GridFinishedFuture<Integer> fut = new GridFinishedFuture<>(1);
+
+            IgniteInternalFuture<Object> chain = fut.chain(new CX1<IgniteInternalFuture<Integer>, Object>() {
+                @Override public Object applyx(IgniteInternalFuture<Integer> fut) throws IgniteCheckedException {
+                    return fut.get() + 1;
+                }
+            }, exec);
+
+            assertEquals(2, chain.get());
+        }
+        finally {
+            exec.shutdown();
+        }
+    }
+
+    /**
+     * @param exec Executor for chain callback.
+     * @throws Exception If failed.
+     */
+    @SuppressWarnings("ErrorNotRethrown")
+    private void checkChaining(ExecutorService exec) throws Exception {
         final CX1<IgniteInternalFuture<Object>, Object> passThrough = new CX1<IgniteInternalFuture<Object>, Object>() {
             @Override public Object applyx(IgniteInternalFuture<Object> f) throws IgniteCheckedException {
                 return f.get();
             }
         };
 
-        final GridTestKernalContext ctx = new GridTestKernalContext(log);
-
-        ctx.setExecutorService(Executors.newFixedThreadPool(1));
-        ctx.setSystemExecutorService(Executors.newFixedThreadPool(1));
-
-        ctx.add(new PoolProcessor(ctx));
-        ctx.add(new GridClosureProcessor(ctx));
+        GridFutureAdapter<Object> fut = new GridFutureAdapter<>();
+        IgniteInternalFuture<Object> chain = exec != null ? fut.chain(passThrough, exec) : fut.chain(passThrough);
 
-        ctx.start();
+        assertFalse(fut.isDone());
+        assertFalse(chain.isDone());
 
         try {
-            // Test result returned.
-
-            GridFutureAdapter<Object> fut = new GridFutureAdapter<>();
-            IgniteInternalFuture<Object> chain = fut.chain(passThrough);
+            chain.get(20);
 
-            assertFalse(fut.isDone());
-            assertFalse(chain.isDone());
-
-            try {
-                chain.get(20);
-
-                fail("Expects timeout exception.");
-            }
-            catch (IgniteFutureTimeoutCheckedException e) {
-                info("Expected timeout exception: " + e.getMessage());
-            }
+            fail("Expects timeout exception.");
+        }
+        catch (IgniteFutureTimeoutCheckedException e) {
+            info("Expected timeout exception: " + e.getMessage());
+        }
 
-            fut.onDone("result");
+        fut.onDone("result");
 
-            assertEquals("result", chain.get(1));
+        assertEquals("result", chain.get(1));
 
-            // Test exception re-thrown.
+        // Test exception re-thrown.
 
-            fut = new GridFutureAdapter<>();
-            chain = fut.chain(passThrough);
+        fut = new GridFutureAdapter<>();
+        chain = exec != null ? fut.chain(passThrough, exec) : fut.chain(passThrough);
 
-            fut.onDone(new ClusterGroupEmptyCheckedException("test exception"));
+        fut.onDone(new ClusterGroupEmptyCheckedException("test exception"));
 
-            try {
-                chain.get();
+        try {
+            chain.get();
 
-                fail("Expects failed with exception.");
-            }
-            catch (ClusterGroupEmptyCheckedException e) {
-                info("Expected exception: " + e.getMessage());
-            }
+            fail("Expects failed with exception.");
+        }
+        catch (ClusterGroupEmptyCheckedException e) {
+            info("Expected exception: " + e.getMessage());
+        }
 
-            // Test error re-thrown.
+        // Test error re-thrown.
 
-            fut = new GridFutureAdapter<>();
-            chain = fut.chain(passThrough);
+        fut = new GridFutureAdapter<>();
+        chain = exec != null ? fut.chain(passThrough, exec) : fut.chain(passThrough);
 
-            try {
-                fut.onDone(new StackOverflowError("test error"));
+        try {
+            fut.onDone(new StackOverflowError("test error"));
 
+            if (exec == null)
                 fail("Expects failed with error.");
-            }
-            catch (StackOverflowError e) {
-                info("Expected error: " + e.getMessage());
-            }
+        }
+        catch (StackOverflowError e) {
+            info("Expected error: " + e.getMessage());
+        }
 
-            try {
-                chain.get();
+        try {
+            chain.get();
 
-                fail("Expects failed with error.");
-            }
-            catch (StackOverflowError e) {
-                info("Expected error: " + e.getMessage());
-            }
+            fail("Expects failed with error.");
         }
-        finally {
-            ctx.stop(false);
+        catch (StackOverflowError e) {
+            info("Expected error: " + e.getMessage());
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
index 201fd27..d403784 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/impl/GridNioFilterChainSelfTest.java
@@ -114,7 +114,7 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
                 proceedExceptionCaught(ses, ex);
             }
 
-            @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+            @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
                 sndEvt.compareAndSet(null, ses.<String>meta(MESSAGE_WRITE_META_NAME));
 
                 sndMsgObj.compareAndSet(null, msg);
@@ -155,7 +155,7 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         chain.onSessionIdleTimeout(ses);
         chain.onSessionWriteTimeout(ses);
         assertNull(chain.onSessionClose(ses));
-        assertNull(chain.onSessionWrite(ses, snd));
+        assertNull(chain.onSessionWrite(ses, snd, true));
 
         assertEquals("DCBA", connectedEvt.get());
         assertEquals("DCBA", disconnectedEvt.get());
@@ -210,10 +210,10 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
             chainMeta(ses, MESSAGE_WRITE_META_NAME);
 
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
         }
 
         /** {@inheritDoc} */
@@ -349,6 +349,11 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
+        @Override public void sendNoFuture(Object msg) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
         @Override public GridNioFuture<Object> resumeReads() {
             return null;
         }
@@ -369,13 +374,28 @@ public class GridNioFilterChainSelfTest extends GridCommonAbstractTest {
         }
 
         /** {@inheritDoc} */
-        @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
             // No-op.
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+        @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
             return null;
         }
+
+        /** {@inheritDoc} */
+        @Override public void systemMessage(Object msg) {
+            // No-op.
+        }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java b/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
index 61a13b1..25dd780 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/nio/GridNioBenchmarkClient.java
@@ -83,7 +83,7 @@ public class GridNioBenchmarkClient {
      */
     public void run() throws IOException, InterruptedException {
         for (int i = 0; i < connCnt; i++)
-            exec.submit(new ClientThread());
+            exec.execute(new ClientThread());
 
         Thread.sleep(5*60*1000);
 
@@ -167,4 +167,4 @@ public class GridNioBenchmarkClient {
             return read;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
index f21f31b..a18ef32 100644
--- a/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/p2p/GridP2PRecursionTaskSelfTest.java
@@ -196,4 +196,4 @@ public class GridP2PRecursionTaskSelfTest extends GridCommonAbstractTest {
             return ignite.compute().execute(FactorialTask.class, arg);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
index 652e47f..5ca8f26 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/GridTcpSpiForwardingSelfTest.java
@@ -69,7 +69,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
     private static final int commExtPort2 = 20100;
 
     /** */
-    private AddressResolver resolver;
+    private AddressResolver rslvr;
 
     /** */
     private boolean ipFinderUseLocPorts;
@@ -111,14 +111,15 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         cfg.setConnectorConfiguration(null);
 
         TcpCommunicationSpi commSpi = new TcpCommunicationSpi() {
-            @Override protected GridCommunicationClient createTcpClient(ClusterNode node) throws IgniteCheckedException {
+            @Override protected GridCommunicationClient createTcpClient(ClusterNode node, int connIdx)
+                throws IgniteCheckedException {
                 Map<String, Object> attrs = new HashMap<>(node.attributes());
 
                 attrs.remove(createSpiAttributeName(ATTR_PORT));
 
                 ((TcpDiscoveryNode)node).setAttributes(attrs);
 
-                return super.createTcpClient(node);
+                return super.createTcpClient(node, connIdx);
             }
         };
 
@@ -126,12 +127,13 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         commSpi.setLocalPort(commLocPort);
         commSpi.setLocalPortRange(1);
         commSpi.setSharedMemoryPort(-1);
+        commSpi.setConnectionsPerNode(1);
 
         cfg.setCommunicationSpi(commSpi);
 
-        assert resolver != null;
+        assert rslvr != null;
 
-        cfg.setAddressResolver(resolver);
+        cfg.setAddressResolver(rslvr);
 
         return cfg;
     }
@@ -147,7 +149,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         map.put(new InetSocketAddress("127.0.0.1", locPort2), F.asList(new InetSocketAddress("127.0.0.1", extPort2)));
         map.put(new InetSocketAddress("127.0.0.1", commLocPort2), F.asList(new InetSocketAddress("127.0.0.1", commExtPort2)));
 
-        resolver = new AddressResolver() {
+        rslvr = new AddressResolver() {
             @Override public Collection<InetSocketAddress> getExternalAddresses(InetSocketAddress addr) {
                 return map.get(addr);
             }
@@ -167,7 +169,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
         map.put("127.0.0.1:" + locPort2, "127.0.0.1:" + extPort2);
         map.put("127.0.0.1:" + commLocPort2, "127.0.0.1:" + commExtPort2);
 
-        resolver = new BasicAddressResolver(map);
+        rslvr = new BasicAddressResolver(map);
 
         doTestForward();
     }
@@ -180,7 +182,7 @@ public class GridTcpSpiForwardingSelfTest extends GridCommonAbstractTest {
 
         map.put("127.0.0.1", "127.0.0.1");
 
-        resolver = new BasicAddressResolver(map);
+        rslvr = new BasicAddressResolver(map);
 
         ipFinderUseLocPorts = true;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/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 076724d..3c4fea0 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
@@ -90,16 +90,36 @@ abstract class GridTcpCommunicationSpiAbstractTest extends GridAbstractCommunica
         super.afterTest();
 
         for (CommunicationSpi spi : spis.values()) {
-            ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
+            ConcurrentMap<UUID, GridCommunicationClient[]> clients = U.field(spi, "clients");
+
+            for (int i = 0; i < 20; i++) {
+                GridCommunicationClient client0 = null;
+
+                for (GridCommunicationClient[] clients0 : clients.values()) {
+                    for (GridCommunicationClient client : clients0) {
+                        if (client != null) {
+                            client0 = client;
+
+                            break;
+                        }
+                    }
+
+                    if (client0 != null)
+                        break;
+                }
+
+                if (client0 == null)
+                    return;
 
-            for (int i = 0; i < 20 && !clients.isEmpty(); i++) {
                 info("Check failed for SPI [grid=" +
-                    GridTestUtils.getFieldValue(spi, IgniteSpiAdapter.class, "gridName") + ", spi=" + spi + ']');
+                    GridTestUtils.getFieldValue(spi, IgniteSpiAdapter.class, "gridName") +
+                    ", client=" + client0 +
+                    ", spi=" + spi + ']');
 
                 U.sleep(1000);
             }
 
-            assert clients.isEmpty() : "Clients: " + clients;
+            fail("Failed to wait when clients are closed.");
         }
     }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/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 8635d94..a649130 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
@@ -31,6 +31,7 @@ import java.util.concurrent.ThreadLocalRandom;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.cluster.ClusterNode;
 import org.apache.ignite.configuration.IgniteConfiguration;
@@ -83,6 +84,12 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
     /** Use ssl. */
     protected boolean useSsl;
 
+    /** */
+    private int connectionsPerNode = 1;
+
+    /** */
+    private boolean pairedConnections = true;
+
     /**
      *
      */
@@ -163,6 +170,34 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
     /**
      * @throws Exception If failed.
      */
+    public void testMultithreaded_10Connections() throws Exception {
+        connectionsPerNode = 10;
+
+        testMultithreaded();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultithreaded_NoPairedConnections() throws Exception {
+        pairedConnections = false;
+
+        testMultithreaded();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testMultithreaded_10ConnectionsNoPaired() throws Exception {
+        pairedConnections = false;
+        connectionsPerNode = 10;
+
+        testMultithreaded();
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
     public void testWithLoad() throws Exception {
         int threads = Runtime.getRuntime().availableProcessors() * 5;
 
@@ -244,7 +279,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
                 final AtomicInteger idx = new AtomicInteger();
 
                 try {
-                    GridTestUtils.runMultiThreaded(new Callable<Void>() {
+                    final Callable<Void> c = new Callable<Void>() {
                         @Override public Void call() throws Exception {
                             int idx0 = idx.getAndIncrement();
 
@@ -270,7 +305,40 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
 
                             return null;
                         }
-                    }, threads, "test");
+                    };
+
+                    List<Thread> threadsList = new ArrayList<>();
+
+                    final AtomicBoolean fail = new AtomicBoolean();
+
+                    final AtomicLong tId = new AtomicLong();
+
+                    for (int t = 0; t < threads; t++) {
+                        Thread t0 = new Thread(new Runnable() {
+                            @Override public void run() {
+                                try {
+                                    c.call();
+                                }
+                                catch (Throwable e) {
+                                    log.error("Unexpected error: " + e, e);
+
+                                    fail.set(true);
+                                }
+                            }
+                        }) {
+                            @Override public long getId() {
+                                // Override getId to use all connections.
+                                return tId.getAndIncrement();
+                            }
+                        };
+
+                        threadsList.add(t0);
+
+                        t0.start();
+                    }
+
+                    for (Thread t0 : threadsList)
+                        t0.join();
 
                     assertTrue(latch.await(10, TimeUnit.SECONDS));
 
@@ -281,17 +349,19 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
 
                         final GridNioServer srv = U.field(spi, "nioSrvr");
 
+                        final int conns = pairedConnections ? 2 : 1;
+
                         GridTestUtils.waitForCondition(new GridAbsPredicate() {
                             @Override public boolean apply() {
                                 Collection sessions = U.field(srv, "sessions");
 
-                                return sessions.size() == 1;
+                                return sessions.size() == conns * connectionsPerNode;
                             }
                         }, 5000);
 
                         Collection sessions = U.field(srv, "sessions");
 
-                        assertEquals(1, sessions.size());
+                        assertEquals(conns * connectionsPerNode, sessions.size());
                     }
 
                     assertEquals(expMsgs, lsnr.cntr.get());
@@ -320,6 +390,8 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
         spi.setIdleConnectionTimeout(60_000);
         spi.setConnectTimeout(10_000);
         spi.setSharedMemoryPort(-1);
+        spi.setConnectionsPerNode(connectionsPerNode);
+        spi.setUsePairedConnections(pairedConnections);
 
         return spi;
     }
@@ -434,4 +506,4 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
             rsrcs.stopThreads();
     }
 
-}
\ No newline at end of file
+}


[19/25] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NIO sessio

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index c8e2e0b..bc1f173 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -37,6 +37,7 @@ import java.nio.channels.WritableByteChannel;
 import java.nio.channels.spi.SelectorProvider;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
 import java.util.Deque;
 import java.util.Iterator;
 import java.util.List;
@@ -44,6 +45,8 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteLogger;
@@ -63,6 +66,7 @@ import org.apache.ignite.internal.util.worker.GridWorker;
 import org.apache.ignite.lang.IgniteBiInClosure;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
+import org.apache.ignite.lang.IgniteRunnable;
 import org.apache.ignite.plugin.extensions.communication.Message;
 import org.apache.ignite.plugin.extensions.communication.MessageReader;
 import org.apache.ignite.plugin.extensions.communication.MessageWriter;
@@ -86,11 +90,14 @@ import static org.apache.ignite.internal.util.nio.GridNioSessionMetaKey.NIO_OPER
  *
  */
 public class GridNioServer<T> {
+    /** */
+    public static final String IGNITE_IO_BALANCE_RANDOM_BALANCE = "IGNITE_IO_BALANCE_RANDOM_BALANCER";
+
     /** Default session write timeout. */
     public static final int DFLT_SES_WRITE_TIMEOUT = 5000;
 
     /** Default send queue limit. */
-    public static final int DFLT_SEND_QUEUE_LIMIT = 1024;
+    public static final int DFLT_SEND_QUEUE_LIMIT = 0;
 
     /** Time, which server will wait before retry operation. */
     private static final long ERR_WAIT_TIME = 2000;
@@ -122,6 +129,9 @@ public class GridNioServer<T> {
         }
     }
 
+    /** Defines how many times selector should do {@code selectNow()} before doing {@code select(long)}. */
+    private long selectorSpins;
+
     /** Accept worker thread. */
     @GridToStringExclude
     private final IgniteThread acceptThread;
@@ -145,9 +155,13 @@ public class GridNioServer<T> {
     /** Flag indicating if this server should use direct buffers. */
     private final boolean directBuf;
 
-    /** Index to select which thread will serve next socket channel. Using round-robin balancing. */
+    /** Index to select which thread will serve next incoming socket channel. Using round-robin balancing. */
+    @GridToStringExclude
+    private int readBalanceIdx;
+
+    /** Index to select which thread will serve next out socket channel. Using round-robin balancing. */
     @GridToStringExclude
-    private int balanceIdx;
+    private int writeBalanceIdx = 1;
 
     /** Tcp no delay flag. */
     private final boolean tcpNoDelay;
@@ -204,12 +218,25 @@ public class GridNioServer<T> {
     /** Optional listener to monitor outbound message queue size. */
     private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
 
+    /** */
+    private final AtomicLong readerMoveCnt = new AtomicLong();
+
+    /** */
+    private final AtomicLong writerMoveCnt = new AtomicLong();
+
+    /** */
+    private final IgniteRunnable balancer;
+
     /**
      * @param addr Address.
      * @param port Port.
      * @param log Log.
      * @param selectorCnt Count of selectors and selecting threads.
      * @param gridName Grid name.
+     * @param srvName Logical server name for threads identification.
+     * @param selectorSpins Defines how many non-blocking {@code selector.selectNow()} should be made before
+     *      falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+     *      Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
      * @param tcpNoDelay If TCP_NODELAY option should be set to accepted sockets.
      * @param directBuf Direct buffer flag.
      * @param order Byte order.
@@ -223,6 +250,7 @@ public class GridNioServer<T> {
      * @param writerFactory Writer factory.
      * @param skipRecoveryPred Skip recovery predicate.
      * @param msgQueueLsnr Message queue size listener.
+     * @param balancing NIO sessions balancing flag.
      * @param filters Filters for this server.
      * @throws IgniteCheckedException If failed.
      */
@@ -232,6 +260,8 @@ public class GridNioServer<T> {
         IgniteLogger log,
         int selectorCnt,
         @Nullable String gridName,
+        @Nullable String srvName,
+        long selectorSpins,
         boolean tcpNoDelay,
         boolean directBuf,
         ByteOrder order,
@@ -245,6 +275,7 @@ public class GridNioServer<T> {
         GridNioMessageWriterFactory writerFactory,
         IgnitePredicate<Message> skipRecoveryPred,
         IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr,
+        boolean balancing,
         GridNioFilter... filters
     ) throws IgniteCheckedException {
         if (port != -1)
@@ -268,6 +299,7 @@ public class GridNioServer<T> {
         this.sockSndBuf = sockSndBuf;
         this.sndQueueLimit = sndQueueLimit;
         this.msgQueueLsnr = msgQueueLsnr;
+        this.selectorSpins = selectorSpins;
 
         filterChain = new GridNioFilterChain<>(log, lsnr, new HeadFilter(), filters);
 
@@ -299,9 +331,16 @@ public class GridNioServer<T> {
         clientThreads = new IgniteThread[selectorCnt];
 
         for (int i = 0; i < selectorCnt; i++) {
+            String threadName;
+
+            if (srvName == null)
+                threadName = "grid-nio-worker-" + i;
+            else
+                threadName = "grid-nio-worker-" + srvName + "-" + i;
+
             AbstractNioClientWorker worker = directMode ?
-                new DirectNioClientWorker(i, gridName, "grid-nio-worker-" + i, log) :
-                new ByteBufferNioClientWorker(i, gridName, "grid-nio-worker-" + i, log);
+                new DirectNioClientWorker(i, gridName, threadName, log) :
+                new ByteBufferNioClientWorker(i, gridName, threadName, log);
 
             clientWorkers.add(worker);
 
@@ -315,6 +354,32 @@ public class GridNioServer<T> {
         this.writerFactory = writerFactory;
 
         this.skipRecoveryPred = skipRecoveryPred != null ? skipRecoveryPred : F.<Message>alwaysFalse();
+
+        long balancePeriod = IgniteSystemProperties.getLong(IgniteSystemProperties.IGNITE_IO_BALANCE_PERIOD, 5000);
+
+        IgniteRunnable balancer0 = null;
+
+        if (balancing && balancePeriod > 0) {
+            boolean rndBalance = IgniteSystemProperties.getBoolean(IGNITE_IO_BALANCE_RANDOM_BALANCE, false);
+
+            balancer0 = rndBalance ? new RandomBalancer() : new SizeBasedBalancer(balancePeriod);
+        }
+
+        this.balancer = balancer0;
+    }
+
+    /**
+     * @return Number of reader sessions move.
+     */
+    public long readerMoveCount() {
+        return readerMoveCnt.get();
+    }
+
+    /**
+     * @return Number of reader writer move.
+     */
+    public long writerMoveCount() {
+        return writerMoveCnt.get();
     }
 
     /**
@@ -377,6 +442,13 @@ public class GridNioServer<T> {
     }
 
     /**
+     * @return Selector spins.
+     */
+    public long selectorSpins() {
+        return selectorSpins;
+    }
+
+    /**
      * @param ses Session to close.
      * @return Future for operation.
      */
@@ -390,7 +462,7 @@ public class GridNioServer<T> {
 
         NioOperationFuture<Boolean> fut = new NioOperationFuture<>(impl, NioOperation.CLOSE);
 
-        clientWorkers.get(impl.selectorIndex()).offer(fut);
+        impl.offerStateChange(fut);
 
         return fut;
     }
@@ -398,61 +470,91 @@ public class GridNioServer<T> {
     /**
      * @param ses Session.
      * @param msg Message.
+     * @param createFut {@code True} if future should be created.
      * @return Future for operation.
      */
-    GridNioFuture<?> send(GridNioSession ses, ByteBuffer msg) {
-        assert ses instanceof GridSelectorNioSessionImpl;
+    GridNioFuture<?> send(GridNioSession ses, ByteBuffer msg, boolean createFut) throws IgniteCheckedException {
+        assert ses instanceof GridSelectorNioSessionImpl : ses;
 
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
-        NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
+        if (createFut) {
+            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg);
 
-        send0(impl, fut, false);
+            send0(impl, fut, false);
 
-        return fut;
+            return fut;
+        }
+        else {
+            SessionWriteRequest req = new WriteRequestImpl(ses, msg, true);
+
+            send0(impl, req, false);
+
+            return null;
+        }
     }
 
     /**
      * @param ses Session.
      * @param msg Message.
+     * @param createFut {@code True} if future should be created.
      * @return Future for operation.
      */
-    GridNioFuture<?> send(GridNioSession ses, Message msg) {
+    GridNioFuture<?> send(GridNioSession ses, Message msg, boolean createFut) throws IgniteCheckedException {
         assert ses instanceof GridSelectorNioSessionImpl;
 
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
-        NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
-            skipRecoveryPred.apply(msg));
+        if (createFut) {
+            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
+                skipRecoveryPred.apply(msg));
 
-        send0(impl, fut, false);
+            send0(impl, fut, false);
 
-        return fut;
+            return fut;
+        }
+        else {
+            SessionWriteRequest req = new WriteRequestImpl(ses, msg, skipRecoveryPred.apply(msg));
+
+            send0(impl, req, false);
+
+            return null;
+        }
     }
 
     /**
      * @param ses Session.
-     * @param fut Future.
+     * @param req Request.
      * @param sys System message flag.
+     * @throws IgniteCheckedException If session was closed.
      */
-    private void send0(GridSelectorNioSessionImpl ses, NioOperationFuture<?> fut, boolean sys) {
+    private void send0(GridSelectorNioSessionImpl ses, SessionWriteRequest req, boolean sys) throws IgniteCheckedException {
         assert ses != null;
-        assert fut != null;
+        assert req != null;
 
-        int msgCnt = sys ? ses.offerSystemFuture(fut) : ses.offerFuture(fut);
+        int msgCnt = sys ? ses.offerSystemFuture(req) : ses.offerFuture(req);
 
         IgniteInClosure<IgniteException> ackC;
 
         if (!sys && (ackC = ses.removeMeta(ACK_CLOSURE.ordinal())) != null)
-            fut.ackClosure(ackC);
+            req.ackClosure(ackC);
 
         if (ses.closed()) {
-            if (ses.removeFuture(fut))
-                fut.connectionClosed();
+            if (ses.removeFuture(req)) {
+                IOException err = new IOException("Failed to send message (connection was closed): " + ses);
+
+                req.onError(err);
+
+                if (!(req instanceof GridNioFuture))
+                    throw new IgniteCheckedException(err);
+            }
+        }
+        else if (!ses.procWrite.get() && ses.procWrite.compareAndSet(false, true)) {
+            AbstractNioClientWorker worker = (AbstractNioClientWorker)ses.worker();
+
+            if (worker != null)
+                worker.offer((SessionChangeRequest)req);
         }
-        else if (msgCnt == 1)
-            // Change from 0 to 1 means that worker thread should be waken up.
-            clientWorkers.get(ses.selectorIndex()).offer(fut);
 
         if (msgQueueLsnr != null)
             msgQueueLsnr.apply(ses, msgCnt);
@@ -463,10 +565,10 @@ public class GridNioServer<T> {
      *
      * @param ses Session.
      * @param msg Message.
-     * @return Future.
+     * @throws IgniteCheckedException If session was closed.
      */
-    public GridNioFuture<?> sendSystem(GridNioSession ses, Message msg) {
-        return sendSystem(ses, msg, null);
+    public void sendSystem(GridNioSession ses, Message msg) throws IgniteCheckedException {
+        sendSystem(ses, msg, null);
     }
 
     /**
@@ -475,27 +577,30 @@ public class GridNioServer<T> {
      * @param ses Session.
      * @param msg Message.
      * @param lsnr Future listener notified from the session thread.
-     * @return Future.
+     * @throws IgniteCheckedException If session was closed.
      */
-    public GridNioFuture<?> sendSystem(GridNioSession ses,
+    public void sendSystem(GridNioSession ses,
         Message msg,
-        @Nullable IgniteInClosure<? super IgniteInternalFuture<?>> lsnr) {
+        @Nullable IgniteInClosure<? super IgniteInternalFuture<?>> lsnr) throws IgniteCheckedException {
         assert ses instanceof GridSelectorNioSessionImpl;
 
         GridSelectorNioSessionImpl impl = (GridSelectorNioSessionImpl)ses;
 
-        NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
-            skipRecoveryPred.apply(msg));
-
         if (lsnr != null) {
+            NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, NioOperation.REQUIRE_WRITE, msg,
+                skipRecoveryPred.apply(msg));
+
             fut.listen(lsnr);
 
             assert !fut.isDone();
-        }
 
-        send0(impl, fut, true);
+            send0(impl, fut, true);
+        }
+        else {
+            SessionWriteRequest req = new WriteRequestSystemImpl(ses, msg);
 
-        return fut;
+            send0(impl, req, true);
+        }
     }
 
     /**
@@ -504,37 +609,69 @@ public class GridNioServer<T> {
     public void resend(GridNioSession ses) {
         assert ses instanceof GridSelectorNioSessionImpl;
 
-        GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+        GridNioRecoveryDescriptor recoveryDesc = ses.outRecoveryDescriptor();
 
-        if (recoveryDesc != null && !recoveryDesc.messagesFutures().isEmpty()) {
-            Deque<GridNioFuture<?>> futs = recoveryDesc.messagesFutures();
+        if (recoveryDesc != null && !recoveryDesc.messagesRequests().isEmpty()) {
+            Deque<SessionWriteRequest> futs = recoveryDesc.messagesRequests();
 
             if (log.isDebugEnabled())
                 log.debug("Resend messages [rmtNode=" + recoveryDesc.node().id() + ", msgCnt=" + futs.size() + ']');
 
             GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
 
-            GridNioFuture<?> fut0 = futs.iterator().next();
+            SessionWriteRequest fut0 = futs.iterator().next();
 
-            for (GridNioFuture<?> fut : futs) {
+            for (SessionWriteRequest fut : futs) {
                 fut.messageThread(true);
 
-                ((NioOperationFuture)fut).resetSession(ses0);
+                fut.resetSession(ses0);
             }
 
             ses0.resend(futs);
 
             // Wake up worker.
-            clientWorkers.get(ses0.selectorIndex()).offer(((NioOperationFuture)fut0));
+            ses0.offerStateChange((GridNioServer.SessionChangeRequest)fut0);
         }
     }
 
     /**
+     * @return Sessions.
+     */
+    public Collection<? extends GridNioSession> sessions() {
+        return sessions;
+    }
+
+    /**
+     * @return Workers.
+     */
+    public List<AbstractNioClientWorker> workers() {
+        return clientWorkers;
+    }
+
+    /**
+     * @param ses Session.
+     * @param from Move from index.
+     * @param to Move to index.
+     */
+    private void moveSession(GridNioSession ses, int from, int to) {
+        assert from >= 0 && from < clientWorkers.size() : from;
+        assert to >= 0 && to < clientWorkers.size() : to;
+        assert from != to;
+
+        GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
+
+        SessionMoveFuture fut = new SessionMoveFuture(ses0, to);
+
+        if (!ses0.offerMove(clientWorkers.get(from), fut))
+            fut.onDone(false);
+    }
+
+    /**
      * @param ses Session.
      * @param op Operation.
      * @return Future for operation.
      */
-    GridNioFuture<?> pauseResumeReads(GridNioSession ses, NioOperation op) {
+    private GridNioFuture<?> pauseResumeReads(GridNioSession ses, NioOperation op) {
         assert ses instanceof GridSelectorNioSessionImpl;
         assert op == NioOperation.PAUSE_READ || op == NioOperation.RESUME_READ;
 
@@ -546,7 +683,7 @@ public class GridNioServer<T> {
 
         NioOperationFuture<?> fut = new NioOperationFuture<Void>(impl, op);
 
-        clientWorkers.get(impl.selectorIndex()).offer(fut);
+        impl.offerStateChange(fut);
 
         return fut;
     }
@@ -555,6 +692,9 @@ public class GridNioServer<T> {
      *
      */
     public void dumpStats() {
+        U.warn(log, "NIO server statistics [readerSesBalanceCnt=" + readerMoveCnt.get() +
+            ", writerSesBalanceCnt=" + writerMoveCnt.get() + ']');
+
         for (int i = 0; i < clientWorkers.size(); i++)
             clientWorkers.get(i).offer(new NioOperationFuture<Void>(null, NioOperation.DUMP_STATS));
     }
@@ -675,12 +815,35 @@ public class GridNioServer<T> {
      * @param req Request to balance.
      */
     private synchronized void offerBalanced(NioOperationFuture req) {
-        clientWorkers.get(balanceIdx).offer(req);
+        assert req.operation() == NioOperation.REGISTER : req;
+        assert req.socketChannel() != null : req;
+
+        int workers = clientWorkers.size();
+
+        int balanceIdx;
+
+        if (workers > 1) {
+            if (req.accepted()) {
+                balanceIdx = readBalanceIdx;
+
+                readBalanceIdx += 2;
+
+                if (readBalanceIdx >= workers)
+                    readBalanceIdx = 0;
+            }
+            else {
+                balanceIdx = writeBalanceIdx;
 
-        balanceIdx++;
+                writeBalanceIdx += 2;
 
-        if (balanceIdx == clientWorkers.size())
+                if (writeBalanceIdx >= workers)
+                    writeBalanceIdx = 1;
+            }
+        }
+        else
             balanceIdx = 0;
+
+        clientWorkers.get(balanceIdx).offer(req);
     }
 
     /** {@inheritDoc} */
@@ -792,21 +955,30 @@ public class GridNioServer<T> {
 
             while (true) {
                 ByteBuffer buf = ses.removeMeta(BUF_META_KEY);
-                NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
+                SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
 
                 // Check if there were any pending data from previous writes.
                 if (buf == null) {
                     assert req == null;
 
-                    req = (NioOperationFuture<?>)ses.pollFuture();
+                    req = ses.pollFuture();
 
                     if (req == null) {
-                        key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                        if (ses.procWrite.get()) {
+                            ses.procWrite.set(false);
+
+                            if (ses.writeQueue().isEmpty()) {
+                                if ((key.interestOps() & SelectionKey.OP_WRITE) != 0)
+                                    key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                            }
+                            else
+                                ses.procWrite.set(true);
+                        }
 
                         break;
                     }
 
-                    buf = req.message();
+                    buf = (ByteBuffer)req.message();
                 }
 
                 if (!skipWrite) {
@@ -841,10 +1013,15 @@ public class GridNioServer<T> {
                     // Message was successfully written.
                     assert req != null;
 
-                    req.onDone();
+                    req.onMessageWritten();
                 }
             }
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(ByteBufferNioClientWorker.class, this, super.toString());
+        }
     }
 
     /**
@@ -909,6 +1086,7 @@ public class GridNioServer<T> {
                 metricsLsnr.onBytesReceived(cnt);
 
             ses.bytesReceived(cnt);
+            onRead(cnt);
 
             readBuf.flip();
 
@@ -921,6 +1099,12 @@ public class GridNioServer<T> {
                     readBuf.compact();
                 else
                     readBuf.clear();
+
+                if (ses.hasSystemMessage() && !ses.procWrite.get()) {
+                    ses.procWrite.set(true);
+
+                    registerWrite(ses);
+                }
             }
             catch (IgniteCheckedException e) {
                 close(ses, e);
@@ -993,16 +1177,29 @@ public class GridNioServer<T> {
                 if (ses.meta(WRITE_BUF_LIMIT) != null)
                     buf.limit((int)ses.meta(WRITE_BUF_LIMIT));
 
-                NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
+                SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
 
                 while (true) {
                     if (req == null) {
-                        req = (NioOperationFuture<?>)ses.pollFuture();
+                        req = systemMessage(ses);
 
-                        if (req == null && buf.position() == 0) {
-                            key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                        if (req == null) {
+                            req = ses.pollFuture();
 
-                            break;
+                            if (req == null && buf.position() == 0) {
+                                if (ses.procWrite.get()) {
+                                    ses.procWrite.set(false);
+
+                                    if (ses.writeQueue().isEmpty()) {
+                                        if ((key.interestOps() & SelectionKey.OP_WRITE) != 0)
+                                            key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                                    }
+                                    else
+                                        ses.procWrite.set(true);
+                                }
+
+                                break;
+                            }
                         }
                     }
 
@@ -1010,7 +1207,7 @@ public class GridNioServer<T> {
                     boolean finished = false;
 
                     if (req != null) {
-                        msg = req.directMessage();
+                        msg = (Message)req.message();
 
                         assert msg != null;
 
@@ -1025,14 +1222,17 @@ public class GridNioServer<T> {
 
                     // Fill up as many messages as possible to write buffer.
                     while (finished) {
-                        req.onDone();
+                        req.onMessageWritten();
 
-                        req = (NioOperationFuture<?>)ses.pollFuture();
+                        req = systemMessage(ses);
+
+                        if (req == null)
+                            req = ses.pollFuture();
 
                         if (req == null)
                             break;
 
-                        msg = req.directMessage();
+                        msg = (Message)req.message();
 
                         assert msg != null;
 
@@ -1129,13 +1329,31 @@ public class GridNioServer<T> {
                 ses.bytesSent(cnt);
 
                 if (!buf.hasRemaining())
-                    queue.remove(buf);
+                    queue.poll();
                 else
                     break;
             }
         }
 
         /**
+         * @param ses Session.
+         * @return System message request.
+         */
+        private SessionWriteRequest systemMessage(GridSelectorNioSessionImpl ses) {
+            if (ses.hasSystemMessage()) {
+                Object msg = ses.systemMessage();
+
+                SessionWriteRequest req = new WriteRequestSystemImpl(ses, msg);
+
+                assert !ses.hasSystemMessage();
+
+                return req;
+            }
+
+            return null;
+        }
+
+        /**
          * Processes write-ready event on the key.
          *
          * @param key Key that is ready to be written.
@@ -1147,7 +1365,7 @@ public class GridNioServer<T> {
 
             GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
             ByteBuffer buf = ses.writeBuffer();
-            NioOperationFuture<?> req = ses.removeMeta(NIO_OPERATION.ordinal());
+            SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
 
             MessageWriter writer = ses.meta(MSG_WRITER.ordinal());
 
@@ -1161,12 +1379,25 @@ public class GridNioServer<T> {
             }
 
             if (req == null) {
-                req = (NioOperationFuture<?>)ses.pollFuture();
+                req = systemMessage(ses);
 
-                if (req == null && buf.position() == 0) {
-                    key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                if (req == null) {
+                    req = ses.pollFuture();
 
-                    return;
+                    if (req == null && buf.position() == 0) {
+                        if (ses.procWrite.get()) {
+                            ses.procWrite.set(false);
+
+                            if (ses.writeQueue().isEmpty()) {
+                                if ((key.interestOps() & SelectionKey.OP_WRITE) != 0)
+                                    key.interestOps(key.interestOps() & (~SelectionKey.OP_WRITE));
+                            }
+                            else
+                                ses.procWrite.set(true);
+                        }
+
+                        return;
+                    }
                 }
             }
 
@@ -1174,9 +1405,9 @@ public class GridNioServer<T> {
             boolean finished = false;
 
             if (req != null) {
-                msg = req.directMessage();
+                msg = (Message)req.message();
 
-                assert msg != null;
+                assert msg != null : req;
 
                 if (writer != null)
                     writer.setCurrentWriteClass(msg.getClass());
@@ -1189,14 +1420,17 @@ public class GridNioServer<T> {
 
             // Fill up as many messages as possible to write buffer.
             while (finished) {
-                req.onDone();
+                req.onMessageWritten();
 
-                req = (NioOperationFuture<?>)ses.pollFuture();
+                req = systemMessage(ses);
+
+                if (req == null)
+                    req = ses.pollFuture();
 
                 if (req == null)
                     break;
 
-                msg = req.directMessage();
+                msg = (Message)req.message();
 
                 assert msg != null;
 
@@ -1223,6 +1457,7 @@ public class GridNioServer<T> {
                     metricsLsnr.onBytesSent(cnt);
 
                 ses.bytesSent(cnt);
+                onWrite(cnt);
             }
             else {
                 // For test purposes only (skipWrite is set to true in tests only).
@@ -1242,14 +1477,19 @@ public class GridNioServer<T> {
             else
                 buf.clear();
         }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(DirectNioClientWorker.class, this, super.toString());
+        }
     }
 
     /**
      * Thread performing only read operations from the channel.
      */
-    private abstract class AbstractNioClientWorker extends GridWorker {
+    private abstract class AbstractNioClientWorker extends GridWorker implements GridNioWorker {
         /** Queue of change requests on this selector. */
-        private final ConcurrentLinkedQueue<NioOperationFuture> changeReqs = new ConcurrentLinkedQueue<>();
+        private final ConcurrentLinkedQueue<SessionChangeRequest> changeReqs = new ConcurrentLinkedQueue<>();
 
         /** Selector to select read events. */
         private Selector selector;
@@ -1260,6 +1500,25 @@ public class GridNioServer<T> {
         /** Worker index. */
         private final int idx;
 
+        /** */
+        private long bytesRcvd;
+
+        /** */
+        private long bytesSent;
+
+        /** */
+        private volatile long bytesRcvd0;
+
+        /** */
+        private volatile long bytesSent0;
+
+        /** Sessions assigned to this worker. */
+        private final GridConcurrentHashSet<GridSelectorNioSessionImpl> workerSessions =
+            new GridConcurrentHashSet<>();
+
+        /** {@code True} if worker has called or is about to call {@code Selector.select()}. */
+        private volatile boolean select;
+
         /**
          * @param idx Index of this worker in server's array.
          * @param gridName Grid name.
@@ -1322,15 +1581,15 @@ public class GridNioServer<T> {
             try {
                 SelectedSelectionKeySet selectedKeySet = new SelectedSelectionKeySet();
 
-                Class<?> selectorImplClass =
+                Class<?> selectorImplCls =
                     Class.forName("sun.nio.ch.SelectorImpl", false, U.gridClassLoader());
 
                 // Ensure the current selector implementation is what we can instrument.
-                if (!selectorImplClass.isAssignableFrom(selector.getClass()))
+                if (!selectorImplCls.isAssignableFrom(selector.getClass()))
                     return;
 
-                Field selectedKeysField = selectorImplClass.getDeclaredField("selectedKeys");
-                Field publicSelectedKeysField = selectorImplClass.getDeclaredField("publicSelectedKeys");
+                Field selectedKeysField = selectorImplCls.getDeclaredField("selectedKeys");
+                Field publicSelectedKeysField = selectorImplCls.getDeclaredField("publicSelectedKeys");
 
                 selectedKeysField.setAccessible(true);
                 publicSelectedKeysField.setAccessible(true);
@@ -1357,48 +1616,126 @@ public class GridNioServer<T> {
          *
          * @param req Change request.
          */
-        private void offer(NioOperationFuture req) {
+        public void offer(SessionChangeRequest req) {
             changeReqs.offer(req);
 
+            if (select)
+                selector.wakeup();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void offer(Collection<SessionChangeRequest> reqs) {
+            for (SessionChangeRequest req : reqs)
+                changeReqs.offer(req);
+
             selector.wakeup();
         }
 
+        /** {@inheritDoc} */
+        @Override public List<SessionChangeRequest> clearSessionRequests(GridNioSession ses) {
+            List<SessionChangeRequest> sesReqs = null;
+
+            for (SessionChangeRequest changeReq : changeReqs) {
+                if (changeReq.session() == ses && !(changeReq instanceof SessionMoveFuture)) {
+                    boolean rmv = changeReqs.remove(changeReq);
+
+                    assert rmv : changeReq;
+
+                    if (sesReqs == null)
+                        sesReqs = new ArrayList<>();
+
+                    sesReqs.add(changeReq);
+                }
+            }
+
+            return sesReqs;
+        }
+
         /**
          * Processes read and write events and registration requests.
          *
          * @throws IgniteCheckedException If IOException occurred or thread was unable to add worker to workers pool.
          */
         @SuppressWarnings("unchecked")
-        private void bodyInternal() throws IgniteCheckedException {
+        private void bodyInternal() throws IgniteCheckedException, InterruptedException {
             try {
                 long lastIdleCheck = U.currentTimeMillis();
 
+                mainLoop:
                 while (!closed && selector.isOpen()) {
-                    NioOperationFuture req;
+                    SessionChangeRequest req0;
 
-                    while ((req = changeReqs.poll()) != null) {
-                        switch (req.operation()) {
+                    while ((req0 = changeReqs.poll()) != null) {
+                        switch (req0.operation()) {
                             case REGISTER: {
-                                register(req);
+                                register((NioOperationFuture)req0);
 
                                 break;
                             }
 
-                            case REQUIRE_WRITE: {
-                                //Just register write key.
-                                SelectionKey key = req.session().key();
+                            case MOVE: {
+                                SessionMoveFuture f = (SessionMoveFuture)req0;
 
-                                if (key.isValid()) {
-                                    key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+                                GridSelectorNioSessionImpl ses = f.session();
+
+                                if (idx == f.toIdx) {
+                                    assert f.movedSocketChannel() != null : f;
+
+                                    boolean add = workerSessions.add(ses);
+
+                                    assert add;
 
-                                    // Update timestamp to protected against false write timeout.
-                                    ((GridNioSessionImpl)key.attachment()).bytesSent(0);
+                                    ses.finishMoveSession(this);
+
+                                    if (idx % 2 == 0)
+                                        readerMoveCnt.incrementAndGet();
+                                    else
+                                        writerMoveCnt.incrementAndGet();
+
+                                    SelectionKey key = f.movedSocketChannel().register(selector,
+                                        SelectionKey.OP_READ | SelectionKey.OP_WRITE,
+                                        ses);
+
+                                    ses.key(key);
+
+                                    ses.procWrite.set(true);
+
+                                    f.onDone(true);
+                                }
+                                else {
+                                    assert f.movedSocketChannel() == null : f;
+
+                                    if (workerSessions.remove(ses)) {
+                                        ses.startMoveSession(this);
+
+                                        SelectionKey key = ses.key();
+
+                                        assert key.channel() != null : key;
+
+                                        f.movedSocketChannel((SocketChannel)key.channel());
+
+                                        key.cancel();
+
+                                        clientWorkers.get(f.toIndex()).offer(f);
+                                    }
+                                    else
+                                        f.onDone(false);
                                 }
 
                                 break;
                             }
 
+                            case REQUIRE_WRITE: {
+                                SessionWriteRequest req = (SessionWriteRequest)req0;
+
+                                registerWrite((GridSelectorNioSessionImpl)req.session());
+
+                                break;
+                            }
+
                             case CLOSE: {
+                                NioOperationFuture req = (NioOperationFuture)req0;
+
                                 if (close(req.session(), null))
                                     req.onDone(true);
                                 else
@@ -1408,6 +1745,8 @@ public class GridNioServer<T> {
                             }
 
                             case PAUSE_READ: {
+                                NioOperationFuture req = (NioOperationFuture)req0;
+
                                 SelectionKey key = req.session().key();
 
                                 if (key.isValid()) {
@@ -1426,6 +1765,8 @@ public class GridNioServer<T> {
                             }
 
                             case RESUME_READ: {
+                                NioOperationFuture req = (NioOperationFuture)req0;
+
                                 SelectionKey key = req.session().key();
 
                                 if (key.isValid()) {
@@ -1444,76 +1785,66 @@ public class GridNioServer<T> {
                             }
 
                             case DUMP_STATS: {
-                                StringBuilder sb = new StringBuilder();
+                                NioOperationFuture req = (NioOperationFuture)req0;
 
-                                Set<SelectionKey> keys = selector.keys();
-
-                                sb.append(U.nl())
-                                    .append(">> Selector info [idx=").append(idx)
-                                    .append(", keysCnt=").append(keys.size())
-                                    .append("]").append(U.nl());
-
-                                for (SelectionKey key : keys) {
-                                    GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
-
-                                    MessageWriter writer = ses.meta(MSG_WRITER.ordinal());
-                                    MessageReader reader = ses.meta(GridDirectParser.READER_META_KEY);
-
-                                    sb.append("    Connection info [")
-                                        .append("rmtAddr=").append(ses.remoteAddress())
-                                        .append(", locAddr=").append(ses.localAddress());
-
-                                    GridNioRecoveryDescriptor desc = ses.recoveryDescriptor();
+                                try {
+                                    dumpStats();
+                                }
+                                finally {
+                                    // Complete the request just in case (none should wait on this future).
+                                    req.onDone(true);
+                                }
+                            }
+                        }
+                    }
 
-                                    if (desc != null) {
-                                        sb.append(", msgsSent=").append(desc.sent())
-                                            .append(", msgsAckedByRmt=").append(desc.acked())
-                                            .append(", msgsRcvd=").append(desc.received())
-                                            .append(", descIdHash=").append(System.identityHashCode(desc));
-                                    }
-                                    else
-                                        sb.append(", recoveryDesc=null");
+                    int res = 0;
 
-                                    sb.append(", bytesRcvd=").append(ses.bytesReceived())
-                                        .append(", bytesSent=").append(ses.bytesSent())
-                                        .append(", opQueueSize=").append(ses.writeQueueSize())
-                                        .append(", msgWriter=").append(writer != null ? writer.toString() : "null")
-                                        .append(", msgReader=").append(reader != null ? reader.toString() : "null");
+                    for (long i = 0; i < selectorSpins && res == 0; i++) {
+                        res = selector.selectNow();
 
-                                    int cnt = 0;
+                        if (res > 0) {
+                            // Walk through the ready keys collection and process network events.
+                            if (selectedKeys == null)
+                                processSelectedKeys(selector.selectedKeys());
+                            else
+                                processSelectedKeysOptimized(selectedKeys.flip());
+                        }
 
-                                    for (GridNioFuture<?> fut : ses.writeQueue()) {
-                                        if (cnt == 0)
-                                            sb.append(",\n opQueue=[").append(fut);
-                                        else
-                                            sb.append(',').append(fut);
+                        if (!changeReqs.isEmpty())
+                            continue mainLoop;
 
-                                        if (++cnt == 5) {
-                                            sb.append(']');
+                        // Just in case we do busy selects.
+                        long now = U.currentTimeMillis();
 
-                                            break;
-                                        }
-                                    }
+                        if (now - lastIdleCheck > 2000) {
+                            lastIdleCheck = now;
 
+                            checkIdle(selector.keys());
+                        }
 
-                                    sb.append("]").append(U.nl());
-                                }
+                        if (isCancelled())
+                            return;
+                    }
 
-                                U.warn(log, sb.toString());
+                    // Falling to blocking select.
+                    select = true;
 
-                                // Complete the request just in case (none should wait on this future).
-                                req.onDone(true);
-                            }
+                    try {
+                        if (!changeReqs.isEmpty())
+                            continue;
+
+                        // Wake up every 2 seconds to check if closed.
+                        if (selector.select(2000) > 0) {
+                            // Walk through the ready keys collection and process network events.
+                            if (selectedKeys == null)
+                                processSelectedKeys(selector.selectedKeys());
+                            else
+                                processSelectedKeysOptimized(selectedKeys.flip());
                         }
                     }
-
-                    // Wake up every 2 seconds to check if closed.
-                    if (selector.select(2000) > 0) {
-                        // Walk through the ready keys collection and process network events.
-                        if (selectedKeys == null)
-                            processSelectedKeys(selector.selectedKeys());
-                        else
-                            processSelectedKeysOptimized(selectedKeys.flip());
+                    finally {
+                        select = false;
                     }
 
                     long now = U.currentTimeMillis();
@@ -1554,6 +1885,98 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @param ses Session.
+         */
+        public final void registerWrite(GridSelectorNioSessionImpl ses) {
+            SelectionKey key = ses.key();
+
+            if (key.isValid()) {
+                if ((key.interestOps() & SelectionKey.OP_WRITE) == 0)
+                    key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+
+                // Update timestamp to protected against false write timeout.
+                ses.bytesSent(0);
+            }
+        }
+
+        /**
+         *
+         */
+        private void dumpStats() {
+            StringBuilder sb = new StringBuilder();
+
+            Set<SelectionKey> keys = selector.keys();
+
+            sb.append(U.nl())
+                .append(">> Selector info [idx=").append(idx)
+                .append(", keysCnt=").append(keys.size())
+                .append(", bytesRcvd=").append(bytesRcvd)
+                .append(", bytesRcvd0=").append(bytesRcvd0)
+                .append(", bytesSent=").append(bytesSent)
+                .append(", bytesSent0=").append(bytesSent0)
+                .append("]").append(U.nl());
+
+            for (SelectionKey key : keys) {
+                GridSelectorNioSessionImpl ses = (GridSelectorNioSessionImpl)key.attachment();
+
+                MessageWriter writer = ses.meta(MSG_WRITER.ordinal());
+                MessageReader reader = ses.meta(GridDirectParser.READER_META_KEY);
+
+                sb.append("    Connection info [")
+                    .append("in=").append(ses.accepted())
+                    .append(", rmtAddr=").append(ses.remoteAddress())
+                    .append(", locAddr=").append(ses.localAddress());
+
+                GridNioRecoveryDescriptor outDesc = ses.outRecoveryDescriptor();
+
+                if (outDesc != null) {
+                    sb.append(", msgsSent=").append(outDesc.sent())
+                        .append(", msgsAckedByRmt=").append(outDesc.acked())
+                        .append(", descIdHash=").append(System.identityHashCode(outDesc));
+                }
+                else
+                    sb.append(", outRecoveryDesc=null");
+
+                GridNioRecoveryDescriptor inDesc = ses.inRecoveryDescriptor();
+
+                if (inDesc != null) {
+                    sb.append(", msgsRcvd=").append(inDesc.received())
+                        .append(", lastAcked=").append(inDesc.lastAcknowledged())
+                        .append(", descIdHash=").append(System.identityHashCode(inDesc));
+                }
+                else
+                    sb.append(", inRecoveryDesc=null");
+
+                sb.append(", bytesRcvd=").append(ses.bytesReceived())
+                    .append(", bytesRcvd0=").append(ses.bytesReceived0())
+                    .append(", bytesSent=").append(ses.bytesSent())
+                    .append(", bytesSent0=").append(ses.bytesSent0())
+                    .append(", opQueueSize=").append(ses.writeQueueSize())
+                    .append(", msgWriter=").append(writer != null ? writer.toString() : "null")
+                    .append(", msgReader=").append(reader != null ? reader.toString() : "null");
+
+                int cnt = 0;
+
+                for (SessionWriteRequest req : ses.writeQueue()) {
+                    if (cnt == 0)
+                        sb.append(",\n opQueue=[").append(req);
+                    else
+                        sb.append(',').append(req);
+
+                    if (++cnt == 5) {
+                        sb.append(']');
+
+                        break;
+                    }
+                }
+
+                sb.append("]").append(U.nl());
+            }
+
+            U.warn(log, sb.toString());
+        }
+
+        /**
          * Processes keys selected by a selector.
          *
          * @param keys Selected keys.
@@ -1671,7 +2094,9 @@ public class GridNioServer<T> {
 
                     long idleTimeout0 = idleTimeout;
 
-                    if (!opWrite && now - ses.lastReceiveTime() > idleTimeout0 && now - ses.lastSendScheduleTime() > idleTimeout0) {
+                    if (!opWrite &&
+                        now - ses.lastReceiveTime() > idleTimeout0 &&
+                        now - ses.lastSendScheduleTime() > idleTimeout0) {
                         filterChain.onSessionIdleTimeout(ses);
 
                         // Update timestamp to avoid multiple notifications within one timeout interval.
@@ -1715,7 +2140,7 @@ public class GridNioServer<T> {
 
                 final GridSelectorNioSessionImpl ses = new GridSelectorNioSessionImpl(
                     log,
-                    idx,
+                    this,
                     filterChain,
                     (InetSocketAddress)sockCh.getLocalAddress(),
                     (InetSocketAddress)sockCh.getRemoteAddress(),
@@ -1739,6 +2164,7 @@ public class GridNioServer<T> {
                     resend(ses);
 
                 sessions.add(ses);
+                workerSessions.add(ses);
 
                 try {
                     filterChain.onSessionOpened(ses);
@@ -1764,7 +2190,7 @@ public class GridNioServer<T> {
         }
 
         /**
-         * Closes the ses and all associated resources, then notifies the listener.
+         * Closes the session and all associated resources, then notifies the listener.
          *
          * @param ses Session to be closed.
          * @param e Exception to be passed to the listener, if any.
@@ -1781,12 +2207,10 @@ public class GridNioServer<T> {
             }
 
             sessions.remove(ses);
+            workerSessions.remove(ses);
 
             SelectionKey key = ses.key();
 
-            // Shutdown input and output so that remote client will see correct socket close.
-            Socket sock = ((SocketChannel)key.channel()).socket();
-
             if (ses.setClosed()) {
                 ses.onClosed();
 
@@ -1798,6 +2222,9 @@ public class GridNioServer<T> {
                         ((DirectBuffer)ses.readBuffer()).cleaner().clean();
                 }
 
+                // Shutdown input and output so that remote client will see correct socket close.
+                Socket sock = ((SocketChannel)key.channel()).socket();
+
                 try {
                     try {
                         sock.shutdownInput();
@@ -1824,28 +2251,35 @@ public class GridNioServer<T> {
                 ses.removeMeta(BUF_META_KEY);
 
                 // Since ses is in closed state, no write requests will be added.
-                NioOperationFuture<?> fut = ses.removeMeta(NIO_OPERATION.ordinal());
+                SessionWriteRequest req = ses.removeMeta(NIO_OPERATION.ordinal());
+
+                GridNioRecoveryDescriptor outRecovery = ses.outRecoveryDescriptor();
+                GridNioRecoveryDescriptor inRecovery = ses.inRecoveryDescriptor();
 
-                GridNioRecoveryDescriptor recovery = ses.recoveryDescriptor();
+                IOException err = new IOException("Failed to send message (connection was closed): " + ses);
 
-                if (recovery != null) {
+                if (outRecovery != null || inRecovery != null) {
                     try {
                         // Poll will update recovery data.
-                        while ((fut = (NioOperationFuture<?>)ses.pollFuture()) != null) {
-                            if (fut.skipRecovery())
-                                fut.connectionClosed();
+                        while ((req = ses.pollFuture()) != null) {
+                            if (req.skipRecovery())
+                                req.onError(err);
                         }
                     }
                     finally {
-                        recovery.release();
+                        if (outRecovery != null)
+                            outRecovery.release();
+
+                        if (inRecovery != null && inRecovery != outRecovery)
+                            inRecovery.release();
                     }
                 }
                 else {
-                    if (fut != null)
-                        fut.connectionClosed();
+                    if (req != null)
+                        req.onError(err);
 
-                    while ((fut = (NioOperationFuture<?>)ses.pollFuture()) != null)
-                        fut.connectionClosed();
+                    while ((req = ses.pollFuture()) != null)
+                        req.onError(err);
                 }
 
                 try {
@@ -1876,12 +2310,44 @@ public class GridNioServer<T> {
          * @throws IOException If write failed.
          */
         protected abstract void processWrite(SelectionKey key) throws IOException;
-    }
 
-    /**
-     * Gets outbound messages queue size.
-     *
-     * @return Write queue size.
+        /**
+         * @param cnt
+         */
+        final void onRead(int cnt) {
+            bytesRcvd += cnt;
+            bytesRcvd0 += cnt;
+        }
+
+        /**
+         * @param cnt
+         */
+        final void onWrite(int cnt) {
+            bytesSent += cnt;
+            bytesSent0 += cnt;
+        }
+
+        /**
+         *
+         */
+        final void reset0() {
+            bytesSent0 = 0;
+            bytesRcvd0 = 0;
+
+            for (GridSelectorNioSessionImpl ses : workerSessions)
+                ses.reset0();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(AbstractNioClientWorker.class, this, super.toString());
+        }
+    }
+
+    /**
+     * Gets outbound messages queue size.
+     *
+     * @return Write queue size.
      */
     public int outboundMessagesQueueSize() {
         int res = 0;
@@ -1952,6 +2418,9 @@ public class GridNioServer<T> {
                     if (selector.select(2000) > 0)
                         // Walk through the ready keys collection and process date requests.
                         processSelectedKeys(selector.selectedKeys());
+
+                    if (balancer != null)
+                        balancer.run();
                 }
             }
             // Ignore this exception as thread interruption is equal to 'close' call.
@@ -2048,10 +2517,13 @@ public class GridNioServer<T> {
     /**
      * Asynchronous operation that may be requested on selector.
      */
-    private enum NioOperation {
+    enum NioOperation {
         /** Register read key selection. */
         REGISTER,
 
+        /** Move session between workers. */
+        MOVE,
+
         /** Register write key selection. */
         REQUIRE_WRITE,
 
@@ -2069,9 +2541,193 @@ public class GridNioServer<T> {
     }
 
     /**
+     *
+     */
+    private static final class WriteRequestSystemImpl implements SessionWriteRequest, SessionChangeRequest {
+        /** */
+        private final Object msg;
+
+        /** */
+        private final GridNioSession ses;
+
+        /**
+         * @param ses Session.
+         * @param msg Message.
+         */
+        WriteRequestSystemImpl(GridNioSession ses, Object msg) {
+            this.ses = ses;
+            this.msg = msg;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void messageThread(boolean msgThread) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean messageThread() {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean skipRecovery() {
+            return true;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void ackClosure(IgniteInClosure<IgniteException> c) {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteInClosure<IgniteException> ackClosure() {
+            return null;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onError(Exception e) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object message() {
+            return msg;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessageWritten() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void resetSession(GridNioSession ses) {
+            throw new UnsupportedOperationException();
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioSession session() {
+            return ses;
+        }
+
+        /** {@inheritDoc} */
+        @Override public NioOperation operation() {
+            return NioOperation.REQUIRE_WRITE;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(WriteRequestSystemImpl.class, this);
+        }
+    }
+
+    /**
+     *
+     */
+    private static final class WriteRequestImpl implements SessionWriteRequest, SessionChangeRequest {
+        /** */
+        private GridNioSession ses;
+
+        /** */
+        private final Object msg;
+
+        /** */
+        private boolean msgThread;
+
+        /** */
+        private final boolean skipRecovery;
+
+        /** */
+        private IgniteInClosure<IgniteException> ackC;
+
+        /**
+         * @param ses Session.
+         * @param msg Message.
+         * @param skipRecovery Skip recovery flag.
+         */
+        WriteRequestImpl(GridNioSession ses, Object msg, boolean skipRecovery) {
+            this.ses = ses;
+            this.msg = msg;
+            this.skipRecovery = skipRecovery;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void messageThread(boolean msgThread) {
+            this.msgThread = msgThread;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean messageThread() {
+            return msgThread;
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean skipRecovery() {
+            return skipRecovery;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void ackClosure(IgniteInClosure<IgniteException> c) {
+            ackC = c;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onAckReceived() {
+            assert msg instanceof Message;
+
+            ((Message)msg).onAckReceived();
+        }
+
+        /** {@inheritDoc} */
+        @Override public IgniteInClosure<IgniteException> ackClosure() {
+            return ackC;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onError(Exception e) {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public Object message() {
+            return msg;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessageWritten() {
+            // No-op.
+        }
+
+        /** {@inheritDoc} */
+        @Override public void resetSession(GridNioSession ses) {
+            this.ses = ses;
+        }
+
+        /** {@inheritDoc} */
+        @Override public GridNioSession session() {
+            return ses;
+        }
+
+        /** {@inheritDoc} */
+        @Override public NioOperation operation() {
+            return NioOperation.REQUIRE_WRITE;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(WriteRequestImpl.class, this);
+        }
+    }
+
+    /**
      * Class for requesting write and session close operations.
      */
-    private static class NioOperationFuture<R> extends GridNioFutureImpl<R> {
+    private static class NioOperationFuture<R> extends GridNioFutureImpl<R> implements SessionWriteRequest,
+        SessionChangeRequest {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2087,11 +2743,7 @@ public class GridNioServer<T> {
         private NioOperation op;
 
         /** Message. */
-        @GridToStringExclude
-        private ByteBuffer msg;
-
-        /** Direct message. */
-        private Message commMsg;
+        private Object msg;
 
         /** */
         @GridToStringExclude
@@ -2153,8 +2805,7 @@ public class GridNioServer<T> {
          * @param op Requested operation.
          * @param msg Message.
          */
-        NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op,
-            ByteBuffer msg) {
+        NioOperationFuture(GridSelectorNioSessionImpl ses, NioOperation op, Object msg) {
             assert ses != null;
             assert op != null;
             assert op != NioOperation.REGISTER;
@@ -2182,51 +2833,36 @@ public class GridNioServer<T> {
 
             this.ses = ses;
             this.op = op;
-            this.commMsg = commMsg;
+            this.msg = commMsg;
             this.skipRecovery = skipRecovery;
         }
 
-        /**
-         * @return Requested change operation.
-         */
-        private NioOperation operation() {
+        /** {@inheritDoc} */
+        public NioOperation operation() {
             return op;
         }
 
-        /**
-         * @return Message.
-         */
-        private ByteBuffer message() {
+        /** {@inheritDoc} */
+        public Object message() {
             return msg;
         }
 
-        /**
-         * @return Direct message.
-         */
-        private Message directMessage() {
-            return commMsg;
-        }
-
-        /**
-         * @param ses New session instance.
-         */
-        private void resetSession(GridSelectorNioSessionImpl ses) {
-            assert commMsg != null;
+        /** {@inheritDoc} */
+        public void resetSession(GridNioSession ses) {
+            assert msg instanceof Message : msg;
 
-            this.ses = ses;
+            this.ses = (GridSelectorNioSessionImpl)ses;
         }
 
         /**
          * @return Socket channel for register request.
          */
-        private SocketChannel socketChannel() {
+        SocketChannel socketChannel() {
             return sockCh;
         }
 
-        /**
-         * @return Session for this change request.
-         */
-        private GridSelectorNioSessionImpl session() {
+        /** {@inheritDoc} */
+        public GridSelectorNioSessionImpl session() {
             return ses;
         }
 
@@ -2244,21 +2880,21 @@ public class GridNioServer<T> {
             return meta;
         }
 
-        /**
-         * Applicable to write futures only. Fails future with corresponding IOException.
-         */
-        private void connectionClosed() {
-            assert op == NioOperation.REQUIRE_WRITE;
-            assert ses != null;
-
-            onDone(new IOException("Failed to send message (connection was closed): " + ses));
+        /** {@inheritDoc} */
+        @Override public void onError(Exception e) {
+            onDone(e);
         }
 
         /** {@inheritDoc} */
         @Override public void onAckReceived() {
-            assert commMsg != null;
+            assert msg instanceof Message : msg;
 
-            commMsg.onAckReceived();
+            ((Message)msg).onAckReceived();
+        }
+
+        /** {@inheritDoc} */
+        @Override public void onMessageWritten() {
+            onDone();
         }
 
         /** {@inheritDoc} */
@@ -2273,6 +2909,59 @@ public class GridNioServer<T> {
     }
 
     /**
+     *
+     */
+    private static class SessionMoveFuture extends NioOperationFuture<Boolean> {
+        /** */
+        private final int toIdx;
+
+        /** */
+        @GridToStringExclude
+        private SocketChannel movedSockCh;
+
+        /**
+         * @param ses Session.
+         * @param toIdx Target worker index.
+         */
+        SessionMoveFuture(
+            GridSelectorNioSessionImpl ses,
+            int toIdx
+        ) {
+            super(ses, NioOperation.MOVE);
+
+            this.toIdx = toIdx;
+        }
+
+        /**
+         * @return Target worker index.
+         */
+        int toIndex() {
+            return toIdx;
+        }
+
+        /**
+         * @return Moved session socket channel.
+         */
+        SocketChannel movedSocketChannel() {
+            return movedSockCh;
+        }
+
+        /**
+         * @param movedSockCh Moved session socket channel.
+         */
+        void movedSocketChannel(SocketChannel movedSockCh) {
+            assert movedSockCh != null;
+
+            this.movedSockCh = movedSockCh;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(SessionMoveFuture.class, this, super.toString());
+        }
+    }
+
+    /**
      * Filter forwarding messages from chain's head to this server.
      */
     private class HeadFilter extends GridNioFilterAdapter {
@@ -2302,7 +2991,7 @@ public class GridNioServer<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
             if (directMode) {
                 boolean sslSys = sslFilter != null && msg instanceof ByteBuffer;
 
@@ -2313,18 +3002,18 @@ public class GridNioServer<T> {
 
                     queue.offer((ByteBuffer)msg);
 
-                    SelectionKey key = ((GridSelectorNioSessionImpl)ses).key();
+                    GridSelectorNioSessionImpl ses0 = (GridSelectorNioSessionImpl)ses;
 
-                    if (key.isValid())
-                        key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+                    if (!ses0.procWrite.get() && ses0.procWrite.compareAndSet(false, true))
+                        ses0.worker().registerWrite(ses0);
 
                     return null;
                 }
                 else
-                    return send(ses, (Message)msg);
+                    return send(ses, (Message)msg, fut);
             }
             else
-                return send(ses, (ByteBuffer)msg);
+                return send(ses, (ByteBuffer)msg, fut);
         }
 
         /** {@inheritDoc} */
@@ -2429,6 +3118,15 @@ public class GridNioServer<T> {
         /** Message queue size listener. */
         private IgniteBiInClosure<GridNioSession, Integer> msgQueueLsnr;
 
+        /** Name for threads identification. */
+        private String srvName;
+
+        /** */
+        private long selectorSpins;
+
+        /** NIO sessions balancing flag. */
+        private boolean balancing;
+
         /**
          * Finishes building the instance.
          *
@@ -2442,6 +3140,8 @@ public class GridNioServer<T> {
                 log,
                 selectorCnt,
                 gridName,
+                srvName,
+                selectorSpins,
                 tcpNoDelay,
                 directBuf,
                 byteOrder,
@@ -2455,6 +3155,7 @@ public class GridNioServer<T> {
                 writerFactory,
                 skipRecoveryPred,
                 msgQueueLsnr,
+                balancing,
                 filters != null ? Arrays.copyOf(filters, filters.length) : EMPTY_FILTERS
             );
 
@@ -2468,6 +3169,16 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @param balancing NIO sessions balancing flag.
+         * @return This for chaining.
+         */
+        public Builder<T> balancing(boolean balancing) {
+            this.balancing = balancing;
+
+            return this;
+        }
+
+        /**
          * @param addr Local address.
          * @return This for chaining.
          */
@@ -2519,6 +3230,28 @@ public class GridNioServer<T> {
         }
 
         /**
+         * @param srvName Logical server name for threads identification.
+         * @return This for chaining.
+         */
+        public Builder<T> serverName(@Nullable String srvName) {
+            this.srvName = srvName;
+
+            return this;
+        }
+
+        /**
+         * @param selectorSpins Defines how many non-blocking {@code selector.selectNow()} should be made before
+         *      falling into {@code selector.select(long)} in NIO server. Long value. Default is {@code 0}.
+         *      Can be set to {@code Long.MAX_VALUE} so selector threads will never block.
+         * @return This for chaining.
+         */
+        public Builder<T> selectorSpins(long selectorSpins) {
+            this.selectorSpins = selectorSpins;
+
+            return this;
+        }
+
+        /**
          * @param tcpNoDelay If TCP_NODELAY option should be set to accepted sockets.
          * @return This for chaining.
          */
@@ -2678,4 +3411,225 @@ public class GridNioServer<T> {
             return this;
         }
     }
+
+    /**
+     *
+     */
+    private class SizeBasedBalancer implements IgniteRunnable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** */
+        private long lastBalance;
+
+        /** */
+        private final long balancePeriod;
+
+        /**
+         * @param balancePeriod Period.
+         */
+        SizeBasedBalancer(long balancePeriod) {
+            this.balancePeriod = balancePeriod;
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            long now = U.currentTimeMillis();
+
+            if (lastBalance + balancePeriod < now) {
+                lastBalance = now;
+
+                long maxRcvd0 = -1, minRcvd0 = -1, maxSent0 = -1, minSent0 = -1;
+                int maxRcvdIdx = -1, minRcvdIdx = -1, maxSentIdx = -1, minSentIdx = -1;
+
+                for (int i = 0; i < clientWorkers.size(); i++) {
+                    GridNioServer.AbstractNioClientWorker worker = clientWorkers.get(i);
+
+                    int sesCnt = worker.workerSessions.size();
+
+                    if (i % 2 == 0) {
+                        // Reader.
+                        long bytesRcvd0 = worker.bytesRcvd0;
+
+                        if ((maxRcvd0 == -1 || bytesRcvd0 > maxRcvd0) && bytesRcvd0 > 0 && sesCnt > 1) {
+                            maxRcvd0 = bytesRcvd0;
+                            maxRcvdIdx = i;
+                        }
+
+                        if (minRcvd0 == -1 || bytesRcvd0 < minRcvd0) {
+                            minRcvd0 = bytesRcvd0;
+                            minRcvdIdx = i;
+                        }
+                    }
+                    else {
+                        // Writer.
+                        long bytesSent0 = worker.bytesSent0;
+
+                        if ((maxSent0 == -1 || bytesSent0 > maxSent0) && bytesSent0 > 0 && sesCnt > 1) {
+                            maxSent0 = bytesSent0;
+                            maxSentIdx = i;
+                        }
+
+                        if (minSent0 == -1 || bytesSent0 < minSent0) {
+                            minSent0 = bytesSent0;
+                            minSentIdx = i;
+                        }
+                    }
+                }
+
+                if (log.isDebugEnabled())
+                    log.debug("Balancing data [minSent0=" + minSent0 + ", minSentIdx=" + minSentIdx +
+                        ", maxSent0=" + maxSent0 + ", maxSentIdx=" + maxSentIdx +
+                        ", minRcvd0=" + minRcvd0 + ", minRcvdIdx=" + minRcvdIdx +
+                        ", maxRcvd0=" + maxRcvd0 + ", maxRcvdIdx=" + maxRcvdIdx + ']');
+
+                if (maxSent0 != -1 && minSent0 != -1) {
+                    GridSelectorNioSessionImpl ses = null;
+
+                    long sentDiff = maxSent0 - minSent0;
+                    long delta = sentDiff;
+                    double threshold = sentDiff * 0.9;
+
+                    GridConcurrentHashSet<GridSelectorNioSessionImpl> sessions =
+                        clientWorkers.get(maxSentIdx).workerSessions;
+
+                    for (GridSelectorNioSessionImpl ses0 : sessions) {
+                        long bytesSent0 = ses0.bytesSent0();
+
+                        if (bytesSent0 < threshold &&
+                            (ses == null || delta > U.safeAbs(bytesSent0 - sentDiff / 2))) {
+                            ses = ses0;
+                            delta = U.safeAbs(bytesSent0 - sentDiff / 2);
+                        }
+                    }
+
+                    if (ses != null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Will move session to less loaded writer [ses=" + ses +
+                                ", from=" + maxSentIdx + ", to=" + minSentIdx + ']');
+
+                        moveSession(ses, maxSentIdx, minSentIdx);
+                    }
+                    else {
+                        if (log.isDebugEnabled())
+                            log.debug("Unable to find session to move for writers.");
+                    }
+                }
+
+                if (maxRcvd0 != -1 && minRcvd0 != -1) {
+                    GridSelectorNioSessionImpl ses = null;
+
+                    long rcvdDiff = maxRcvd0 - minRcvd0;
+                    long delta = rcvdDiff;
+                    double threshold = rcvdDiff * 0.9;
+
+                    GridConcurrentHashSet<GridSelectorNioSessionImpl> sessions =
+                        clientWorkers.get(maxRcvdIdx).workerSessions;
+
+                    for (GridSelectorNioSessionImpl ses0 : sessions) {
+                        long bytesRcvd0 = ses0.bytesReceived0();
+
+                        if (bytesRcvd0 < threshold &&
+                            (ses == null || delta > U.safeAbs(bytesRcvd0 - rcvdDiff / 2))) {
+                            ses = ses0;
+                            delta = U.safeAbs(bytesRcvd0 - rcvdDiff / 2);
+                        }
+                    }
+
+                    if (ses != null) {
+                        if (log.isDebugEnabled())
+                            log.debug("Will move session to less loaded reader [ses=" + ses +
+                                ", from=" + maxRcvdIdx + ", to=" + minRcvdIdx + ']');
+
+                        moveSession(ses, maxRcvdIdx, minRcvdIdx);
+                    }
+                    else {
+                        if (log.isDebugEnabled())
+                            log.debug("Unable to find session to move for readers.");
+                    }
+                }
+
+                for (int i = 0; i < clientWorkers.size(); i++) {
+                    GridNioServer.AbstractNioClientWorker worker = clientWorkers.get(i);
+
+                    worker.reset0();
+                }
+            }
+        }
+    }
+
+    /**
+     * For tests only.
+     */
+    @SuppressWarnings("unchecked")
+    private class RandomBalancer implements IgniteRunnable {
+        /** */
+        private static final long serialVersionUID = 0L;
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            ThreadLocalRandom rnd = ThreadLocalRandom.current();
+
+            int w1 = rnd.nextInt(clientWorkers.size());
+
+            if (clientWorkers.get(w1).workerSessions.isEmpty())
+                return;
+
+            int w2 = rnd.nextInt(clientWorkers.size());
+
+            while (w2 == w1)
+                w2 = rnd.nextInt(clientWorkers.size());
+
+            GridNioSession ses = randomSession(clientWorkers.get(w1));
+
+            if (ses != null) {
+                log.info("Move session [from=" + w1 +
+                    ", to=" + w2 +
+                    ", ses=" + ses + ']');
+
+                moveSession(ses, w1, w2);
+            }
+        }
+
+        /**
+         * @param worker Worker.
+         * @return NIO session.
+         */
+        private GridNioSession randomSession(GridNioServer.AbstractNioClientWorker worker) {
+            Collection<GridNioSession> sessions = worker.workerSessions;
+
+            int size = sessions.size();
+
+            if (size == 0)
+                return null;
+
+            int idx = ThreadLocalRandom.current().nextInt(size);
+
+            Iterator<GridNioSession> it = sessions.iterator();
+
+            int cnt = 0;
+
+            while (it.hasNext()) {
+                GridNioSession ses = it.next();
+
+                if (cnt == idx)
+                    return ses;
+            }
+
+            return null;
+        }
+
+    }
+
+    /**
+     *
+     */
+    interface SessionChangeRequest {
+        GridNioSession session();
+
+        /**
+         * @return Requested change operation.
+         */
+        NioOperation operation();
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
index e4a7225..c1b60ab 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSession.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.nio;
 
 import java.net.InetSocketAddress;
+import org.apache.ignite.IgniteCheckedException;
 import org.jetbrains.annotations.Nullable;
 
 /**
@@ -105,6 +106,11 @@ public interface GridNioSession {
     public GridNioFuture<?> send(Object msg);
 
     /**
+     * @param msg Message to be sent.
+     */
+    public void sendNoFuture(Object msg) throws IgniteCheckedException;
+
+    /**
      * Gets metadata associated with specified key.
      *
      * @param key Key to look up.
@@ -158,10 +164,25 @@ public interface GridNioSession {
     /**
      * @param recoveryDesc Recovery descriptor.
      */
-    public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
+    public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
+
+    /**
+     * @param recoveryDesc Recovery descriptor.
+     */
+    public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc);
 
     /**
      * @return Recovery descriptor if recovery is supported, {@code null otherwise.}
      */
-    @Nullable public GridNioRecoveryDescriptor recoveryDescriptor();
+    @Nullable public GridNioRecoveryDescriptor outRecoveryDescriptor();
+
+    /**
+     * @return Recovery descriptor if recovery is supported, {@code null otherwise.}
+     */
+    @Nullable public GridNioRecoveryDescriptor inRecoveryDescriptor();
+
+    /**
+     * @param msg System message to send.
+     */
+    public void systemMessage(Object msg);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
index 0bcfe64..7424531 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioSessionImpl.java
@@ -51,6 +51,12 @@ public class GridNioSessionImpl implements GridNioSession {
     /** Received bytes counter. */
     private volatile long bytesRcvd;
 
+    /** Sent bytes since last NIO sessions balancing. */
+    private volatile long bytesSent0;
+
+    /** Received bytes since last NIO sessions balancing. */
+    private volatile long bytesRcvd0;
+
     /** Last send schedule timestamp. */
     private volatile long sndSchedTime;
 
@@ -99,7 +105,7 @@ public class GridNioSessionImpl implements GridNioSession {
         try {
             resetSendScheduleTime();
 
-            return chain().onSessionWrite(this, msg);
+            return chain().onSessionWrite(this, msg, true);
         }
         catch (IgniteCheckedException e) {
             close();
@@ -109,6 +115,18 @@ public class GridNioSessionImpl implements GridNioSession {
     }
 
     /** {@inheritDoc} */
+    @Override public void sendNoFuture(Object msg) throws IgniteCheckedException {
+        try {
+            chain().onSessionWrite(this, msg, false);
+        }
+        catch (IgniteCheckedException e) {
+            close();
+
+            throw e;
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public GridNioFuture<?> resumeReads() {
         try {
             return chain().onResumeReads(this);
@@ -163,6 +181,28 @@ public class GridNioSessionImpl implements GridNioSession {
         return bytesRcvd;
     }
 
+    /**
+     * @return Sent bytes since last NIO sessions balancing.
+     */
+    public long bytesSent0() {
+        return bytesSent0;
+    }
+
+    /**
+     * @return Received bytes since last NIO sessions balancing.
+     */
+    public long bytesReceived0() {
+        return bytesRcvd0;
+    }
+
+    /**
+     *
+     */
+    public void reset0() {
+        bytesSent0 = 0;
+        bytesRcvd0 = 0;
+    }
+
     /** {@inheritDoc} */
     @Override public long createTime() {
         return createTime;
@@ -240,6 +280,7 @@ public class GridNioSessionImpl implements GridNioSession {
      */
     public void bytesSent(int cnt) {
         bytesSent += cnt;
+        bytesSent0 += cnt;
 
         lastSndTime = U.currentTimeMillis();
     }
@@ -253,6 +294,7 @@ public class GridNioSessionImpl implements GridNioSession {
      */
     public void bytesReceived(int cnt) {
         bytesRcvd += cnt;
+        bytesRcvd0 += cnt;
 
         lastRcvTime = U.currentTimeMillis();
     }
@@ -296,17 +338,32 @@ public class GridNioSessionImpl implements GridNioSession {
     }
 
     /** {@inheritDoc} */
-    @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+    @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
         throw new UnsupportedOperationException();
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+    @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
         return null;
     }
 
     /** {@inheritDoc} */
+    @Override public void systemMessage(Object msg) {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridNioSessionImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java
new file mode 100644
index 0000000..62985ff
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioWorker.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util.nio;
+
+import java.util.Collection;
+import java.util.List;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ *
+ */
+interface GridNioWorker {
+    /**
+     * @param req Change request.
+     */
+    public void offer(GridNioServer.SessionChangeRequest req);
+
+    /**
+     * @param reqs Change requests.
+     */
+    public void offer(Collection<GridNioServer.SessionChangeRequest> reqs);
+
+    /**
+     * @param ses Session.
+     * @return Session state change requests.
+     */
+    @Nullable public List<GridNioServer.SessionChangeRequest> clearSessionRequests(GridNioSession ses);
+
+    /**
+     * @param ses Session to register write interest for.
+     */
+    public void registerWrite(GridSelectorNioSessionImpl ses);
+}


[21/25] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NIO sessio

Posted by sb...@apache.org.
IGNITE-3220 I/O bottleneck on server/client cluster configuration
Communications optimizations:
- possibility to open separate in/out connections
- possibility to have multiple connections between nodes
- implemented NIO sessions balancing between NIO threads
- reduced amount of work and blocking calls in NIO threads
Other:
- implemented StripedExecutor for cache messages handling
- added 'io test' messages for IO performance testing


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

Branch: refs/heads/ignite-4371
Commit: 10ade2849880790a0fc05a9528ca9b0e1e4617ad
Parents: 065ca4a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 9 12:28:47 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 9 12:28:50 2016 +0300

----------------------------------------------------------------------
 .../ignite/examples/ExampleNodeStartup.java     |    2 +-
 .../examples/datagrid/CachePutGetExample.java   |    2 +-
 .../scalar/examples/ScalarJvmCloudExample.scala |    2 +-
 .../rest/ClientMemcachedProtocolSelfTest.java   |    4 +-
 .../rest/protocols/tcp/MockNioSession.java      |   25 +-
 .../apache/ignite/IgniteSystemProperties.java   |    3 +
 .../cache/store/CacheLoadOnlyStoreAdapter.java  |    6 +-
 .../configuration/IgniteConfiguration.java      |   50 +-
 .../internal/GridEventConsumeHandler.java       |    2 +-
 .../ignite/internal/GridJobContextImpl.java     |    4 +-
 .../ignite/internal/GridKernalContext.java      |    9 +
 .../ignite/internal/GridKernalContextImpl.java  |   16 +-
 .../internal/GridPerformanceSuggestions.java    |    2 +-
 .../org/apache/ignite/internal/GridTopic.java   |    5 +-
 .../ignite/internal/IgniteInternalFuture.java   |   11 +
 .../apache/ignite/internal/IgniteKernal.java    |   85 +-
 .../org/apache/ignite/internal/IgnitionEx.java  |   32 +-
 .../GridClientConnectionManagerAdapter.java     |    1 +
 .../client/router/impl/GridTcpRouterImpl.java   |    1 +
 .../managers/communication/GridIoManager.java   |  207 ++-
 .../managers/communication/GridIoMessage.java   |   13 +
 .../communication/GridIoMessageFactory.java     |   12 +-
 .../communication/IgniteIoTestMessage.java      |  235 +++
 .../processors/cache/GridCacheAdapter.java      |   26 +-
 .../processors/cache/GridCacheMessage.java      |    7 +
 .../processors/cache/GridCacheUtils.java        |   35 +
 .../processors/cache/IgniteCacheProxy.java      |    8 +
 .../distributed/GridDistributedLockRequest.java |    5 +
 .../GridDistributedTxFinishResponse.java        |    6 +
 .../GridDistributedUnlockRequest.java           |    5 +
 .../distributed/dht/GridDhtCacheAdapter.java    |    3 +-
 .../distributed/dht/GridDhtLockResponse.java    |    9 +-
 .../dht/atomic/GridDhtAtomicCache.java          |    5 +-
 .../GridDhtAtomicSingleUpdateRequest.java       |    5 +
 .../dht/atomic/GridDhtAtomicUpdateRequest.java  |    5 +
 .../atomic/GridNearAtomicFullUpdateRequest.java |    5 +
 .../GridNearAtomicSingleUpdateRequest.java      |    5 +
 .../distributed/near/GridNearGetRequest.java    |    5 +
 .../local/atomic/GridLocalAtomicCache.java      |    3 +
 .../query/GridCacheDistributedQueryManager.java |    2 +-
 .../cache/query/GridCacheQueryRequest.java      |    6 +-
 .../transactions/IgniteTxLocalAdapter.java      |    8 +-
 .../datastreamer/DataStreamProcessor.java       |   22 +-
 .../internal/processors/igfs/IgfsContext.java   |    4 +-
 .../processors/igfs/IgfsDataManager.java        |    6 +-
 .../internal/processors/igfs/IgfsImpl.java      |    2 +-
 .../internal/processors/odbc/OdbcProcessor.java |    1 +
 .../platform/compute/PlatformCompute.java       |    6 +
 .../tcp/GridTcpMemcachedNioListener.java        |   15 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |    2 +-
 .../rest/protocols/tcp/GridTcpRestProtocol.java |    1 +
 .../service/GridServiceProcessor.java           |    6 +-
 .../ignite/internal/util/IgniteUtils.java       |   62 +-
 .../ignite/internal/util/StripedExecutor.java   |  667 +++++++++
 .../util/future/GridFinishedFuture.java         |   24 +
 .../internal/util/future/GridFutureAdapter.java |   15 +-
 .../util/future/GridFutureChainListener.java    |   30 +-
 .../internal/util/ipc/IpcToNioAdapter.java      |    2 +-
 .../nio/GridAbstractCommunicationClient.java    |   12 +-
 .../util/nio/GridCommunicationClient.java       |    9 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |   15 +-
 .../util/nio/GridNioAsyncNotifyFilter.java      |   10 +-
 .../internal/util/nio/GridNioCodecFilter.java   |   17 +-
 .../ignite/internal/util/nio/GridNioFilter.java |   16 +-
 .../internal/util/nio/GridNioFilterAdapter.java |   10 +-
 .../internal/util/nio/GridNioFilterChain.java   |   14 +-
 .../ignite/internal/util/nio/GridNioFuture.java |    4 +-
 .../util/nio/GridNioRecoveryDescriptor.java     |  124 +-
 .../ignite/internal/util/nio/GridNioServer.java | 1404 +++++++++++++++---
 .../internal/util/nio/GridNioSession.java       |   25 +-
 .../internal/util/nio/GridNioSessionImpl.java   |   65 +-
 .../ignite/internal/util/nio/GridNioWorker.java |   48 +
 .../util/nio/GridSelectorNioSessionImpl.java    |  221 ++-
 .../util/nio/GridShmemCommunicationClient.java  |    7 +-
 .../util/nio/GridTcpNioCommunicationClient.java |   55 +-
 .../internal/util/nio/SessionWriteRequest.java  |   85 ++
 .../internal/util/nio/ssl/GridNioSslFilter.java |   10 +-
 .../util/nio/ssl/GridNioSslHandler.java         |    4 +-
 .../util/tostring/GridToStringBuilder.java      |    2 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 1340 ++++++++++++-----
 .../tcp/TcpCommunicationSpiMBean.java           |   40 +
 .../ignite/spi/discovery/tcp/ServerImpl.java    |   14 +-
 .../ignite/stream/socket/SocketStreamer.java    |    1 +
 .../ignite/thread/IgniteThreadFactory.java      |    8 +-
 .../IgniteSlowClientDetectionSelfTest.java      |    1 +
 ...unicationBalanceMultipleConnectionsTest.java |   28 +
 .../IgniteCommunicationBalanceTest.java         |  339 +++++
 .../communication/IgniteIoTestMessagesTest.java |   95 ++
 .../IgniteVariousConnectionNumberTest.java      |  166 +++
 .../cache/CrossCacheTxRandomOperationsTest.java |   30 +-
 ...idAbstractCacheInterceptorRebalanceTest.java |    4 +-
 ...CacheOffHeapMultiThreadedUpdateSelfTest.java |    6 +-
 ...eAtomicMessageRecovery10ConnectionsTest.java |   28 +
 ...cMessageRecoveryNoPairedConnectionsTest.java |   47 +
 ...acheConnectionRecovery10ConnectionsTest.java |   35 +
 .../distributed/IgniteCacheCreatePutTest.java   |    2 +-
 .../IgniteCacheMessageRecoveryAbstractTest.java |   24 +-
 .../IgniteCacheMessageWriteTimeoutTest.java     |   17 +-
 .../dht/IgniteCacheMultiTxLockSelfTest.java     |    6 +-
 ...erNoStripedPoolMultiNodeFullApiSelfTest.java |   35 +
 ...edNoStripedPoolMultiNodeFullApiSelfTest.java |   35 +
 .../TxDeadlockDetectionNoHangsTest.java         |    2 +-
 .../TxOptimisticDeadlockDetectionTest.java      |   29 +-
 .../GridServiceProcessorProxySelfTest.java      |    2 +-
 .../util/future/GridFutureAdapterSelfTest.java  |  122 +-
 .../nio/impl/GridNioFilterChainSelfTest.java    |   32 +-
 .../loadtests/nio/GridNioBenchmarkClient.java   |    4 +-
 .../p2p/GridP2PRecursionTaskSelfTest.java       |    2 +-
 .../spi/GridTcpSpiForwardingSelfTest.java       |   18 +-
 .../GridTcpCommunicationSpiAbstractTest.java    |   28 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   82 +-
 .../GridTcpCommunicationSpiConfigSelfTest.java  |    5 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |   23 +-
 ...dTcpCommunicationSpiRecoveryAckSelfTest.java |    9 +-
 ...tionSpiRecoveryFailureDetectionSelfTest.java |    1 +
 ...ationSpiRecoveryNoPairedConnectionsTest.java |   28 +
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   67 +-
 ...CommunicationRecoveryAckClosureSelfTest.java |    9 +-
 .../junits/GridTestKernalContext.java           |    4 +-
 .../IgniteCacheFullApiSelfTestSuite.java        |    6 +
 .../ignite/testsuites/IgniteCacheTestSuite.java |   17 +-
 .../IgniteSpiCommunicationSelfTestSuite.java    |    2 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |    4 +-
 .../HadoopExternalCommunication.java            |    5 +-
 .../communication/HadoopIpcToNioAdapter.java    |    2 +-
 .../communication/HadoopMarshallerFilter.java   |    6 +-
 .../ignite/stream/kafka/KafkaStreamer.java      |    2 +-
 .../ignite/tools/classgen/ClassesGenerator.java |    8 +-
 .../ignite/yardstick/IgniteBenchmarkUtils.java  |    6 +-
 .../yardstick/cache/CacheEntryEventProbe.java   |    2 +-
 .../yardstick/cache/IgniteIoTestBenchmark.java  |   73 +
 .../io/IgniteIoTestAbstractBenchmark.java       |   61 +
 .../io/IgniteIoTestSendAllBenchmark.java        |   32 +
 .../io/IgniteIoTestSendRandomBenchmark.java     |   35 +
 134 files changed, 5935 insertions(+), 998 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java b/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
index ad12297..dd8a72b 100644
--- a/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
+++ b/examples/src/main/java/org/apache/ignite/examples/ExampleNodeStartup.java
@@ -33,4 +33,4 @@ public class ExampleNodeStartup {
     public static void main(String[] args) throws IgniteException {
         Ignition.start("examples/config/example-ignite.xml");
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
index 82a76b8..b9bae5b 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CachePutGetExample.java
@@ -105,4 +105,4 @@ public class CachePutGetExample {
         for (Map.Entry<Integer, String> e : vals.entrySet())
             System.out.println("Got entry [key=" + e.getKey() + ", val=" + e.getValue() + ']');
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
index 1014726..814bb2e 100644
--- a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
+++ b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarJvmCloudExample.scala
@@ -50,7 +50,7 @@ object ScalarJvmCloudExample {
             val pool = Executors.newFixedThreadPool(NODES.size)
 
             // Concurrently startup all nodes.
-            NODES.foreach(name => pool.submit(new Runnable {
+            NODES.foreach(name => pool.execute(new Runnable {
                 @impl def run() {
                     // All defaults.
                     val cfg = new IgniteConfiguration

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
index 0f56c73..c03c06e 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/ClientMemcachedProtocolSelfTest.java
@@ -111,6 +111,8 @@ public class ClientMemcachedProtocolSelfTest extends AbstractRestProcessorSelfTe
 
         Map<String, Object> map = client.getBulk("getKey1", "getKey2");
 
+        info("Map: " + map);
+
         Assert.assertEquals(2, map.size());
 
         Assert.assertEquals("getVal1", map.get("getKey1"));
@@ -443,4 +445,4 @@ public class ClientMemcachedProtocolSelfTest extends AbstractRestProcessorSelfTe
             return res;
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
index c82c73e..9bc4e7f 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/protocols/tcp/MockNioSession.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.rest.protocols.tcp;
 
 import java.net.InetSocketAddress;
+import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.util.lang.GridMetadataAwareAdapter;
 import org.apache.ignite.internal.util.nio.GridNioFinishedFuture;
 import org.apache.ignite.internal.util.nio.GridNioFuture;
@@ -111,6 +112,11 @@ public class MockNioSession extends GridMetadataAwareAdapter implements GridNioS
     }
 
     /** {@inheritDoc} */
+    @Override public void sendNoFuture(Object msg) throws IgniteCheckedException {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public GridNioFuture<Object> resumeReads() {
         return null;
     }
@@ -131,12 +137,27 @@ public class MockNioSession extends GridMetadataAwareAdapter implements GridNioS
     }
 
     /** {@inheritDoc} */
-    @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+    @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
+    @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
         return null;
     }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void systemMessage(Object msg) {
+        // No-op.
+    }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index b83b5ef..4d85c54 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -483,6 +483,9 @@ public final class IgniteSystemProperties {
     @Deprecated
     public static final String IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES = "IGNITE_BINARY_DONT_WRAP_TREE_STRUCTURES";
 
+    /** */
+    public static final String IGNITE_IO_BALANCE_PERIOD = "IGNITE_IO_BALANCE_PERIOD";
+
     /**
      * When set to {@code true} fields are written by BinaryMarshaller in sorted order. Otherwise
      * the natural order is used.

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
index 7494e37..d3f381e 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/store/CacheLoadOnlyStoreAdapter.java
@@ -153,14 +153,14 @@ public abstract class CacheLoadOnlyStoreAdapter<K, V, I> implements CacheStore<K
                 buf.add(iter.next());
 
                 if (buf.size() == batchSize) {
-                    exec.submit(new Worker(c, buf, args));
+                    exec.execute(new Worker(c, buf, args));
 
                     buf = new ArrayList<>(batchSize);
                 }
             }
 
             if (!buf.isEmpty())
-                exec.submit(new Worker(c, buf, args));
+                exec.execute(new Worker(c, buf, args));
         }
         catch (RejectedExecutionException ignored) {
             // Because of custom RejectedExecutionHandler.
@@ -330,4 +330,4 @@ public abstract class CacheLoadOnlyStoreAdapter<K, V, I> implements CacheStore<K
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 75145a3..dcd8a80 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -146,7 +146,7 @@ public class IgniteConfiguration {
     public static final int AVAILABLE_PROC_CNT = Runtime.getRuntime().availableProcessors();
 
     /** Default core size of public thread pool. */
-    public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT) * 2;
+    public static final int DFLT_PUBLIC_THREAD_CNT = Math.max(8, AVAILABLE_PROC_CNT);
 
     /** Default keep alive time for public thread pool. */
     @Deprecated
@@ -236,6 +236,12 @@ public class IgniteConfiguration {
     /** Async Callback pool size. */
     private int callbackPoolSize = DFLT_PUBLIC_THREAD_CNT;
 
+    /**
+     * Use striped pool for internal requests processing when possible
+     * (e.g. cache requests per-partition striping).
+     */
+    private int stripedPoolSize = DFLT_PUBLIC_THREAD_CNT;
+
     /** System pool size. */
     private int sysPoolSize = DFLT_SYSTEM_CORE_THREAD_CNT;
 
@@ -553,6 +559,7 @@ public class IgniteConfiguration {
         sndRetryDelay = cfg.getNetworkSendRetryDelay();
         sslCtxFactory = cfg.getSslContextFactory();
         storeSesLsnrs = cfg.getCacheStoreSessionListenerFactories();
+        stripedPoolSize = cfg.getStripedPoolSize();
         svcCfgs = cfg.getServiceConfiguration();
         sysPoolSize = cfg.getSystemThreadPoolSize();
         timeSrvPortBase = cfg.getTimeServerPortBase();
@@ -712,6 +719,47 @@ public class IgniteConfiguration {
     }
 
     /**
+     * Returns striped pool size that should be used for cache requests
+     * processing.
+     * <p>
+     * If set to non-positive value then requests get processed in system pool.
+     * <p>
+     * Striped pool is better for typical cache operations.
+     *
+     * @return Positive value if striped pool should be initialized
+     *      with configured number of threads (stripes) and used for requests processing
+     *      or non-positive value to process requests in system pool.
+     *
+     * @see #getPublicThreadPoolSize()
+     * @see #getSystemThreadPoolSize()
+     */
+    public int getStripedPoolSize() {
+        return stripedPoolSize;
+    }
+
+    /**
+     * Sets striped pool size that should be used for cache requests
+     * processing.
+     * <p>
+     * If set to non-positive value then requests get processed in system pool.
+     * <p>
+     * Striped pool is better for typical cache operations.
+     *
+     * @param stripedPoolSize Positive value if striped pool should be initialized
+     *      with passed in number of threads (stripes) and used for requests processing
+     *      or non-positive value to process requests in system pool.
+     * @return {@code this} for chaining.
+     *
+     * @see #getPublicThreadPoolSize()
+     * @see #getSystemThreadPoolSize()
+     */
+    public IgniteConfiguration setStripedPoolSize(int stripedPoolSize) {
+        this.stripedPoolSize = stripedPoolSize;
+
+        return this;
+    }
+
+    /**
      * Should return a thread pool size to be used in grid.
      * This executor service will be in charge of processing {@link ComputeJob GridJobs}
      * and user messages sent to node.

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
index 715f8a5..68d34ce 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEventConsumeHandler.java
@@ -181,7 +181,7 @@ class GridEventConsumeHandler implements GridContinuousHandler {
                         notificationQueue.add(new T3<>(nodeId, routineId, evt));
 
                         if (!notificationInProgress) {
-                            ctx.getSystemExecutorService().submit(new Runnable() {
+                            ctx.getSystemExecutorService().execute(new Runnable() {
                                 @Override public void run() {
                                     if (!ctx.continuous().lockStopping())
                                         return;

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
index 804d228..dbfa0b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobContextImpl.java
@@ -217,7 +217,7 @@ public class GridJobContextImpl implements ComputeJobContext, Externalizable {
 
                                 assert execSvc != null;
 
-                                execSvc.submit(new Runnable() {
+                                execSvc.execute(new Runnable() {
                                     @Override public void run() {
                                         callcc0();
                                     }
@@ -300,4 +300,4 @@ public class GridJobContextImpl implements ComputeJobContext, Externalizable {
     @Override public String toString() {
         return S.toString(GridJobContextImpl.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index ae29223..927944f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
+import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.plugin.PluginNotFoundException;
 import org.apache.ignite.plugin.PluginProvider;
@@ -511,6 +512,14 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public ExecutorService getSystemExecutorService();
 
     /**
+     * Executor service that is in charge of processing internal system messages
+     * in stripes (dedicated threads).
+     *
+     * @return Thread pool implementation to be used in grid for internal system messages.
+     */
+    public StripedExecutor getStripedExecutorService();
+
+    /**
      * Executor service that is in charge of processing internal and Visor
      * {@link org.apache.ignite.compute.ComputeJob GridJobs}.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 94c6448..a2ad1b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -83,6 +83,7 @@ import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
 import org.apache.ignite.internal.util.IgniteExceptionRegistry;
+import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -300,6 +301,10 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringExclude
+    protected StripedExecutor stripedExecSvc;
+
+    /** */
+    @GridToStringExclude
     private ExecutorService p2pExecSvc;
 
     /** */
@@ -381,6 +386,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
      * @param marshCachePool Marshaller cache pool.
      * @param execSvc Public executor service.
      * @param sysExecSvc System executor service.
+     * @param stripedExecSvc Striped executor.
      * @param p2pExecSvc P2P executor service.
      * @param mgmtExecSvc Management executor service.
      * @param igfsExecSvc IGFS executor service.
@@ -400,6 +406,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         ExecutorService marshCachePool,
         ExecutorService execSvc,
         ExecutorService sysExecSvc,
+        StripedExecutor stripedExecSvc,
         ExecutorService p2pExecSvc,
         ExecutorService mgmtExecSvc,
         ExecutorService igfsExecSvc,
@@ -407,7 +414,8 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         ExecutorService affExecSvc,
         @Nullable ExecutorService idxExecSvc,
         IgniteStripedThreadPoolExecutor callbackExecSvc,
-        List<PluginProvider> plugins) throws IgniteCheckedException {
+        List<PluginProvider> plugins
+    ) throws IgniteCheckedException {
         assert grid != null;
         assert cfg != null;
         assert gw != null;
@@ -419,6 +427,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
         this.marshCachePool = marshCachePool;
         this.execSvc = execSvc;
         this.sysExecSvc = sysExecSvc;
+        this.stripedExecSvc = stripedExecSvc;
         this.p2pExecSvc = p2pExecSvc;
         this.mgmtExecSvc = mgmtExecSvc;
         this.igfsExecSvc = igfsExecSvc;
@@ -948,6 +957,11 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
+    @Override public StripedExecutor getStripedExecutorService() {
+        return stripedExecSvc;
+    }
+
+    /** {@inheritDoc} */
     @Override public ExecutorService getManagementExecutorService() {
         return mgmtExecSvc;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java b/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
index b040a97..5e8e520 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridPerformanceSuggestions.java
@@ -89,4 +89,4 @@ public class GridPerformanceSuggestions {
     @Override public String toString() {
         return S.toString(GridPerformanceSuggestions.class, this);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
index b5608db..24ddcd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTopic.java
@@ -97,7 +97,10 @@ public enum GridTopic {
     TOPIC_QUERY,
 
     /** */
-    TOPIC_TX;
+    TOPIC_TX,
+
+    /** */
+    TOPIC_IO_TEST;
 
     /** Enum values. */
     private static final GridTopic[] VALS = values();

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
index b80a755..789556d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal;
 
+import java.util.concurrent.Executor;
 import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCheckedException;
@@ -133,6 +134,16 @@ public interface IgniteInternalFuture<R> {
     public <T> IgniteInternalFuture<T> chain(IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb);
 
     /**
+     * Make a chained future to convert result of this future (when complete) into a new format.
+     * It is guaranteed that done callback will be called only ONCE.
+     *
+     * @param doneCb Done callback that is applied to this future when it finishes to produce chained future result.
+     * @param exec Executor to run callback.
+     * @return Chained future that finishes after this future completes and done callback is called.
+     */
+    public <T> IgniteInternalFuture<T> chain(IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb, Executor exec);
+
+    /**
      * @return Error value if future has already been completed with error.
      */
     public Throwable error();

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 8fda72f..4972d1f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -61,10 +61,10 @@ import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteEvents;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteFileSystem;
+import org.apache.ignite.IgniteLock;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteMessaging;
 import org.apache.ignite.IgniteQueue;
-import org.apache.ignite.IgniteLock;
 import org.apache.ignite.IgniteScheduler;
 import org.apache.ignite.IgniteSemaphore;
 import org.apache.ignite.IgniteServices;
@@ -115,7 +115,6 @@ import org.apache.ignite.internal.processors.continuous.GridContinuousProcessor;
 import org.apache.ignite.internal.processors.datastreamer.DataStreamProcessor;
 import org.apache.ignite.internal.processors.datastructures.DataStructuresProcessor;
 import org.apache.ignite.internal.processors.hadoop.Hadoop;
-import org.apache.ignite.internal.processors.hadoop.HadoopClassLoader;
 import org.apache.ignite.internal.processors.hadoop.HadoopProcessorAdapter;
 import org.apache.ignite.internal.processors.job.GridJobProcessor;
 import org.apache.ignite.internal.processors.jobmetrics.GridJobMetricsProcessor;
@@ -139,6 +138,7 @@ import org.apache.ignite.internal.processors.service.GridServiceProcessor;
 import org.apache.ignite.internal.processors.session.GridTaskSessionProcessor;
 import org.apache.ignite.internal.processors.task.GridTaskProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutProcessor;
+import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.future.GridCompoundFuture;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
@@ -175,6 +175,7 @@ import org.apache.ignite.spi.discovery.tcp.internal.TcpDiscoveryNode;
 import org.apache.ignite.thread.IgniteStripedThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 
+import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_CONFIG_URL;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_DAEMON;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_NO_ASCII;
@@ -183,7 +184,6 @@ import static org.apache.ignite.IgniteSystemProperties.IGNITE_REST_START_ON_CLIE
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SKIP_CONFIGURATION_CONSISTENCY_CHECK;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_STARVATION_CHECK_INTERVAL;
 import static org.apache.ignite.IgniteSystemProperties.IGNITE_SUCCESS_FILE;
-import static org.apache.ignite.IgniteSystemProperties.IGNITE_BINARY_MARSHALLER_USE_STRING_SERIALIZATION_VER_2;
 import static org.apache.ignite.IgniteSystemProperties.getBoolean;
 import static org.apache.ignite.IgniteSystemProperties.snapshot;
 import static org.apache.ignite.internal.GridKernalState.DISCONNECTED;
@@ -200,7 +200,6 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_BUILD_VER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CLIENT_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_CONSISTENCY_CHECK_SKIPPED;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DAEMON;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_DEPLOYMENT_MODE;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_GRID_NAME;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_IPS;
@@ -209,11 +208,12 @@ import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JMX_PORT;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JVM_ARGS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_JVM_PID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LANG_RUNTIME;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_LATE_AFFINITY_ASSIGNMENT;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MACS;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_COMPACT_FOOTER;
-import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_BINARY_STRING_SER_VER_2;
+import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_MARSHALLER_USE_DFLT_SUID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_NODE_CONSISTENT_ID;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PEER_CLASSLOADING;
 import static org.apache.ignite.internal.IgniteNodeAttributes.ATTR_PHY_RAM;
@@ -664,6 +664,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @param utilityCachePool Utility cache pool.
      * @param execSvc Executor service.
      * @param sysExecSvc System executor service.
+     * @param stripedExecSvc Striped executor.
      * @param p2pExecSvc P2P executor service.
      * @param mgmtExecSvc Management executor service.
      * @param igfsExecSvc IGFS executor service.
@@ -674,11 +675,13 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @SuppressWarnings({"CatchGenericClass", "unchecked"})
-    public void start(final IgniteConfiguration cfg,
+    public void start(
+        final IgniteConfiguration cfg,
         ExecutorService utilityCachePool,
         ExecutorService marshCachePool,
         final ExecutorService execSvc,
         final ExecutorService sysExecSvc,
+        final StripedExecutor stripedExecSvc,
         ExecutorService p2pExecSvc,
         ExecutorService mgmtExecSvc,
         ExecutorService igfsExecSvc,
@@ -686,7 +689,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         ExecutorService affExecSvc,
         @Nullable ExecutorService idxExecSvc,
         IgniteStripedThreadPoolExecutor callbackExecSvc,
-        GridAbsClosure errHnd)
+        GridAbsClosure errHnd
+    )
         throws IgniteCheckedException
     {
         gw.compareAndSet(null, new GridKernalGatewayImpl(cfg.getGridName()));
@@ -786,6 +790,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 marshCachePool,
                 execSvc,
                 sysExecSvc,
+                stripedExecSvc,
                 p2pExecSvc,
                 mgmtExecSvc,
                 igfsExecSvc,
@@ -793,7 +798,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                 affExecSvc,
                 idxExecSvc,
                 callbackExecSvc,
-                plugins);
+                plugins
+            );
 
             cfg.getMarshaller().setContext(ctx.marshallerContext());
 
@@ -987,24 +993,51 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
 
             starveTask = ctx.timeout().schedule(new Runnable() {
                 /** Last completed task count. */
-                private long lastCompletedCnt;
+                private long lastCompletedCntPub;
+
+                /** Last completed task count. */
+                private long lastCompletedCntSys;
 
                 @Override public void run() {
-                    if (!(execSvc instanceof ThreadPoolExecutor))
-                        return;
+                    if (execSvc instanceof ThreadPoolExecutor) {
+                        ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+
+                        lastCompletedCntPub = checkPoolStarvation(exec, lastCompletedCntPub, "public");
+                    }
 
-                    ThreadPoolExecutor exec = (ThreadPoolExecutor)execSvc;
+                    if (sysExecSvc instanceof ThreadPoolExecutor) {
+                        ThreadPoolExecutor exec = (ThreadPoolExecutor)sysExecSvc;
 
+                        lastCompletedCntSys = checkPoolStarvation(exec, lastCompletedCntSys, "system");
+                    }
+
+                    if (stripedExecSvc != null)
+                        stripedExecSvc.checkStarvation();
+                }
+
+                /**
+                 * @param exec Thread pool executor to check.
+                 * @param lastCompletedCnt Last completed tasks count.
+                 * @param pool Pool name for message.
+                 * @return Current completed tasks count.
+                 */
+                private long checkPoolStarvation(
+                    ThreadPoolExecutor exec,
+                    long lastCompletedCnt,
+                    String pool
+                ) {
                     long completedCnt = exec.getCompletedTaskCount();
 
                     // If all threads are active and no task has completed since last time and there is
                     // at least one waiting request, then it is possible starvation.
                     if (exec.getPoolSize() == exec.getActiveCount() && completedCnt == lastCompletedCnt &&
                         !exec.getQueue().isEmpty())
-                        LT.warn(log, "Possible thread pool starvation detected (no task completed in last " +
-                            interval + "ms, is executorService pool size large enough?)");
+                        LT.warn(
+                            log,
+                            "Possible thread pool starvation detected (no task completed in last " +
+                                interval + "ms, is " + pool + " thread pool size large enough?)");
 
-                    lastCompletedCnt = completedCnt;
+                    return completedCnt;
                 }
             }, interval, interval);
         }
@@ -1129,6 +1162,8 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
             }, longOpDumpTimeout, longOpDumpTimeout);
         }
 
+        ctx.performance().add("Disable assertions (remove '-ea' from JVM options)", !U.assertionsEnabled());
+
         ctx.performance().logSuggestions(log, gridName);
 
         U.quietAndInfo(log, "To start Console Management & Monitoring run ignitevisorcmd.{sh|bat}");
@@ -3519,6 +3554,26 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
         }
     }
 
+    /**
+     * @param node Node.
+     * @param payload Message payload.
+     * @param procFromNioThread If {@code true} message is processed from NIO thread.
+     * @return Response future.
+     */
+    public IgniteInternalFuture sendIoTest(ClusterNode node, byte[] payload, boolean procFromNioThread) {
+        return ctx.io().sendIoTest(node, payload, procFromNioThread);
+    }
+
+    /**
+     * @param nodes Nodes.
+     * @param payload Message payload.
+     * @param procFromNioThread If {@code true} message is processed from NIO thread.
+     * @return Response future.
+     */
+    public IgniteInternalFuture sendIoTest(List<ClusterNode> nodes, byte[] payload, boolean procFromNioThread) {
+        return ctx.io().sendIoTest(nodes, payload, procFromNioThread);
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(IgniteKernal.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index b3a9eec..f32a753 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -66,6 +66,7 @@ import org.apache.ignite.internal.processors.igfs.IgfsUtils;
 import org.apache.ignite.internal.processors.resource.GridSpringResourceContext;
 import org.apache.ignite.internal.util.GridConcurrentHashSet;
 import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.internal.util.StripedExecutor;
 import org.apache.ignite.internal.util.spring.IgniteSpringHelper;
 import org.apache.ignite.internal.util.typedef.CA;
 import org.apache.ignite.internal.util.typedef.F;
@@ -1459,6 +1460,9 @@ public class IgnitionEx {
         /** System executor service. */
         private ThreadPoolExecutor sysExecSvc;
 
+        /** */
+        private StripedExecutor stripedExecSvc;
+
         /** Management executor service. */
         private ThreadPoolExecutor mgmtExecSvc;
 
@@ -1652,8 +1656,6 @@ public class IgnitionEx {
 
             execSvc.allowCoreThreadTimeOut(true);
 
-            // Note that since we use 'LinkedBlockingQueue', number of
-            // maximum threads has no effect.
             validateThreadPoolSize(cfg.getSystemThreadPoolSize(), "system");
 
             sysExecSvc = new IgniteThreadPoolExecutor(
@@ -1666,6 +1668,9 @@ public class IgnitionEx {
 
             sysExecSvc.allowCoreThreadTimeOut(true);
 
+            if (cfg.getStripedPoolSize() > 0)
+                stripedExecSvc = new StripedExecutor(cfg.getStripedPoolSize(), cfg.getGridName(), "sys", log);
+
             // Note that since we use 'LinkedBlockingQueue', number of
             // maximum threads has no effect.
             // Note, that we do not pre-start threads here as management pool may
@@ -1791,13 +1796,26 @@ public class IgnitionEx {
                 // Init here to make grid available to lifecycle listeners.
                 grid = grid0;
 
-                grid0.start(myCfg, utilityCacheExecSvc, marshCacheExecSvc, execSvc, sysExecSvc, p2pExecSvc, mgmtExecSvc,
-                    igfsExecSvc, restExecSvc, affExecSvc, idxExecSvc, callbackExecSvc,
+                grid0.start(
+                    myCfg,
+                    utilityCacheExecSvc,
+                    marshCacheExecSvc,
+                    execSvc,
+                    sysExecSvc,
+                    stripedExecSvc,
+                    p2pExecSvc,
+                    mgmtExecSvc,
+                    igfsExecSvc,
+                    restExecSvc,
+                    affExecSvc,
+                    idxExecSvc,
+                    callbackExecSvc,
                     new CA() {
                         @Override public void apply() {
                             startLatch.countDown();
                         }
-                    });
+                    }
+                );
 
                 state = STARTED;
 
@@ -2415,6 +2433,10 @@ public class IgnitionEx {
 
             sysExecSvc = null;
 
+            U.shutdownNow(getClass(), stripedExecSvc, log);
+
+            stripedExecSvc = null;
+
             U.shutdownNow(getClass(), mgmtExecSvc, log);
 
             mgmtExecSvc = null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
index 6ea7c22..12baee0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/impl/connection/GridClientConnectionManagerAdapter.java
@@ -200,6 +200,7 @@ public abstract class GridClientConnectionManagerAdapter implements GridClientCo
                     .socketSendBufferSize(0)
                     .idleTimeout(Long.MAX_VALUE)
                     .gridName(routerClient ? "routerClient" : "gridClient")
+                    .serverName("tcp-client")
                     .daemon(cfg.isDaemon())
                     .build();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
index 06a4929..3566830 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/client/router/impl/GridTcpRouterImpl.java
@@ -258,6 +258,7 @@ public class GridTcpRouterImpl implements GridTcpRouter, GridTcpRouterMBean, Lif
                 .logger(log)
                 .selectorCount(Runtime.getRuntime().availableProcessors())
                 .gridName(gridName)
+                .serverName("router")
                 .tcpNoDelay(tcpNoDelay)
                 .directBuffer(false)
                 .byteOrder(ByteOrder.nativeOrder())

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
index 3df29cf..7ef7bc0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoManager.java
@@ -26,15 +26,17 @@ import java.util.Map;
 import java.util.Map.Entry;
 import java.util.Queue;
 import java.util.UUID;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentLinkedQueue;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.RejectedExecutionException;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
-
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
@@ -44,6 +46,7 @@ import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.GridTopic;
 import org.apache.ignite.internal.IgniteComponentType;
 import org.apache.ignite.internal.IgniteDeploymentCheckedException;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.direct.DirectMessageReader;
 import org.apache.ignite.internal.direct.DirectMessageWriter;
 import org.apache.ignite.internal.managers.GridManagerAdapter;
@@ -55,6 +58,8 @@ import org.apache.ignite.internal.processors.pool.PoolProcessor;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.util.GridBoundedConcurrentLinkedHashSet;
 import org.apache.ignite.internal.util.GridSpinReadWriteLock;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.lang.GridTuple3;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.F;
@@ -83,6 +88,7 @@ import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_JOINED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
 import static org.apache.ignite.internal.GridTopic.TOPIC_COMM_USER;
+import static org.apache.ignite.internal.GridTopic.TOPIC_IO_TEST;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.AFFINITY_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IDX_POOL;
 import static org.apache.ignite.internal.managers.communication.GridIoPolicy.IGFS_POOL;
@@ -176,6 +182,12 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
     /** Stopping flag. */
     private boolean stopping;
 
+    /** */
+    private final AtomicReference<ConcurrentHashMap<Long, IoTestFuture>> ioTestMap = new AtomicReference<>();
+
+    /** */
+    private final AtomicLong ioTestId = new AtomicLong();
+
     /**
      * @param ctx Grid kernal context.
      */
@@ -297,6 +309,114 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
 
         if (log.isDebugEnabled())
             log.debug(startInfo());
+
+        addMessageListener(GridTopic.TOPIC_IO_TEST, new GridMessageListener() {
+            @Override public void onMessage(UUID nodeId, Object msg) {
+                ClusterNode node = ctx.discovery().node(nodeId);
+
+                if (node == null)
+                    return;
+
+                IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg;
+
+                if (msg0.request()) {
+                    IgniteIoTestMessage res = new IgniteIoTestMessage(msg0.id(), false, null);
+
+                    res.flags(msg0.flags());
+
+                    try {
+                        send(node, GridTopic.TOPIC_IO_TEST, res, GridIoPolicy.SYSTEM_POOL);
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to send IO test response [msg=" + msg0 + "]", e);
+                    }
+                }
+                else {
+                    IoTestFuture fut = ioTestMap().get(msg0.id());
+
+                    if (fut == null)
+                        U.warn(log, "Failed to find IO test future [msg=" + msg0 + ']');
+                    else
+                        fut.onResponse();
+                }
+            }
+        });
+    }
+
+    /**
+     * @param nodes Nodes.
+     * @param payload Payload.
+     * @param procFromNioThread If {@code true} message is processed from NIO thread.
+     * @return Response future.
+     */
+    public IgniteInternalFuture sendIoTest(List<ClusterNode> nodes, byte[] payload, boolean procFromNioThread) {
+        long id = ioTestId.getAndIncrement();
+
+        IoTestFuture fut = new IoTestFuture(id, nodes.size());
+
+        IgniteIoTestMessage msg = new IgniteIoTestMessage(id, true, payload);
+
+        msg.processFromNioThread(procFromNioThread);
+
+        ioTestMap().put(id, fut);
+
+        for (int i = 0; i < nodes.size(); i++) {
+            ClusterNode node = nodes.get(i);
+
+            try {
+                send(node, GridTopic.TOPIC_IO_TEST, msg, GridIoPolicy.SYSTEM_POOL);
+            }
+            catch (IgniteCheckedException e) {
+                ioTestMap().remove(msg.id());
+
+                return new GridFinishedFuture(e);
+            }
+        }
+
+        return fut;
+    }
+
+    /**
+     * @param node Node.
+     * @param payload Payload.
+     * @param procFromNioThread If {@code true} message is processed from NIO thread.
+     * @return Response future.
+     */
+    public IgniteInternalFuture sendIoTest(ClusterNode node, byte[] payload, boolean procFromNioThread) {
+        long id = ioTestId.getAndIncrement();
+
+        IoTestFuture fut = new IoTestFuture(id, 1);
+
+        IgniteIoTestMessage msg = new IgniteIoTestMessage(id, true, payload);
+
+        msg.processFromNioThread(procFromNioThread);
+
+        ioTestMap().put(id, fut);
+
+        try {
+            send(node, GridTopic.TOPIC_IO_TEST, msg, GridIoPolicy.SYSTEM_POOL);
+        }
+        catch (IgniteCheckedException e) {
+            ioTestMap().remove(msg.id());
+
+            return new GridFinishedFuture(e);
+        }
+
+        return fut;
+    }
+
+    /**
+     * @return IO test futures map.
+     */
+    private ConcurrentHashMap<Long, IoTestFuture> ioTestMap() {
+        ConcurrentHashMap<Long, IoTestFuture> map = ioTestMap.get();
+
+        if (map == null) {
+            if (!ioTestMap.compareAndSet(null, map = new ConcurrentHashMap<>()))
+                map = ioTestMap.get();
+        }
+
+        return map;
     }
 
     /** {@inheritDoc} */
@@ -514,16 +634,6 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                 return;
             }
 
-            // Check discovery.
-            ClusterNode node = ctx.discovery().node(nodeId);
-
-            if (node == null) {
-                if (log.isDebugEnabled())
-                    log.debug("Ignoring message from dead node [senderId=" + nodeId + ", msg=" + msg + ']');
-
-                return; // We can't receive messages from non-discovered ones.
-            }
-
             if (msg.topic() == null) {
                 int topicOrd = msg.topicOrdinal();
 
@@ -678,8 +788,31 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
                     msgC.run();
                 }
             }
+
+            @Override public String toString() {
+                return "Message closure [msg=" + msg + ']';
+            }
         };
 
+        if (msg.topicOrdinal() == TOPIC_IO_TEST.ordinal()) {
+            IgniteIoTestMessage msg0 = (IgniteIoTestMessage)msg.message();
+
+            if (msg0.processFromNioThread()) {
+                c.run();
+
+                return;
+            }
+        }
+
+        if (ctx.config().getStripedPoolSize() > 0 &&
+            plc == GridIoPolicy.SYSTEM_POOL &&
+            msg.partition() != Integer.MIN_VALUE
+            ) {
+            ctx.getStripedExecutorService().execute(msg.partition(), c);
+
+            return;
+        }
+
         try {
             pools.poolForPolicy(plc).execute(c);
         }
@@ -2460,4 +2593,56 @@ public class GridIoManager extends GridManagerAdapter<CommunicationSpi<Serializa
             return S.toString(DelayedMessage.class, this, super.toString());
         }
     }
+
+    /**
+     *
+     */
+    private class IoTestFuture extends GridFutureAdapter<Object> {
+        /** */
+        private final long id;
+
+        /** */
+        private int cntr;
+
+        /**
+         * @param id ID.
+         * @param cntr Counter.
+         */
+        IoTestFuture(long id, int cntr) {
+            assert cntr > 0 : cntr;
+
+            this.id = id;
+            this.cntr = cntr;
+        }
+
+        /**
+         *
+         */
+        void onResponse() {
+            boolean complete;
+
+            synchronized (this) {
+                complete = --cntr == 0;
+            }
+
+            if (complete)
+                onDone();
+        }
+
+        /** {@inheritDoc} */
+        @Override public boolean onDone(@Nullable Object res, @Nullable Throwable err) {
+            if (super.onDone(res, err)) {
+                ioTestMap().remove(id);
+
+                return true;
+            }
+
+            return false;
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(IoTestFuture.class, this);
+        }
+    }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
index b28ced2..b1a26e9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessage.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.managers.communication;
 import java.io.Externalizable;
 import java.nio.ByteBuffer;
 import org.apache.ignite.internal.GridDirectTransient;
+import org.apache.ignite.internal.processors.cache.GridCacheMessage;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.plugin.extensions.communication.Message;
@@ -321,6 +322,18 @@ public class GridIoMessage implements Message {
         return 7;
     }
 
+    /**
+     * Get single partition for this message (if applicable).
+     *
+     * @return Partition ID.
+     */
+    public int partition() {
+        if (msg instanceof GridCacheMessage)
+            return ((GridCacheMessage)msg).partition();
+        else
+            return Integer.MIN_VALUE;
+    }
+
     /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridIoMessage.class, this);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
index 504e683..b1fe910 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
@@ -171,6 +171,16 @@ public class GridIoMessageFactory implements MessageFactory {
         Message msg = null;
 
         switch (type) {
+            case -44:
+                msg = new TcpCommunicationSpi.HandshakeMessage2();
+
+                break;
+
+            case -43:
+                msg = new IgniteIoTestMessage();
+
+                break;
+
             case -42:
                 msg = new HadoopDirectShuffleMessage();
 
@@ -816,7 +826,7 @@ public class GridIoMessageFactory implements MessageFactory {
 
                 break;
 
-            // [-3..119] [124..127] [-36]- this
+            // [-3..119] [124..127] [-36..-44]- this
             // [120..123] - DR
             // [-4..-22, -30..-35] - SQL
             default:

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
new file mode 100644
index 0000000..77aaa09
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
@@ -0,0 +1,235 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.managers.communication;
+
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.plugin.extensions.communication.Message;
+import org.apache.ignite.plugin.extensions.communication.MessageReader;
+import org.apache.ignite.plugin.extensions.communication.MessageWriter;
+
+import java.nio.ByteBuffer;
+
+/**
+ *
+ */
+public class IgniteIoTestMessage implements Message {
+    /** */
+    private static byte FLAG_PROC_FROM_NIO = 1;
+
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** */
+    private long id;
+
+    /** */
+    private byte flags;
+
+    /** */
+    private boolean req;
+
+    /** */
+    private byte payload[];
+
+    /**
+     *
+     */
+    IgniteIoTestMessage() {
+        // No-op.
+    }
+
+    /**
+     * @param id Message ID.
+     * @param req Request flag.
+     * @param payload Payload.
+     */
+    IgniteIoTestMessage(long id, boolean req, byte[] payload) {
+        this.id = id;
+        this.req = req;
+        this.payload = payload;
+    }
+
+    /**
+     * @return {@code True} if message should be processed from NIO thread
+     * (otherwise message is submitted to system pool).
+     */
+    boolean processFromNioThread() {
+        return isFlag(FLAG_PROC_FROM_NIO);
+    }
+
+    /**
+     * @param procFromNioThread {@code True} if message should be processed from NIO thread.
+     */
+    void processFromNioThread(boolean procFromNioThread) {
+        setFlag(procFromNioThread, FLAG_PROC_FROM_NIO);
+    }
+
+    /**
+     * @param flags Flags.
+     */
+    public void flags(byte flags) {
+        this.flags = flags;
+    }
+
+    /**
+     * @return Flags.
+     */
+    public byte flags() {
+        return flags;
+    }
+
+    /**
+     * Sets flag mask.
+     *
+     * @param flag Set or clear.
+     * @param mask Mask.
+     */
+    private void setFlag(boolean flag, int mask) {
+        flags = flag ? (byte)(flags | mask) : (byte)(flags & ~mask);
+    }
+
+    /**
+     * Reads flag mask.
+     *
+     * @param mask Mask to read.
+     * @return Flag value.
+     */
+    private boolean isFlag(int mask) {
+        return (flags & mask) != 0;
+    }
+
+    /**
+     * @return {@code true} if this is request.
+     */
+    public boolean request() {
+        return req;
+    }
+
+    /**
+     * @return ID.
+     */
+    public long id() {
+        return id;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
+        writer.setBuffer(buf);
+
+        if (!writer.isHeaderWritten()) {
+            if (!writer.writeHeader(directType(), fieldsCount()))
+                return false;
+
+            writer.onHeaderWritten();
+        }
+
+        switch (writer.state()) {
+            case 0:
+                if (!writer.writeByte("flags", flags))
+                    return false;
+
+                writer.incrementState();
+
+            case 1:
+                if (!writer.writeLong("id", id))
+                    return false;
+
+                writer.incrementState();
+
+            case 2:
+                if (!writer.writeByteArray("payload", payload))
+                    return false;
+
+                writer.incrementState();
+
+            case 3:
+                if (!writer.writeBoolean("req", req))
+                    return false;
+
+                writer.incrementState();
+
+        }
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean readFrom(ByteBuffer buf, MessageReader reader) {
+        reader.setBuffer(buf);
+
+        if (!reader.beforeMessageRead())
+            return false;
+
+        switch (reader.state()) {
+            case 0:
+                flags = reader.readByte("flags");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 1:
+                id = reader.readLong("id");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 2:
+                payload = reader.readByteArray("payload");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+            case 3:
+                req = reader.readBoolean("req");
+
+                if (!reader.isLastRead())
+                    return false;
+
+                reader.incrementState();
+
+        }
+
+        return reader.afterMessageRead(IgniteIoTestMessage.class);
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte directType() {
+        return -43;
+    }
+
+    /** {@inheritDoc} */
+    @Override public byte fieldsCount() {
+        return 4;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void onAckReceived() {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(IgniteIoTestMessage.class, this);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 0f39c69..1db3931 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -288,6 +288,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     /** Asynchronous operations limit semaphore. */
     private Semaphore asyncOpsSem;
 
+    /** */
+    protected volatile boolean asyncToggled;
+
     /** {@inheritDoc} */
     @Override public String name() {
         return cacheCfg.getName();
@@ -364,6 +367,18 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
     }
 
     /**
+     * Toggles async flag if someone calls {@code withAsync()}
+     * on proxy and since that we have to properly handle all cache
+     * operations (sync and async) to put them in proper sequence.
+     *
+     * TODO: https://issues.apache.org/jira/browse/IGNITE-4393
+     */
+    void toggleAsync() {
+        if (!asyncToggled)
+            asyncToggled = true;
+    }
+
+    /**
      * Prints memory stats.
      */
     public void printMemoryStats() {
@@ -1134,7 +1149,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                 execSvc = Executors.newFixedThreadPool(jobs.size() - 1);
 
                 for (int i = 1; i < jobs.size(); i++)
-                    execSvc.submit(jobs.get(i));
+                    execSvc.execute(jobs.get(i));
             }
 
             try {
@@ -2534,6 +2549,8 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Put future.
      */
     public IgniteInternalFuture<Boolean> putAsync(K key, V val, @Nullable CacheEntryPredicate filter) {
+        A.notNull(key, "key", val, "val");
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -2554,8 +2571,6 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      */
     public IgniteInternalFuture<Boolean> putAsync0(final K key, final V val,
         @Nullable final CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
         if (keyCheck)
             validateCacheKey(key);
 
@@ -4592,6 +4607,9 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * @return Failed future if waiting was interrupted.
      */
     @Nullable protected <T> IgniteInternalFuture<T> asyncOpAcquire() {
+        if (!asyncToggled)
+            return null;
+
         try {
             if (asyncOpsSem != null)
                 asyncOpsSem.acquire();
@@ -4610,7 +4628,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
      * Releases asynchronous operations permit, if limited.
      */
     protected void asyncOpRelease() {
-        if (asyncOpsSem != null)
+        if (asyncOpsSem != null && asyncToggled)
             asyncOpsSem.release();
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
index 71f99d3..0646d5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMessage.java
@@ -129,6 +129,13 @@ public abstract class GridCacheMessage implements Message {
     }
 
     /**
+     * @return Partition ID this message is targeted to or {@code -1} if it cannot be determined.
+     */
+    public int partition() {
+        return -1;
+    }
+
+    /**
      * If class loading error occurred during unmarshalling and {@link #ignoreClassErrors()} is
      * set to {@code true}, then the error will be passed into this method.
      *

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index d32f4c1..47abf2f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -110,6 +110,41 @@ import static org.apache.ignite.internal.processors.cache.GridCacheOperation.REA
  * Cache utility methods.
  */
 public class GridCacheUtils {
+    /** Cheat cache ID for debugging and benchmarking purposes. */
+    public static final int cheatCacheId;
+
+    /*
+     *
+     */
+    static {
+        String cheatCache = System.getProperty("CHEAT_CACHE");
+
+        if (cheatCache != null) {
+            cheatCacheId = cheatCache.hashCode();
+
+            if (cheatCacheId == 0)
+                throw new RuntimeException();
+
+            System.out.println(">>> Cheat cache ID [id=" + cheatCacheId + ", name=" + cheatCache + ']');
+        }
+        else
+            cheatCacheId = 0;
+    }
+
+    /**
+     * Quickly checks if passed in cache ID is a "cheat cache ID" set by -DCHEAT_CACHE=user_cache_name
+     * and resolved in static block above.
+     *
+     * FOR DEBUGGING AND TESTING PURPOSES!
+     *
+     * @param id Cache ID to check.
+     * @return {@code True} if this is cheat cache ID.
+     */
+    @Deprecated
+    public static boolean cheatCache(int id) {
+        return cheatCacheId != 0 && id == cheatCacheId;
+    }
+
     /**  Hadoop syste cache name. */
     public static final String SYS_CACHE_HADOOP_MR = "ignite-hadoop-mr-sys-cache";
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f03a3b5..90898f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -334,6 +334,14 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
     }
 
     /** {@inheritDoc} */
+    @Override public IgniteCache<K, V> withAsync() {
+        if (delegate instanceof GridCacheAdapter)
+            ((GridCacheAdapter)delegate).toggleAsync();
+
+        return super.withAsync();
+    }
+
+    /** {@inheritDoc} */
     @Override public IgniteCache<K, V> withSkipStore() {
         return skipStore();
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
index 9639a9a..a671296 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedLockRequest.java
@@ -310,6 +310,11 @@ public class GridDistributedLockRequest extends GridDistributedBaseMessage {
         return keys;
     }
 
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
     /**
      * @return Max lock wait time.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
index 109d665..c5cf332 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxFinishResponse.java
@@ -86,6 +86,12 @@ public class GridDistributedTxFinishResponse extends GridCacheMessage {
     }
 
     /** {@inheritDoc} */
+    @Override public int partition() {
+        // TODO https://issues.apache.org/jira/browse/IGNITE-4371
+        return Integer.MIN_VALUE;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
index df6acdd..5d70ec1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedUnlockRequest.java
@@ -89,6 +89,11 @@ public class GridDistributedUnlockRequest extends GridDistributedBaseMessage {
         partIds.add(key.partition());
     }
 
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
     /** {@inheritDoc}
      * @param ctx*/
     @Override public void prepareMarshal(GridCacheSharedContext ctx) throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 35e6267..519d0fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -218,7 +218,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
     @Override public void onKernalStart() throws IgniteCheckedException {
         super.onKernalStart();
 
-        preldr.onKernalStart();
+        if (preldr != null)
+            preldr.onKernalStart();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
index 1e92b54..63e3309 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockResponse.java
@@ -22,6 +22,7 @@ import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.HashSet;
 import java.util.List;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.GridDirectCollection;
@@ -31,7 +32,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.GridDistributedLockResponse;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
 import org.apache.ignite.internal.processors.cache.version.GridCacheVersion;
-import org.apache.ignite.internal.util.GridLeanSet;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.lang.IgniteUuid;
@@ -57,7 +57,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
     /** Invalid partitions. */
     @GridToStringInclude
     @GridDirectCollection(int.class)
-    private Collection<Integer> invalidParts = new GridLeanSet<>();
+    private Collection<Integer> invalidParts;
 
     /** Preload entries. */
     @GridDirectCollection(GridCacheEntryInfo.class)
@@ -127,6 +127,9 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
      * @param part Invalid partition.
      */
     public void addInvalidPartition(int part) {
+        if (invalidParts == null)
+            invalidParts = new HashSet<>();
+
         invalidParts.add(part);
     }
 
@@ -134,7 +137,7 @@ public class GridDhtLockResponse extends GridDistributedLockResponse {
      * @return Invalid partitions.
      */
     public Collection<Integer> invalidPartitions() {
-        return invalidParts;
+        return invalidParts == null ? Collections.<Integer>emptySet() : invalidParts;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 940c74e..0e60ff4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -613,8 +613,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteInternalFuture<Boolean> putAsync0(K key, V val, @Nullable CacheEntryPredicate filter) {
-        A.notNull(key, "key", val, "val");
-
         return updateAsync0(
             key,
             val,
@@ -814,6 +812,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      */
     @SuppressWarnings("unchecked")
     protected <T> IgniteInternalFuture<T> asyncOp(final CO<IgniteInternalFuture<T>> op) {
+        if (!asyncToggled)
+            return op.apply();
+
         IgniteInternalFuture<T> fail = asyncOpAcquire();
 
         if (fail != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
index a03d948..0af7cf5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicSingleUpdateRequest.java
@@ -235,6 +235,11 @@ public class GridDhtAtomicSingleUpdateRequest extends GridDhtAtomicAbstractUpdat
     }
 
     /** {@inheritDoc} */
+    @Override public int partition() {
+        return partId;
+    }
+
+    /** {@inheritDoc} */
     @Override public int partitionId(int idx) {
         assert idx == 0 : idx;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
index f2fbb0e..1854e52 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateRequest.java
@@ -487,6 +487,11 @@ public class GridDhtAtomicUpdateRequest extends GridDhtAtomicAbstractUpdateReque
     }
 
     /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
+    /** {@inheritDoc} */
     @Override public long conflictExpireTime(int idx) {
         if (conflictExpireTimes != null) {
             assert idx >= 0 && idx < conflictExpireTimes.size();


[20/25] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NIO sessio

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
index 1b11688..87d9225 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicFullUpdateRequest.java
@@ -611,6 +611,11 @@ public class GridNearAtomicFullUpdateRequest extends GridNearAtomicAbstractUpdat
     }
 
     /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean writeTo(ByteBuffer buf, MessageWriter writer) {
         writer.setBuffer(buf);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
index 1c1addd..c3e9fbe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicSingleUpdateRequest.java
@@ -125,6 +125,11 @@ public class GridNearAtomicSingleUpdateRequest extends GridNearAtomicAbstractSin
         );
     }
 
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partId;
+    }
+
     /**
      * @param key Key to add.
      * @param val Optional update value.

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
index fa7f367..4272a4d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetRequest.java
@@ -244,6 +244,11 @@ public class GridNearGetRequest extends GridCacheMessage implements GridCacheDep
         return accessTtl;
     }
 
+    /** {@inheritDoc} */
+    @Override public int partition() {
+        return partIds != null && !partIds.isEmpty() ? partIds.get(0) : -1;
+    }
+
     /**
      * @param ctx Cache context.
      * @throws IgniteCheckedException If failed.

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index a419887..bc16ff4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -1585,6 +1585,9 @@ public class GridLocalAtomicCache<K, V> extends GridLocalCache<K, V> {
      */
     @SuppressWarnings("unchecked")
     protected IgniteInternalFuture asyncOp(final Callable<?> op) {
+        if (!asyncToggled)
+            return ctx.closures().callLocalSafe(op);
+
         IgniteInternalFuture fail = asyncOpAcquire();
 
         if (fail != null)

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index d34047e..eb5e214 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -272,7 +272,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
                 false,
                 null,
                 req.keyValueFilter(),
-                req.partition(),
+                req.partition() == -1 ? null : req.partition(),
                 req.className(),
                 req.clause(),
                 req.includeMetaData(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
index 60c4662..9f965d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryRequest.java
@@ -121,7 +121,7 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
     private int taskHash;
 
     /** Partition. */
-    private int part;
+    private int part = -1;
 
     /** */
     private AffinityTopologyVersion topVer;
@@ -478,8 +478,8 @@ public class GridCacheQueryRequest extends GridCacheMessage implements GridCache
     /**
      * @return partition.
      */
-    @Nullable public Integer partition() {
-        return part == -1 ? null : part;
+    public int partition() {
+        return part;
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 6d21dcf..393fb1a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -391,7 +391,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<Void> loadMissing(
         final GridCacheContext cacheCtx,
-        AffinityTopologyVersion topVer,
+        final AffinityTopologyVersion topVer,
         final boolean readThrough,
         boolean async,
         final Collection<KeyCacheObject> keys,
@@ -472,7 +472,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                             CacheObject cacheVal = cacheCtx.toCacheObject(val);
 
                             while (true) {
-                                GridCacheEntryEx entry = cacheCtx.cache().entryEx(key);
+                                GridCacheEntryEx entry = cacheCtx.cache().entryEx(key, topVer);
 
                                 try {
                                     GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null);
@@ -1507,7 +1507,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
 
                         assert txEntry != null || readCommitted() || skipVals;
 
-                        GridCacheEntryEx e = txEntry == null ? entryEx(cacheCtx, txKey) : txEntry.cached();
+                        GridCacheEntryEx e = txEntry == null ? entryEx(cacheCtx, txKey, topVer) : txEntry.cached();
 
                         if (readCommitted() || skipVals) {
                             cacheCtx.evicts().touch(e, topologyVersion());
@@ -1658,7 +1658,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                                             IgniteTxLocalAdapter.this,
                                             /*swap*/cacheCtx.isSwapOrOffheapEnabled(),
                                             /*unmarshal*/true,
-                                            /**update-metrics*/true,
+                                            /*update-metrics*/true,
                                             /*event*/!skipVals,
                                             CU.subjectId(IgniteTxLocalAdapter.this, cctx),
                                             transformClo,

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
index 32fda87..fee4dd6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/datastreamer/DataStreamProcessor.java
@@ -328,6 +328,8 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
             if (!allowOverwrite)
                 cctx.topology().readLock();
 
+            GridDhtTopologyFuture topWaitFut = null;
+
             try {
                 GridDhtTopologyFuture fut = cctx.topologyVersionFuture();
 
@@ -352,19 +354,25 @@ public class DataStreamProcessor<K, V> extends GridProcessorAdapter {
 
                     waitFut = allowOverwrite ? null : cctx.mvcc().addDataStreamerFuture(topVer);
                 }
-                else {
-                    fut.listen(new IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>() {
-                        @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> e) {
-                            localUpdate(nodeId, req, updater, topic);
-                        }
-                    });
-                }
+                else
+                    topWaitFut = fut;
             }
             finally {
                 if (!allowOverwrite)
                     cctx.topology().readUnlock();
             }
 
+            if (topWaitFut != null) {
+                // Need call 'listen' after topology read lock is released.
+                topWaitFut.listen(new IgniteInClosure<IgniteInternalFuture<AffinityTopologyVersion>>() {
+                    @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> e) {
+                        localUpdate(nodeId, req, updater, topic);
+                    }
+                });
+
+                return;
+            }
+
             if (job != null) {
                 try {
                     job.call();

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
index 3405b53..4c037b7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsContext.java
@@ -226,7 +226,7 @@ public class IgfsContext {
      */
     public void runInIgfsThreadPool(Runnable r) {
         try {
-            igfsSvc.submit(r);
+            igfsSvc.execute(r);
         }
         catch (RejectedExecutionException ignored) {
             // This exception will happen if network speed is too low and data comes faster
@@ -252,4 +252,4 @@ public class IgfsContext {
 
         return mgr;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
index e534800..4490a68 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsDataManager.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.concurrent.Executor;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteDataStreamer;
 import org.apache.ignite.IgniteException;
@@ -36,6 +37,7 @@ import org.apache.ignite.igfs.secondary.IgfsSecondaryFileSystemPositionedReadabl
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.IgniteInterruptedCheckedException;
 import org.apache.ignite.internal.cluster.ClusterTopologyCheckedException;
+import org.apache.ignite.internal.managers.communication.GridIoPolicy;
 import org.apache.ignite.internal.managers.communication.GridMessageListener;
 import org.apache.ignite.internal.managers.eventstorage.GridLocalEventListener;
 import org.apache.ignite.internal.processors.cache.IgniteInternalCache;
@@ -325,6 +327,8 @@ public class IgfsDataManager extends IgfsManager {
         IgniteInternalFuture<byte[]> fut = dataCachePrj.getAsync(key);
 
         if (secReader != null) {
+            Executor exec = igfsCtx.kernalContext().pools().poolForPolicy(GridIoPolicy.IGFS_POOL);
+
             fut = fut.chain(new CX1<IgniteInternalFuture<byte[]>, byte[]>() {
                 @Override public byte[] applyx(IgniteInternalFuture<byte[]> fut) throws IgniteCheckedException {
                     byte[] res = fut.get();
@@ -365,7 +369,7 @@ public class IgfsDataManager extends IgfsManager {
 
                     return res;
                 }
-            });
+            }, exec);
         }
         else
             igfsCtx.metrics().addReadBlocks(1, 0);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index ab4ee85..6b23e80 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -326,7 +326,7 @@ public final class IgfsImpl implements IgfsEx {
                 // Submit it to the thread pool immediately.
                 assert dualPool != null;
 
-                dualPool.submit(batch);
+                dualPool.execute(batch);
 
                 // Spin in case another batch is currently running.
                 while (true) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
index 9388a8e..7cba9bb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/odbc/OdbcProcessor.java
@@ -139,6 +139,7 @@ public class OdbcProcessor extends GridProcessorAdapter {
                             .logger(log)
                             .selectorCount(DFLT_SELECTOR_CNT)
                             .gridName(ctx.gridName())
+                            .serverName("odbc")
                             .tcpNoDelay(DFLT_TCP_NODELAY)
                             .directBuffer(DFLT_TCP_DIRECT_BUF)
                             .byteOrder(ByteOrder.nativeOrder())

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index fd1c2d4..9d9a4d2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.platform.compute;
 
+import java.util.concurrent.Executor;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteCompute;
 import org.apache.ignite.binary.BinaryObject;
@@ -410,6 +411,11 @@ public class PlatformCompute extends PlatformAbstractTarget {
         }
 
         /** {@inheritDoc} */
+        @Override public IgniteInternalFuture chain(IgniteClosure doneCb, Executor exec) {
+            throw new UnsupportedOperationException("Chain operation is not supported.");
+        }
+
+        /** {@inheritDoc} */
         @Override public Throwable error() {
             return fut.error();
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
index b403654..71eca65 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
@@ -21,7 +21,6 @@ import java.util.HashMap;
 import java.util.Map;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
-import org.apache.ignite.internal.GridKernalContext;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.processors.rest.GridRestCommand;
 import org.apache.ignite.internal.processors.rest.GridRestProtocolHandler;
@@ -38,8 +37,6 @@ import org.apache.ignite.internal.util.nio.GridNioSession;
 import org.apache.ignite.internal.util.typedef.C2;
 import org.apache.ignite.internal.util.typedef.CIX1;
 import org.apache.ignite.internal.util.typedef.internal.U;
-import org.apache.ignite.marshaller.Marshaller;
-import org.apache.ignite.marshaller.jdk.JdkMarshaller;
 import org.jetbrains.annotations.Nullable;
 
 import static org.apache.ignite.internal.processors.rest.GridRestCommand.ATOMIC_DECREMENT;
@@ -72,24 +69,16 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
     /** Handler. */
     private final GridRestProtocolHandler hnd;
 
-    /** JDK marshaller. */
-    private final Marshaller jdkMarshaller = new JdkMarshaller();
-
-    /** Context. */
-    private final GridKernalContext ctx;
-
     /**
      * Creates listener which will convert incoming tcp packets to rest requests and forward them to
      * a given rest handler.
      *
      * @param log Logger to use.
      * @param hnd Rest handler.
-     * @param ctx Context.
      */
-    public GridTcpMemcachedNioListener(IgniteLogger log, GridRestProtocolHandler hnd, GridKernalContext ctx) {
+    public GridTcpMemcachedNioListener(IgniteLogger log, GridRestProtocolHandler hnd) {
         this.log = log;
         this.hnd = hnd;
-        this.ctx = ctx;
     }
 
     /** {@inheritDoc} */
@@ -462,4 +451,4 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
 
         return new GridTuple3<>(cmd, quiet, retKey);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
index 1c1c6dc..3ba6d8e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
@@ -145,7 +145,7 @@ public class GridTcpRestNioListener extends GridNioServerListenerAdapter<GridCli
      */
     public GridTcpRestNioListener(IgniteLogger log, GridTcpRestProtocol proto, GridRestProtocolHandler hnd,
         GridKernalContext ctx) {
-        memcachedLsnr = new GridTcpMemcachedNioListener(log, hnd, ctx);
+        memcachedLsnr = new GridTcpMemcachedNioListener(log, hnd);
         redisLsnr = new GridRedisNioListener(log, hnd, ctx);
 
         this.log = log;

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
index 6338fcc..2a002a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
@@ -257,6 +257,7 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
                 .logger(log)
                 .selectorCount(cfg.getSelectorCount())
                 .gridName(ctx.gridName())
+                .serverName("tcp-rest")
                 .tcpNoDelay(cfg.isNoDelay())
                 .directBuffer(cfg.isDirectBuffer())
                 .byteOrder(ByteOrder.nativeOrder())

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index b9b92b8..9a5f077 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -1131,7 +1131,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
             // Start service in its own thread.
             final ExecutorService exe = svcCtx.executor();
 
-            exe.submit(new Runnable() {
+            exe.execute(new Runnable() {
                 @Override public void run() {
                     try {
                         svc.execute(svcCtx);
@@ -1394,7 +1394,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 return;
 
             try {
-                depExe.submit(new BusyRunnable() {
+                depExe.execute(new BusyRunnable() {
                     @Override public void run0() {
                         onSystemCacheUpdated(deps);
                     }
@@ -1587,7 +1587,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
                 else
                     topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);
 
-                depExe.submit(new BusyRunnable() {
+                depExe.execute(new BusyRunnable() {
                     @Override public void run0() {
                         ClusterNode oldest = ctx.discovery().oldestAliveCacheServerNode(topVer);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index e1937bb..3dfb3c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -153,7 +153,6 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.IgniteInterruptedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.IgniteSystemProperties;
-import org.apache.ignite.binary.BinaryObject;
 import org.apache.ignite.binary.BinaryRawReader;
 import org.apache.ignite.binary.BinaryRawWriter;
 import org.apache.ignite.cluster.ClusterGroupEmptyException;
@@ -506,10 +505,27 @@ public abstract class IgniteUtils {
         }
     };
 
-    /**
-     * Initializes enterprise check.
+    /** */
+    private static final boolean assertionsEnabled;
+
+    /*
+     *
      */
     static {
+        boolean assertionsEnabled0 = true;
+
+        try {
+            assert false;
+
+            assertionsEnabled0 = false;
+        }
+        catch (AssertionError ignored) {
+            assertionsEnabled0 = true;
+        }
+        finally {
+            assertionsEnabled = assertionsEnabled0;
+        }
+
         String osName = System.getProperty("os.name");
 
         String osLow = osName.toLowerCase();
@@ -1284,6 +1300,27 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * @param threadId Thread ID.
+     * @param sb Builder.
+     */
+    public static void printStackTrace(long threadId, GridStringBuilder sb) {
+        ThreadMXBean mxBean = ManagementFactory.getThreadMXBean();
+
+        ThreadInfo threadInfo = mxBean.getThreadInfo(threadId, Integer.MAX_VALUE);
+
+        printThreadInfo(threadInfo, sb, Collections.<Long>emptySet());
+    }
+
+    /**
+     * @return {@code true} if there is java level deadlock.
+     */
+    public static boolean deadlockPresent() {
+        ThreadMXBean mxBean = ManagementFactory.getThreadMXBean();
+
+        return !F.isEmpty(mxBean.findDeadlockedThreads());
+    }
+
+    /**
      * Prints single thread info to a buffer.
      *
      * @param threadInfo Thread info.
@@ -6141,6 +6178,13 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * @return {@code True} if assertions enabled.
+     */
+    public static boolean assertionsEnabled() {
+        return assertionsEnabled;
+    }
+
+    /**
      * Gets OS JDK string.
      *
      * @return OS JDK string.
@@ -8337,6 +8381,18 @@ public abstract class IgniteUtils {
     }
 
     /**
+     * Gets absolute value for long. If argument is {@link Long#MIN_VALUE}, then {@code 0} is returned.
+     *
+     * @param i Argument.
+     * @return Absolute value.
+     */
+    public static long safeAbs(long i) {
+        i = Math.abs(i);
+
+        return i < 0 ? 0 : i;
+    }
+
+    /**
      * Gets wrapper class for a primitive type.
      *
      * @param cls Class. If {@code null}, method is no-op.

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
new file mode 100644
index 0000000..e9ec74b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/StripedExecutor.java
@@ -0,0 +1,667 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.Queue;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Future;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ThreadLocalRandom;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.locks.LockSupport;
+import org.apache.ignite.IgniteInterruptedException;
+import org.apache.ignite.IgniteLogger;
+import org.apache.ignite.internal.util.typedef.internal.A;
+import org.apache.ignite.internal.util.typedef.internal.S;
+import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.thread.IgniteThread;
+import org.jetbrains.annotations.NotNull;
+
+/**
+ * Striped executor.
+ */
+public class StripedExecutor implements ExecutorService {
+    /** Stripes. */
+    private final Stripe[] stripes;
+
+    /** For starvation checks. */
+    private final long[] completedCntrs;
+
+    /** */
+    private final IgniteLogger log;
+
+    /**
+     * Constructor.
+     *
+     * @param cnt Count.
+     */
+    public StripedExecutor(int cnt, String gridName, String poolName, final IgniteLogger log) {
+        A.ensure(cnt > 0, "cnt > 0");
+
+        boolean success = false;
+
+        stripes = new Stripe[cnt];
+
+        completedCntrs = new long[cnt];
+
+        Arrays.fill(completedCntrs, -1);
+
+        this.log = log;
+
+        try {
+            for (int i = 0; i < cnt; i++) {
+                stripes[i] = new StripeConcurrentQueue(
+                    gridName,
+                    poolName,
+                    i,
+                    log);
+
+                stripes[i].start();
+            }
+
+            success = true;
+        }
+        catch (Error | RuntimeException e) {
+            U.error(log, "Failed to initialize striped pool.", e);
+
+            throw e;
+        }
+        finally {
+            if (!success) {
+                for (Stripe stripe : stripes) {
+                    if (stripe != null)
+                        stripe.signalStop();
+                }
+
+                for (Stripe stripe : stripes) {
+                    if (stripe != null)
+                        stripe.awaitStop();
+                }
+            }
+        }
+    }
+
+    /**
+     * Checks starvation in striped pool. Maybe too verbose
+     * but this is needed to faster debug possible issues.
+     */
+    public void checkStarvation() {
+        for (int i = 0; i < stripes.length; i++) {
+            Stripe stripe = stripes[i];
+
+            long completedCnt = stripe.completedCnt;
+
+            boolean active = stripe.active;
+
+            if (completedCntrs[i] != -1 &&
+                completedCntrs[i] == completedCnt &&
+                active) {
+                boolean deadlockPresent = U.deadlockPresent();
+
+                GridStringBuilder sb = new GridStringBuilder();
+
+                sb.a(">>> Possible starvation in striped pool: ")
+                    .a(stripe.thread.getName()).a(U.nl())
+                    .a(stripe.queueToString()).a(U.nl())
+                    .a("deadlock: ").a(deadlockPresent).a(U.nl())
+                    .a("completed: ").a(completedCnt).a(U.nl());
+
+                U.printStackTrace(
+                    stripe.thread.getId(),
+                    sb);
+
+                String msg = sb.toString();
+
+                U.warn(log, msg);
+            }
+
+            if (active || completedCnt > 0)
+                completedCntrs[i] = completedCnt;
+        }
+    }
+
+    /**
+     * @return Stripes count.
+     */
+    public int stripes() {
+        return stripes.length;
+    }
+
+    /**
+     * Execute command.
+     *
+     * @param idx Index.
+     * @param cmd Command.
+     */
+    public void execute(int idx, Runnable cmd) {
+        if (idx == -1)
+            execute(cmd);
+        else {
+            assert idx >= 0 : idx;
+
+            stripes[idx % stripes.length].execute(cmd);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override public void shutdown() {
+        signalStop();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void execute(@NotNull Runnable cmd) {
+        stripes[ThreadLocalRandom.current().nextInt(stripes.length)].execute(cmd);
+    }
+
+    /**
+     * {@inheritDoc}
+     *
+     * @return Empty list (always).
+     */
+    @NotNull @Override public List<Runnable> shutdownNow() {
+        signalStop();
+
+        return Collections.emptyList();
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean awaitTermination(
+        long timeout,
+        @NotNull TimeUnit unit
+    ) throws InterruptedException {
+        awaitStop();
+
+        return true;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isShutdown() {
+        for (Stripe stripe : stripes) {
+            if (stripe != null && stripe.stopping)
+                return true;
+        }
+
+        return false;
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isTerminated() {
+        for (Stripe stripe : stripes) {
+            if (stripe.thread.getState() != Thread.State.TERMINATED)
+                return false;
+        }
+
+        return true;
+    }
+
+    /**
+     * Stops executor.
+     */
+    public void stop() {
+        signalStop();
+
+        awaitStop();
+    }
+
+    /**
+     * Signals all stripes.
+     */
+    private void signalStop() {
+        for (Stripe stripe : stripes)
+            stripe.signalStop();
+    }
+
+    /**
+     * @throws IgniteInterruptedException If interrupted.
+     */
+    private void awaitStop() throws IgniteInterruptedException {
+        for (Stripe stripe : stripes)
+            stripe.awaitStop();
+    }
+
+    /**
+     * @return Return total queue size of all stripes.
+     */
+    public int queueSize() {
+        int size = 0;
+
+        for (Stripe stripe : stripes)
+            size += stripe.queueSize();
+
+        return size;
+    }
+
+    /**
+     * @return Completed tasks count.
+     */
+    public long completedTasks() {
+        long cnt = 0;
+
+        for (Stripe stripe : stripes)
+            cnt += stripe.completedCnt;
+
+        return cnt;
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> Future<T> submit(
+        @NotNull Runnable task,
+        T res
+    ) {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public Future<?> submit(@NotNull Runnable task) {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> Future<T> submit(@NotNull Callable<T> task) {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> List<Future<T>> invokeAll(@NotNull Collection<? extends Callable<T>> tasks)
+        throws InterruptedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> List<Future<T>> invokeAll(
+        @NotNull Collection<? extends Callable<T>> tasks,
+        long timeout,
+        @NotNull TimeUnit unit
+    ) throws InterruptedException {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @NotNull @Override public <T> T invokeAny(@NotNull Collection<? extends Callable<T>> tasks)
+        throws InterruptedException, ExecutionException {
+        throw new UnsupportedOperationException();
+    }
+
+    /**
+     * Operation not supported.
+     */
+    @Override public <T> T invokeAny(
+        @NotNull Collection<? extends Callable<T>> tasks,
+        long timeout,
+        @NotNull TimeUnit unit
+    ) throws InterruptedException, ExecutionException, TimeoutException {
+        throw new UnsupportedOperationException();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return S.toString(StripedExecutor.class, this);
+    }
+
+    /**
+     * Stripe.
+     */
+    private static abstract class Stripe implements Runnable {
+        /** */
+        private final String gridName;
+
+        /** */
+        private final String poolName;
+
+        /** */
+        private final int idx;
+
+        /** */
+        private final IgniteLogger log;
+
+        /** Stopping flag. */
+        private volatile boolean stopping;
+
+        /** */
+        private volatile long completedCnt;
+
+        /** */
+        private volatile boolean active;
+
+        /** Thread executing the loop. */
+        protected Thread thread;
+
+        /**
+         * @param gridName Grid name.
+         * @param poolName Pool name.
+         * @param idx Stripe index.
+         * @param log Logger.
+         */
+        public Stripe(
+            String gridName,
+            String poolName,
+            int idx,
+            IgniteLogger log
+        ) {
+            this.gridName = gridName;
+            this.poolName = poolName;
+            this.idx = idx;
+            this.log = log;
+        }
+
+        /**
+         * Starts the stripe.
+         */
+        void start() {
+            thread = new IgniteThread(gridName, poolName + "-stripe-" + idx, this);
+
+            thread.start();
+        }
+
+        /**
+         * Stop the stripe.
+         */
+        void signalStop() {
+            stopping = true;
+
+            U.interrupt(thread);
+        }
+
+        /**
+         * Await thread stop.
+         */
+        void awaitStop() {
+            try {
+                if (thread != null)
+                    thread.join();
+            }
+            catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+
+                throw new IgniteInterruptedException(e);
+            }
+        }
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            while (!stopping) {
+                Runnable cmd;
+
+                try {
+                    cmd = take();
+
+                    if (cmd != null) {
+                        active = true;
+
+                        try {
+                            cmd.run();
+                        }
+                        finally {
+                            active = false;
+                            completedCnt++;
+                        }
+                    }
+                }
+                catch (InterruptedException e) {
+                    stopping = true;
+
+                    Thread.currentThread().interrupt();
+
+                    return;
+                }
+                catch (Throwable e) {
+                    U.error(log, "Failed to execute runnable.", e);
+                }
+            }
+        }
+
+        /**
+         * Execute the command.
+         *
+         * @param cmd Command.
+         */
+        abstract void execute(Runnable cmd);
+
+        /**
+         * @return Next runnable.
+         * @throws InterruptedException If interrupted.
+         */
+        abstract Runnable take() throws InterruptedException;
+
+        /**
+         * @return Queue size.
+         */
+        abstract int queueSize();
+
+        /**
+         * @return Stripe's queue to string presentation.
+         */
+        abstract String queueToString();
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(Stripe.class, this);
+        }
+    }
+
+    /**
+     * Stripe.
+     */
+    private static class StripeConcurrentQueue extends Stripe {
+        /** Queue. */
+        private final Queue<Runnable> queue = new ConcurrentLinkedQueue<>();
+
+        /** */
+        private volatile boolean parked;
+
+        /**
+         * @param gridName Grid name.
+         * @param poolName Pool name.
+         * @param idx Stripe index.
+         * @param log Logger.
+         */
+        public StripeConcurrentQueue(
+            String gridName,
+            String poolName,
+            int idx,
+            IgniteLogger log
+        ) {
+            super(gridName,
+                poolName,
+                idx,
+                log);
+        }
+
+        /** {@inheritDoc} */
+        @Override Runnable take() throws InterruptedException {
+            Runnable r;
+
+            for (int i = 0; i < 2048; i++) {
+                r = queue.poll();
+
+                if (r != null)
+                    return r;
+            }
+
+            parked = true;
+
+            try {
+                for (;;) {
+                    r = queue.poll();
+
+                    if (r != null)
+                        return r;
+
+                    LockSupport.park();
+
+                    if (Thread.interrupted())
+                        throw new InterruptedException();
+                }
+            }
+            finally {
+                parked = false;
+            }
+        }
+
+        /** {@inheritDoc} */
+        void execute(Runnable cmd) {
+            queue.add(cmd);
+
+            if (parked)
+                LockSupport.unpark(thread);
+        }
+
+        /** {@inheritDoc} */
+        @Override String queueToString() {
+            return String.valueOf(queue);
+        }
+
+        /** {@inheritDoc} */
+        @Override int queueSize() {
+            return queue.size();
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(StripeConcurrentQueue.class, this, super.toString());
+        }
+    }
+
+    /**
+     * Stripe.
+     */
+    private static class StripeConcurrentQueueNoPark extends Stripe {
+        /** Queue. */
+        private final Queue<Runnable> queue = new ConcurrentLinkedQueue<>();
+
+        /**
+         * @param gridName Grid name.
+         * @param poolName Pool name.
+         * @param idx Stripe index.
+         * @param log Logger.
+         */
+        public StripeConcurrentQueueNoPark(
+            String gridName,
+            String poolName,
+            int idx,
+            IgniteLogger log
+        ) {
+            super(gridName,
+                poolName,
+                idx,
+                log);
+        }
+
+        /** {@inheritDoc} */
+        @Override Runnable take() {
+            for (;;) {
+                Runnable r = queue.poll();
+
+                if (r != null)
+                    return r;
+            }
+        }
+
+        /** {@inheritDoc} */
+        void execute(Runnable cmd) {
+            queue.add(cmd);
+        }
+
+        /** {@inheritDoc} */
+        @Override int queueSize() {
+            return queue.size();
+        }
+
+        /** {@inheritDoc} */
+        @Override String queueToString() {
+            return String.valueOf(queue);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(StripeConcurrentQueueNoPark.class, this, super.toString());
+        }
+    }
+
+    /**
+     * Stripe.
+     */
+    private static class StripeConcurrentBlockingQueue extends Stripe {
+        /** Queue. */
+        private final BlockingQueue<Runnable> queue = new LinkedBlockingQueue<>();
+
+        /**
+         * @param gridName Grid name.
+         * @param poolName Pool name.
+         * @param idx Stripe index.
+         * @param log Logger.
+         */
+        public StripeConcurrentBlockingQueue(
+            String gridName,
+            String poolName,
+            int idx,
+            IgniteLogger log
+        ) {
+            super(gridName,
+                poolName,
+                idx,
+                log);
+        }
+
+        /** {@inheritDoc} */
+        @Override Runnable take() throws InterruptedException {
+            return queue.take();
+        }
+
+        /** {@inheritDoc} */
+        void execute(Runnable cmd) {
+            queue.add(cmd);
+        }
+
+        /** {@inheritDoc} */
+        @Override int queueSize() {
+            return queue.size();
+        }
+
+        /** {@inheritDoc} */
+        @Override String queueToString() {
+            return String.valueOf(queue);
+        }
+
+        /** {@inheritDoc} */
+        @Override public String toString() {
+            return S.toString(StripeConcurrentBlockingQueue.class, this, super.toString());
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
index 6baedbd..dc63adc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.util.future;
 
+import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
@@ -152,6 +153,29 @@ public class GridFinishedFuture<T> implements IgniteInternalFuture<T> {
     }
 
     /** {@inheritDoc} */
+    @Override public <T1> IgniteInternalFuture<T1> chain(final IgniteClosure<? super IgniteInternalFuture<T>, T1> doneCb, Executor exec) {
+        final GridFutureAdapter<T1> fut = new GridFutureAdapter<>();
+
+        exec.execute(new Runnable() {
+            @Override public void run() {
+                try {
+                    fut.onDone(doneCb.apply(GridFinishedFuture.this));
+                }
+                catch (GridClosureException e) {
+                    fut.onDone(e.unwrap());
+                }
+                catch (RuntimeException | Error e) {
+                    fut.onDone(e);
+
+                    throw e;
+                }
+            }
+        });
+
+        return fut;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridFinishedFuture.class, this);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
index 2cd534e..c8d85cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.future;
 
 import java.util.Arrays;
+import java.util.concurrent.Executor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.locks.AbstractQueuedSynchronizer;
 import org.apache.ignite.IgniteCheckedException;
@@ -229,7 +230,13 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
 
     /** {@inheritDoc} */
     @Override public <T> IgniteInternalFuture<T> chain(final IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb) {
-        return new ChainFuture<>(this, doneCb);
+        return new ChainFuture<>(this, doneCb, null);
+    }
+
+    /** {@inheritDoc} */
+    @Override public <T> IgniteInternalFuture<T> chain(final IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb,
+        Executor exec) {
+        return new ChainFuture<>(this, doneCb, exec);
     }
 
     /**
@@ -487,15 +494,17 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
         /**
          * @param fut Future.
          * @param doneCb Closure.
+         * @param cbExec Optional executor to run callback.
          */
         ChainFuture(
             GridFutureAdapter<R> fut,
-            IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb
+            IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb,
+            @Nullable Executor cbExec
         ) {
             this.fut = fut;
             this.doneCb = doneCb;
 
-            fut.listen(new GridFutureChainListener<>(this, doneCb));
+            fut.listen(new GridFutureChainListener<>(this, doneCb, cbExec));
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
index 947b2ad..367f5d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
@@ -17,15 +17,17 @@
 
 package org.apache.ignite.internal.util.future;
 
+import java.util.concurrent.Executor;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.lang.GridClosureException;
 import org.apache.ignite.lang.IgniteClosure;
 import org.apache.ignite.lang.IgniteInClosure;
+import org.jetbrains.annotations.Nullable;
 
 /**
  * Future listener to fill chained future with converted result of the source future.
  */
-public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInternalFuture<T>> {
+class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInternalFuture<T>> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -35,21 +37,43 @@ public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInte
     /** Done callback. */
     private final IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb;
 
+    /** */
+    private Executor cbExec;
+
     /**
      * Constructs chain listener.
+     *
      *  @param fut Target future.
      * @param doneCb Done callback.
+     * @param cbExec Optional executor to run callback.
      */
     public GridFutureChainListener(
         GridFutureAdapter<R> fut,
-        IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb
+        IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb,
+        @Nullable Executor cbExec
     ) {
         this.fut = fut;
         this.doneCb = doneCb;
+        this.cbExec = cbExec;
     }
 
     /** {@inheritDoc} */
-    @Override public void apply(IgniteInternalFuture<T> t) {
+    @Override public void apply(final IgniteInternalFuture<T> t) {
+        if (cbExec != null) {
+            cbExec.execute(new Runnable() {
+                @Override public void run() {
+                    applyCallback(t);
+                }
+            });
+        }
+        else
+            applyCallback(t);
+    }
+
+    /**
+     * @param t Target future.
+     */
+    private void applyCallback(IgniteInternalFuture<T> t) {
         try {
             fut.onDone(doneCb.apply(t));
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
index 6820dc7..d108b56 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/IpcToNioAdapter.java
@@ -201,7 +201,7 @@ public class IpcToNioAdapter<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
             assert ses == IpcToNioAdapter.this.ses;
 
             return send((Message)msg);

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
index 9b014ec..f2ab932 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridAbstractCommunicationClient.java
@@ -35,14 +35,24 @@ public abstract class GridAbstractCommunicationClient implements GridCommunicati
     /** Metrics listener. */
     protected final GridNioMetricsListener metricsLsnr;
 
+    /** */
+    private final int connIdx;
+
     /**
+     * @param connIdx Connection index.
      * @param metricsLsnr Metrics listener.
      */
-    protected GridAbstractCommunicationClient(@Nullable GridNioMetricsListener metricsLsnr) {
+    protected GridAbstractCommunicationClient(int connIdx, @Nullable GridNioMetricsListener metricsLsnr) {
+        this.connIdx = connIdx;
         this.metricsLsnr = metricsLsnr;
     }
 
     /** {@inheritDoc} */
+    @Override public int connectionIndex() {
+        return connIdx;
+    }
+
+    /** {@inheritDoc} */
     @Override public boolean close() {
         return reserves.compareAndSet(0, -1);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
index 0de54e9..71b2c24 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridCommunicationClient.java
@@ -96,15 +96,20 @@ public interface GridCommunicationClient {
     /**
      * @param nodeId Remote node ID. Provided only for sync clients.
      * @param msg Message to send.
-     * @param closure Ack closure.
+     * @param c Ack closure.
      * @throws IgniteCheckedException If failed.
      * @return {@code True} if should try to resend message.
      */
-    public boolean sendMessage(@Nullable UUID nodeId, Message msg, @Nullable IgniteInClosure<IgniteException> closure)
+    public boolean sendMessage(@Nullable UUID nodeId, Message msg, @Nullable IgniteInClosure<IgniteException> c)
         throws IgniteCheckedException;
 
     /**
      * @return {@code True} if send is asynchronous.
      */
     public boolean async();
+
+    /**
+     * @return Connection index.
+     */
+    public int connectionIndex();
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
index 213fd8d..7987d3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
@@ -62,13 +62,20 @@ public class GridConnectionBytesVerifyFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
+    @Override public void onExceptionCaught(
+        GridNioSession ses,
+        IgniteCheckedException ex
+    ) throws IgniteCheckedException {
         proceedExceptionCaught(ses, ex);
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
-        return proceedSessionWrite(ses, msg);
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
+        return proceedSessionWrite(ses, msg, fut);
     }
 
     /** {@inheritDoc} */
@@ -137,4 +144,4 @@ public class GridConnectionBytesVerifyFilter extends GridNioFilterAdapter {
     @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
         proceedSessionWriteTimeout(ses);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
index 9925d2e..40c87cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioAsyncNotifyFilter.java
@@ -107,8 +107,12 @@ public class GridNioAsyncNotifyFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
-        return proceedSessionWrite(ses, msg);
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
+        return proceedSessionWrite(ses, msg, fut);
     }
 
     /** {@inheritDoc} */
@@ -139,4 +143,4 @@ public class GridNioAsyncNotifyFilter extends GridNioFilterAdapter {
                 "originalEx=" + ex + ", ex=" + e + ']');
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
index 7083ccf..343e625 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
@@ -71,20 +71,27 @@ public class GridNioCodecFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onExceptionCaught(GridNioSession ses, IgniteCheckedException ex) throws IgniteCheckedException {
+    @Override public void onExceptionCaught(
+        GridNioSession ses,
+        IgniteCheckedException ex
+    ) throws IgniteCheckedException {
         proceedExceptionCaught(ses, ex);
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
         // No encoding needed in direct mode.
         if (directMode)
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
 
         try {
             ByteBuffer res = parser.encode(ses, msg);
 
-            return proceedSessionWrite(ses, res);
+            return proceedSessionWrite(ses, res, fut);
         }
         catch (IOException e) {
             throw new GridNioException(e);
@@ -137,4 +144,4 @@ public class GridNioCodecFilter extends GridNioFilterAdapter {
     @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
         proceedSessionWriteTimeout(ses);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
index 5f88b1f..f7928c4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilter.java
@@ -105,10 +105,15 @@ public interface GridNioFilter {
      *
      * @param ses Session instance.
      * @param msg Message to send.
-     * @return Write future.
+     * @param fut {@code True} if write future should be created.
+     * @return Write future or {@code null}.
      * @throws IgniteCheckedException If filter is not in chain or GridNioException occurred in the underlying filter.
      */
-    public GridNioFuture<?> proceedSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException;
+    public GridNioFuture<?> proceedSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException;
 
     /**
      * Forwards session close request to the next logical filter in filter chain.
@@ -149,10 +154,11 @@ public interface GridNioFilter {
      *
      * @param ses Session on which message should be written.
      * @param msg Message being written.
-     * @return Write future.
+     * @param fut {@code True} if write future should be created.
+     * @return Write future or {@code null}.
      * @throws GridNioException If GridNioException occurred while handling event.
      */
-    public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException;
+    public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException;
 
     /**
      * Invoked when a new messages received.
@@ -241,4 +247,4 @@ public interface GridNioFilter {
      * @throws IgniteCheckedException If filter is not in chain or GridNioException occurred in the underlying filter.
      */
     public GridNioFuture<?> onResumeReads(GridNioSession ses) throws IgniteCheckedException;
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
index 18ab1b2..58ddae5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterAdapter.java
@@ -108,10 +108,14 @@ public abstract class GridNioFilterAdapter implements GridNioFilter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> proceedSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+    @Override public GridNioFuture<?> proceedSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
         checkNext();
 
-        return nextFilter.onSessionWrite(ses, msg);
+        return nextFilter.onSessionWrite(ses, msg, fut);
     }
 
     /** {@inheritDoc} */
@@ -180,4 +184,4 @@ public abstract class GridNioFilterAdapter implements GridNioFilter {
             throw new GridNioException("Failed to proceed with filter call since previous filter is not set " +
                 "(do you use filter outside the filter chain?): " + getClass().getName());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
index a3a74e3..8cc690b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
@@ -181,8 +181,12 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
      * @return Send future.
      * @throws IgniteCheckedException If IgniteCheckedException occurred while handling event.
      */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
-        return tail.onSessionWrite(ses, msg);
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
+        return tail.onSessionWrite(ses, msg, fut);
     }
 
     /**
@@ -255,9 +259,9 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg)
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut)
             throws IgniteCheckedException {
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
         }
 
         /** {@inheritDoc} */
@@ -290,4 +294,4 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
             return proceedResumeReads(ses);
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
index b02acc8..6c0c9c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
@@ -45,9 +45,9 @@ public interface GridNioFuture<R> extends IgniteInternalFuture<R> {
     /**
      * Sets ack closure which will be applied when ack received.
      *
-     * @param closure Ack closure.
+     * @param c Ack closure.
      */
-    public void ackClosure(IgniteInClosure<IgniteException> closure);
+    public void ackClosure(IgniteInClosure<IgniteException> c);
 
     /**
      * The method will be called when ack received.

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
index 35480ac..6258c13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioRecoveryDescriptor.java
@@ -35,8 +35,8 @@ public class GridNioRecoveryDescriptor {
     /** Number of acknowledged messages. */
     private long acked;
 
-    /** Unacknowledged message futures. */
-    private final ArrayDeque<GridNioFuture<?>> msgFuts;
+    /** Unacknowledged messages. */
+    private final ArrayDeque<SessionWriteRequest> msgReqs;
 
     /** Number of messages to resend. */
     private int resendCnt;
@@ -77,23 +77,40 @@ public class GridNioRecoveryDescriptor {
     /** Number of descriptor reservations (for info purposes). */
     private int reserveCnt;
 
+    /** */
+    private final boolean pairedConnections;
+
     /**
+     * @param pairedConnections {@code True} if in/out connections pair is used for communication with node.
      * @param queueLimit Maximum size of unacknowledged messages queue.
      * @param node Node.
      * @param log Logger.
      */
-    public GridNioRecoveryDescriptor(int queueLimit, ClusterNode node, IgniteLogger log) {
+    public GridNioRecoveryDescriptor(
+        boolean pairedConnections,
+        int queueLimit,
+        ClusterNode node,
+        IgniteLogger log
+    ) {
         assert !node.isLocal() : node;
         assert queueLimit > 0;
 
-        msgFuts = new ArrayDeque<>(queueLimit);
+        msgReqs = new ArrayDeque<>(queueLimit);
 
+        this.pairedConnections = pairedConnections;
         this.queueLimit = queueLimit;
         this.node = node;
         this.log = log;
     }
 
     /**
+     * @return {@code True} if in/out connections pair is used for communication with node.
+     */
+    public boolean pairedConnections() {
+        return pairedConnections;
+    }
+
+    /**
      * @return Connect count.
      */
     public long incrementConnectCount() {
@@ -154,19 +171,19 @@ public class GridNioRecoveryDescriptor {
     }
 
     /**
-     * @param fut NIO future.
+     * @param req Write request.
      * @return {@code False} if queue limit is exceeded.
      */
-    public boolean add(GridNioFuture<?> fut) {
-        assert fut != null;
+    public boolean add(SessionWriteRequest req) {
+        assert req != null;
 
-        if (!fut.skipRecovery()) {
+        if (!req.skipRecovery()) {
             if (resendCnt == 0) {
-                msgFuts.addLast(fut);
+                msgReqs.addLast(req);
 
                 sentCnt++;
 
-                return msgFuts.size() < queueLimit;
+                return msgReqs.size() < queueLimit;
             }
             else
                 resendCnt--;
@@ -181,21 +198,19 @@ public class GridNioRecoveryDescriptor {
     public void ackReceived(long rcvCnt) {
         if (log.isDebugEnabled())
             log.debug("Handle acknowledgment [acked=" + acked + ", rcvCnt=" + rcvCnt +
-                ", msgFuts=" + msgFuts.size() + ']');
+                ", msgReqs=" + msgReqs.size() + ']');
 
         while (acked < rcvCnt) {
-            GridNioFuture<?> fut = msgFuts.pollFirst();
+            SessionWriteRequest req = msgReqs.pollFirst();
 
-            assert fut != null : "Missed message future [rcvCnt=" + rcvCnt +
+            assert req != null : "Missed message [rcvCnt=" + rcvCnt +
                 ", acked=" + acked +
                 ", desc=" + this + ']';
 
-            assert fut.isDone() : fut;
-
-            if (fut.ackClosure() != null)
-                fut.ackClosure().apply(null);
+            if (req.ackClosure() != null)
+                req.ackClosure().apply(null);
 
-            fut.onAckReceived();
+            req.onAckReceived();
 
             acked++;
         }
@@ -214,7 +229,7 @@ public class GridNioRecoveryDescriptor {
      * @return {@code False} if descriptor is reserved.
      */
     public boolean onNodeLeft() {
-        GridNioFuture<?>[] futs = null;
+        SessionWriteRequest[] reqs = null;
 
         synchronized (this) {
             nodeLeft = true;
@@ -222,24 +237,24 @@ public class GridNioRecoveryDescriptor {
             if (reserved)
                 return false;
 
-            if (!msgFuts.isEmpty()) {
-                futs = msgFuts.toArray(new GridNioFuture<?>[msgFuts.size()]);
+            if (!msgReqs.isEmpty()) {
+                reqs = msgReqs.toArray(new SessionWriteRequest[msgReqs.size()]);
 
-                msgFuts.clear();
+                msgReqs.clear();
             }
         }
 
-        if (futs != null)
-            completeOnNodeLeft(futs);
+        if (reqs != null)
+            notifyOnNodeLeft(reqs);
 
         return true;
     }
 
     /**
-     * @return Message futures for unacknowledged messages.
+     * @return Requests for unacknowledged messages.
      */
-    public Deque<GridNioFuture<?>> messagesFutures() {
-        return msgFuts;
+    public Deque<SessionWriteRequest> messagesRequests() {
+        return msgReqs;
     }
 
     /**
@@ -277,14 +292,14 @@ public class GridNioRecoveryDescriptor {
             if (!nodeLeft)
                 ackReceived(rcvCnt);
 
-            resendCnt = msgFuts.size();
+            resendCnt = msgReqs.size();
         }
     }
 
     /**
      *
      */
-    public void connected() {
+    public void onConnected() {
         synchronized (this) {
             assert reserved : this;
             assert !connected : this;
@@ -306,10 +321,37 @@ public class GridNioRecoveryDescriptor {
     }
 
     /**
+     * @return Connected flag.
+     */
+    public boolean connected() {
+        synchronized (this) {
+            return connected;
+        }
+    }
+
+    /**
+     * @return Reserved flag.
+     */
+    public boolean reserved() {
+        synchronized (this) {
+            return reserved;
+        }
+    }
+
+    /**
+     * @return Current handshake index.
+     */
+    public Long handshakeIndex() {
+        synchronized (this) {
+            return handshakeReq != null ? handshakeReq.get1() : null;
+        }
+    }
+
+    /**
      *
      */
     public void release() {
-        GridNioFuture<?>[] futs = null;
+        SessionWriteRequest[] futs = null;
 
         synchronized (this) {
             connected = false;
@@ -329,15 +371,15 @@ public class GridNioRecoveryDescriptor {
                 notifyAll();
             }
 
-            if (nodeLeft && !msgFuts.isEmpty()) {
-                futs = msgFuts.toArray(new GridNioFuture<?>[msgFuts.size()]);
+            if (nodeLeft && !msgReqs.isEmpty()) {
+                futs = msgReqs.toArray(new SessionWriteRequest[msgReqs.size()]);
 
-                msgFuts.clear();
+                msgReqs.clear();
             }
         }
 
         if (futs != null)
-            completeOnNodeLeft(futs);
+            notifyOnNodeLeft(futs);
     }
 
     /**
@@ -398,16 +440,16 @@ public class GridNioRecoveryDescriptor {
     }
 
     /**
-     * @param futs Futures to complete.
+     * @param reqs Requests to notify about error.
      */
-    private void completeOnNodeLeft(GridNioFuture<?>[] futs) {
-        for (GridNioFuture<?> msg : futs) {
-            IOException e = new IOException("Failed to send message, node has left: " + node.id());
+    private void notifyOnNodeLeft(SessionWriteRequest[] reqs) {
+        IOException e = new IOException("Failed to send message, node has left: " + node.id());
 
-            ((GridNioFutureImpl)msg).onDone(e);
+        for (SessionWriteRequest req : reqs) {
+            req.onError(e);
 
-            if (msg.ackClosure() != null)
-                msg.ackClosure().apply(new IgniteException(e));
+            if (req.ackClosure() != null)
+                req.ackClosure().apply(new IgniteException(e));
         }
     }
 


[24/25] ignite git commit: ignite-3699: CreatedExpiryPolicy doesn't work if entry is loaded from store (partial fix)

Posted by sb...@apache.org.
ignite-3699: CreatedExpiryPolicy doesn't work if entry is loaded from store (partial fix)


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

Branch: refs/heads/ignite-4371
Commit: 27a81c865f6574f6c287dad6a401171bf0a9c53a
Parents: fa358cc
Author: Amir Akhmedov <am...@gmail.com>
Authored: Fri Dec 9 14:01:10 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 9 14:01:10 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  7 +++-
 .../processors/cache/GridCacheEntryEx.java      |  4 +-
 .../processors/cache/GridCacheMapEntry.java     | 17 +++++++--
 .../distributed/dht/GridDhtLockFuture.java      | 23 ++++++++++-
 .../dht/atomic/GridDhtAtomicCache.java          |  2 +-
 .../transactions/IgniteTxLocalAdapter.java      |  2 +-
 .../processors/cache/GridCacheTestEntryEx.java  |  3 +-
 ...eCacheExpiryPolicyWithStoreAbstractTest.java | 40 ++++++++++++++++++++
 8 files changed, 87 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 1db3931..88aa4e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -1882,7 +1882,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
         @Nullable final UUID subjId,
         final String taskName,
         final boolean deserializeBinary,
-        @Nullable IgniteCacheExpiryPolicy expiry,
+        @Nullable final IgniteCacheExpiryPolicy expiry,
         final boolean skipVals,
         final boolean keepCacheObjects,
         boolean canRemap,
@@ -2017,7 +2017,10 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
                                             GridCacheEntryEx entry = entryEx(key);
 
                                             try {
-                                                GridCacheVersion verSet = entry.versionedValue(cacheVal, ver, null);
+                                                GridCacheVersion verSet = entry.versionedValue(cacheVal,
+                                                    ver,
+                                                    null,
+                                                    expiry);
 
                                                 boolean set = verSet != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index d8194fc..9e9b496 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -725,13 +725,15 @@ public interface GridCacheEntryEx {
      * @param val New value.
      * @param curVer Version to match or {@code null} if match is not required.
      * @param newVer Version to set.
+     * @param loadExpiryPlc Expiry policy if entry is loaded from store.
      * @return Non null version if value was set.
      * @throws IgniteCheckedException If index could not be updated.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     public GridCacheVersion versionedValue(CacheObject val,
         @Nullable GridCacheVersion curVer,
-        @Nullable GridCacheVersion newVer)
+        @Nullable GridCacheVersion newVer,
+        @Nullable IgniteCacheExpiryPolicy loadExpiryPlc)
         throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 31baeda..52b779d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -3551,7 +3551,8 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
     /** {@inheritDoc} */
     @Override public synchronized GridCacheVersion versionedValue(CacheObject val,
         GridCacheVersion curVer,
-        GridCacheVersion newVer)
+        GridCacheVersion newVer,
+        @Nullable IgniteCacheExpiryPolicy loadExpiryPlc)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
 
         checkObsolete();
@@ -3568,9 +3569,19 @@ public abstract class GridCacheMapEntry extends GridMetadataAwareAdapter impleme
 
                 CacheObject old = rawGetOrUnmarshalUnlocked(false);
 
-                long ttl = ttlExtras();
+                long ttl;
+                long expTime;
 
-                long expTime = CU.toExpireTime(ttl);
+                if (loadExpiryPlc != null) {
+                    IgniteBiTuple<Long, Long> initTtlAndExpireTime = initialTtlAndExpireTime(loadExpiryPlc);
+
+                    ttl = initTtlAndExpireTime.get1();
+                    expTime = initTtlAndExpireTime.get2();
+                }
+                else {
+                    ttl = ttlExtras();
+                    expTime = expireTimeExtras();
+                }
 
                 // Detach value before index update.
                 val = cctx.kernalContext().cacheObjects().prepareForCache(val, cctx);

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 4566044..dd18d7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -28,6 +28,7 @@ import java.util.ListIterator;
 import java.util.Map;
 import java.util.UUID;
 import java.util.concurrent.atomic.AtomicReference;
+import javax.cache.expiry.ExpiryPolicy;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.cluster.ClusterNode;
@@ -1059,10 +1060,28 @@ public final class GridDhtLockFuture extends GridCompoundIdentityFuture<Boolean>
                             try {
                                 CacheObject val0 = cctx.toCacheObject(val);
 
+                                long ttl = CU.TTL_ETERNAL;
+                                long expireTime = CU.EXPIRE_TIME_ETERNAL;
+
+                                ExpiryPolicy expiry = cctx.expiry();
+
+                                if (expiry != null) {
+                                    ttl = CU.toTtl(expiry.getExpiryForCreation());
+
+                                    if (ttl == CU.TTL_ZERO)
+                                        expireTime = CU.expireTimeInPast();
+                                    else {
+                                        if (ttl == CU.TTL_NOT_CHANGED)
+                                            ttl = CU.TTL_ETERNAL;
+
+                                        expireTime = CU.toExpireTime(ttl);
+                                    }
+                                }
+
                                 entry0.initialValue(val0,
                                     ver,
-                                    0,
-                                    0,
+                                    ttl,
+                                    expireTime,
                                     false,
                                     topVer,
                                     GridDrType.DR_LOAD,

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 0e60ff4..b291bd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -2292,7 +2292,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         try {
                             GridCacheVersion ver = entry.version();
 
-                            entry.versionedValue(ctx.toCacheObject(v), null, ver);
+                            entry.versionedValue(ctx.toCacheObject(v), null, ver, null);
                         }
                         catch (GridCacheEntryRemovedException e) {
                             assert false : "Entry should not get obsolete while holding lock [entry=" + entry +

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 393fb1a..ba44655 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -475,7 +475,7 @@ public abstract class IgniteTxLocalAdapter extends IgniteTxAdapter implements Ig
                                 GridCacheEntryEx entry = cacheCtx.cache().entryEx(key, topVer);
 
                                 try {
-                                    GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null);
+                                    GridCacheVersion setVer = entry.versionedValue(cacheVal, ver, null, null);
 
                                     boolean set = setVer != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 48621af..e3b4e9b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -667,7 +667,8 @@ public class GridCacheTestEntryEx extends GridMetadataAwareAdapter implements Gr
     /** @inheritDoc */
     @Override public GridCacheVersion versionedValue(CacheObject val,
         GridCacheVersion curVer,
-        GridCacheVersion newVer) {
+        GridCacheVersion newVer,
+        IgniteCacheExpiryPolicy loadExpiryPlc) {
         assert false;
 
         return null;

http://git-wip-us.apache.org/repos/asf/ignite/blob/27a81c86/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
index 78c59ac..58e6b02 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyWithStoreAbstractTest.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.expiry;
 
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.TimeUnit;
 import javax.cache.configuration.Factory;
 import javax.cache.expiry.Duration;
@@ -174,6 +176,44 @@ public abstract class IgniteCacheExpiryPolicyWithStoreAbstractTest extends Ignit
     }
 
     /**
+     * @throws Exception If failed.
+     */
+    public void testGetReadThrough() throws Exception {
+        IgniteCache<Integer, Integer> cache = jcache(0);
+
+        List<Integer> keys = new ArrayList<>();
+
+        keys.add(primaryKeys(cache, 1, 100_000).get(0));
+        // TODO https://issues.apache.org/jira/browse/IGNITE-3699
+        // TODO: test 'get' inside transactions, 'get' for cache.withAsyncPolicy.
+        //keys.add(backupKeys(cache, 1, 100_000).get(0));
+        //keys.add(nearKeys(cache, 1, 100_000).get(0));
+
+        for (Integer key : keys)
+            storeMap.put(key, 100);
+
+        try {
+            for (Integer key : keys) {
+                Integer res = cache.get(key);
+
+                assertEquals((Integer)100, res);
+
+                checkTtl(key, 500, true);
+
+                assertEquals((Integer)100, res);
+            }
+
+            U.sleep(600);
+
+            for (Integer key : keys)
+                checkExpired(key);
+        }
+        finally {
+            cache.removeAll();
+        }
+    }
+
+    /**
      * @param key Key.
      */
     private void checkExpired(Integer key) {


[06/25] ignite git commit: IGNITE-3770 GridLogThrottle.warn ignores the exception

Posted by sb...@apache.org.
IGNITE-3770 GridLogThrottle.warn ignores the exception


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

Branch: refs/heads/ignite-4371
Commit: bbaa79af8ef526b5d2684db0e3d71d60a8f1ebe7
Parents: ac8602d
Author: agura <ag...@apache.org>
Authored: Wed Dec 7 19:36:11 2016 +0300
Committer: agura <ag...@apache.org>
Committed: Wed Dec 7 19:36:11 2016 +0300

----------------------------------------------------------------------
 .../affinity/fair/FairAffinityFunction.java     |  2 +-
 .../rendezvous/RendezvousAffinityFunction.java  |  2 +-
 .../apache/ignite/internal/IgniteKernal.java    |  2 +-
 .../GridDeploymentPerVersionStore.java          |  2 +-
 .../discovery/GridDiscoveryManager.java         |  2 +-
 .../eventstorage/GridEventStorageManager.java   |  2 +-
 .../cache/GridCacheDeploymentManager.java       |  4 +-
 .../processors/cache/GridCacheEventManager.java | 10 ++---
 .../store/GridCacheStoreManagerAdapter.java     |  2 +-
 .../cache/store/GridCacheWriteBehindStore.java  |  2 +-
 .../clock/GridClockSyncProcessor.java           |  2 +-
 .../igfs/IgfsFragmentizerManager.java           | 29 +++++++-------
 .../internal/processors/igfs/IgfsImpl.java      | 35 ++++++++---------
 .../OsDiscoveryNodeValidationProcessor.java     |  2 +-
 .../processors/task/GridTaskWorker.java         |  2 +-
 .../ignite/internal/util/GridLogThrottle.java   | 35 ++++++++---------
 .../ignite/internal/util/IgniteUtils.java       |  6 +--
 .../ipc/shmem/IpcSharedMemoryNativeLoader.java  |  2 +-
 .../shmem/IpcSharedMemoryServerEndpoint.java    |  4 +-
 .../nio/GridConnectionBytesVerifyFilter.java    |  2 +-
 .../internal/util/nio/GridNioCodecFilter.java   |  2 +-
 .../internal/util/nio/GridNioFilterChain.java   |  2 +-
 .../ignite/internal/util/nio/GridNioServer.java |  2 +-
 .../util/nio/GridSelectorNioSessionImpl.java    |  2 +-
 .../communication/tcp/TcpCommunicationSpi.java  | 12 +++---
 .../ignite/spi/discovery/tcp/ClientImpl.java    |  7 ++--
 .../ignite/spi/discovery/tcp/ServerImpl.java    | 41 +++++++++-----------
 .../spi/discovery/tcp/TcpDiscoveryImpl.java     |  2 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |  8 ++--
 .../TcpDiscoveryMulticastIpFinder.java          |  4 +-
 .../ignite/testframework/GridTestUtils.java     |  2 +-
 .../junits/common/GridCommonAbstractTest.java   |  6 +--
 .../apache/ignite/util/GridLogThrottleTest.java | 27 +++++--------
 .../HadoopExternalCommunication.java            |  6 +--
 .../processors/query/h2/IgniteH2Indexing.java   |  2 +-
 .../cache/IgniteCacheOffheapEvictQueryTest.java |  2 +-
 .../scanners/http/UriDeploymentHttpScanner.java |  8 ++--
 37 files changed, 133 insertions(+), 151 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
index 105efab..cffcf10 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/fair/FairAffinityFunction.java
@@ -331,7 +331,7 @@ public class FairAffinityFunction implements AffinityFunction {
                 balance(tier, pendingParts, fullMap, topSnapshot, true);
 
                 if (!exclNeighborsWarn) {
-                    LT.warn(log, null, "Affinity function excludeNeighbors property is ignored " +
+                    LT.warn(log, "Affinity function excludeNeighbors property is ignored " +
                         "because topology has no enough nodes to assign backups.");
 
                     exclNeighborsWarn = true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
index 75e7c92..cbd0136 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
@@ -462,7 +462,7 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
             }
 
             if (!exclNeighborsWarn) {
-                LT.warn(log, null, "Affinity function excludeNeighbors property is ignored " +
+                LT.warn(log, "Affinity function excludeNeighbors property is ignored " +
                     "because topology has no enough nodes to assign backups.");
 
                 exclNeighborsWarn = true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index c536506..8fda72f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -1001,7 +1001,7 @@ public class IgniteKernal implements IgniteEx, IgniteMXBean, Externalizable {
                     // at least one waiting request, then it is possible starvation.
                     if (exec.getPoolSize() == exec.getActiveCount() && completedCnt == lastCompletedCnt &&
                         !exec.getQueue().isEmpty())
-                        LT.warn(log, null, "Possible thread pool starvation detected (no task completed in last " +
+                        LT.warn(log, "Possible thread pool starvation detected (no task completed in last " +
                             interval + "ms, is executorService pool size large enough?)");
 
                     lastCompletedCnt = completedCnt;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java
index 5e30bf6..0bf8328 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/deployment/GridDeploymentPerVersionStore.java
@@ -317,7 +317,7 @@ public class GridDeploymentPerVersionStore extends GridDeploymentStoreAdapter {
                         if (ctx.localNodeId().equals(e.getKey())) {
                             // Warn only if mode is not CONTINUOUS.
                             if (meta.deploymentMode() != CONTINUOUS)
-                                LT.warn(log, null, "Local node is in participants (most probably, " +
+                                LT.warn(log, "Local node is in participants (most probably, " +
                                     "IgniteConfiguration.getPeerClassLoadingLocalClassPathExclude() " +
                                     "is not used properly " +
                                     "[locNodeId=" + ctx.localNodeId() + ", meta=" + meta + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index ddd4ee3..9aa4db1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -995,7 +995,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                 break;
 
             if (ctx.config().isWaitForSegmentOnStart()) {
-                LT.warn(log, null, "Failed to check network segment (retrying every 2000 ms).");
+                LT.warn(log, "Failed to check network segment (retrying every 2000 ms).");
 
                 // Wait and check again.
                 U.sleep(2000);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
index 5b451a1..607bb96 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
@@ -278,7 +278,7 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
             int type = evt.type();
 
             if (!isRecordable(type)) {
-                LT.warn(log, null, "Trying to record event without checking if it is recordable: " +
+                LT.warn(log, "Trying to record event without checking if it is recordable: " +
                     U.gridEventName(type));
             }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
index 8e66233..ad4892b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheDeploymentManager.java
@@ -399,7 +399,7 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
                     ", daemon=" + daemon + ']');
 
             if (!daemon) {
-                LT.warn(log, null, "Ignoring deployment in PRIVATE or ISOLATED mode " +
+                LT.warn(log, "Ignoring deployment in PRIVATE or ISOLATED mode " +
                     "[sndId=" + sndId + ", ldrId=" + ldrId + ", userVer=" + userVer + ", mode=" + mode +
                     ", participants=" + participants + ", daemon=" + daemon + ']');
             }
@@ -408,7 +408,7 @@ public class GridCacheDeploymentManager<K, V> extends GridCacheSharedManagerAdap
         }
 
         if (mode != cctx.gridConfig().getDeploymentMode()) {
-            LT.warn(log, null, "Local and remote deployment mode mismatch (please fix configuration and restart) " +
+            LT.warn(log, "Local and remote deployment mode mismatch (please fix configuration and restart) " +
                 "[locDepMode=" + cctx.gridConfig().getDeploymentMode() + ", rmtDepMode=" + mode + ", rmtNodeId=" +
                 sndId + ']');
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
index ec8b8cc..1c18738 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEventManager.java
@@ -251,7 +251,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
         assert key != null || type == EVT_CACHE_STARTED || type == EVT_CACHE_STOPPED;
 
         if (!cctx.events().isRecordable(type))
-            LT.warn(log, null, "Added event without checking if event is recordable: " + U.gridEventName(type));
+            LT.warn(log, "Added event without checking if event is recordable: " + U.gridEventName(type));
 
         // Events are not fired for internal entry.
         if (key == null || !key.internal()) {
@@ -261,7 +261,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
                 evtNode = findNodeInHistory(evtNodeId);
 
             if (evtNode == null)
-                LT.warn(log, null, "Failed to find event node in grid topology history " +
+                LT.warn(log, "Failed to find event node in grid topology history " +
                     "(try to increase topology history size configuration property of configured " +
                     "discovery SPI): " + evtNodeId);
 
@@ -284,7 +284,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
                     log.debug("Failed to unmarshall cache object value for the event notification: " + e);
 
                 if (!forceKeepBinary)
-                    LT.warn(log, null, "Failed to unmarshall cache object value for the event notification " +
+                    LT.warn(log, "Failed to unmarshall cache object value for the event notification " +
                         "(all further notifications will keep binary object format).");
 
                 forceKeepBinary = true;
@@ -351,7 +351,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
         assert discoTs > 0;
 
         if (!cctx.events().isRecordable(type))
-            LT.warn(log, null, "Added event without checking if event is recordable: " + U.gridEventName(type));
+            LT.warn(log, "Added event without checking if event is recordable: " + U.gridEventName(type));
 
         cctx.gridEvents().record(new CacheRebalancingEvent(cctx.name(), cctx.localNode(),
             "Cache rebalancing event.", type, part, discoNode, discoType, discoTs));
@@ -364,7 +364,7 @@ public class GridCacheEventManager extends GridCacheManagerAdapter {
      */
     public void addUnloadEvent(int part) {
         if (!cctx.events().isRecordable(EVT_CACHE_REBALANCE_PART_UNLOADED))
-            LT.warn(log, null, "Added event without checking if event is recordable: " +
+            LT.warn(log, "Added event without checking if event is recordable: " +
                 U.gridEventName(EVT_CACHE_REBALANCE_PART_UNLOADED));
 
         cctx.gridEvents().record(new CacheRebalancingEvent(cctx.name(), cctx.localNode(),

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
index cd0c50f..024375e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheStoreManagerAdapter.java
@@ -547,7 +547,7 @@ public abstract class GridCacheStoreManagerAdapter extends GridCacheManagerAdapt
             return true;
         }
 
-        LT.warn(log, null, "Calling Cache.loadCache() method will have no effect, " +
+        LT.warn(log, "Calling Cache.loadCache() method will have no effect, " +
             "CacheConfiguration.getStore() is not defined for cache: " + cctx.namexx());
 
         return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
index 468945b..858d9a7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/store/GridCacheWriteBehindStore.java
@@ -705,7 +705,7 @@ public class GridCacheWriteBehindStore<K, V> implements CacheStore<K, V>, Lifecy
             }
         }
         catch (Exception e) {
-            LT.warn(log, e, "Unable to update underlying store: " + store);
+            LT.error(log, e, "Unable to update underlying store: " + store);
 
             if (writeCache.sizex() > cacheCriticalSize || stopping.get()) {
                 for (Map.Entry<K, Entry<? extends K, ? extends  V>> entry : vals.entrySet()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
index b5c89cf..0764316 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/clock/GridClockSyncProcessor.java
@@ -458,7 +458,7 @@ public class GridClockSyncProcessor extends GridProcessorAdapter {
                     srv.sendPacket(req, addr, port);
                 }
                 catch (IgniteCheckedException e) {
-                    LT.warn(log, e, "Failed to send time request to remote node [rmtNodeId=" + rmtNodeId +
+                    LT.error(log, e, "Failed to send time request to remote node [rmtNodeId=" + rmtNodeId +
                         ", addr=" + addr + ", port=" + port + ']');
                 }
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
index d64c64a..2e82f33 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsFragmentizerManager.java
@@ -17,6 +17,19 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.UUID;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.LinkedBlockingDeque;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
 import org.apache.ignite.cluster.ClusterNode;
@@ -41,20 +54,6 @@ import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.thread.IgniteThread;
 import org.jetbrains.annotations.Nullable;
 
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.LinkedList;
-import java.util.Map;
-import java.util.UUID;
-import java.util.concurrent.BlockingQueue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.LinkedBlockingDeque;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.Condition;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
 import static java.util.concurrent.TimeUnit.MILLISECONDS;
 import static org.apache.ignite.events.EventType.EVT_NODE_FAILED;
 import static org.apache.ignite.events.EventType.EVT_NODE_LEFT;
@@ -383,7 +382,7 @@ public class IgfsFragmentizerManager extends IgfsManager {
                 }
                 catch (IgniteCheckedException | IgniteException e) {
                     if (!X.hasCause(e, InterruptedException.class) && !X.hasCause(e, IgniteInterruptedCheckedException.class))
-                        LT.warn(log, e, "Failed to get fragmentizer file info (will retry).");
+                        LT.error(log, e, "Failed to get fragmentizer file info (will retry).");
                     else {
                         if (log.isDebugEnabled())
                             log.debug("Got interrupted exception in fragmentizer coordinator (grid is stopping).");

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
index 1c985c0..ab4ee85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/igfs/IgfsImpl.java
@@ -17,6 +17,21 @@
 
 package org.apache.ignite.internal.processors.igfs;
 
+import java.io.OutputStream;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Callable;
+import java.util.concurrent.SynchronousQueue;
+import java.util.concurrent.ThreadFactory;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
 import org.apache.ignite.Ignite;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.IgniteException;
@@ -81,22 +96,6 @@ import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
-import java.io.OutputStream;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.SynchronousQueue;
-import java.util.concurrent.ThreadFactory;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
 import static org.apache.ignite.events.EventType.EVT_IGFS_DIR_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_DELETED;
 import static org.apache.ignite.events.EventType.EVT_IGFS_FILE_OPENED_READ;
@@ -1308,7 +1307,7 @@ public final class IgfsImpl implements IgfsEx {
                         secondarySpaceSize = secondaryFs.usedSpaceSize();
                     }
                     catch (IgniteException e) {
-                        LT.warn(log, e, "Failed to get secondary file system consumed space size.");
+                        LT.error(log, e, "Failed to get secondary file system consumed space size.");
 
                         secondarySpaceSize = -1;
                     }
@@ -1841,4 +1840,4 @@ public final class IgfsImpl implements IgfsEx {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java
index a7e06e9..37e59bc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/nodevalidation/OsDiscoveryNodeValidationProcessor.java
@@ -58,7 +58,7 @@ public class OsDiscoveryNodeValidationProcessor extends GridProcessorAdapter imp
                     ", rmtNodeAddrs=" + U.addressesAsString(node) +
                     ", locNodeId=" + locNode.id() + ", rmtNodeId=" + node.id() + ']';
 
-                LT.warn(log, null, errMsg);
+                LT.warn(log, errMsg);
 
                 // Always output in debug.
                 if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
index 0be69d1..3478c70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/task/GridTaskWorker.java
@@ -626,7 +626,7 @@ class GridTaskWorker<T, R> extends GridWorker implements GridTimeoutObject {
                 res.setOccupied(true);
 
                 if (resCache && jobRes.size() > ctx.discovery().size() && jobRes.size() % SPLIT_WARN_THRESHOLD == 0)
-                    LT.warn(log, null, "Number of jobs in task is too large for task: " + ses.getTaskName() +
+                    LT.warn(log, "Number of jobs in task is too large for task: " + ses.getTaskName() +
                         ". Consider reducing number of jobs or disabling job result cache with " +
                         "@ComputeTaskNoResultCache annotation.");
             }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
index c8ba865..7f30dd7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridLogThrottle.java
@@ -93,13 +93,12 @@ public class GridLogThrottle {
      * Logs warning if needed.
      *
      * @param log Logger.
-     * @param e Error (optional).
      * @param msg Message.
      */
-    public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String msg) {
+    public static void warn(@Nullable IgniteLogger log, String msg) {
         assert !F.isEmpty(msg);
 
-        log(log, e, msg, null, LogLevel.WARN, false, false);
+        log(log, null, msg, null, LogLevel.WARN, false, false);
     }
 
     /**
@@ -122,28 +121,26 @@ public class GridLogThrottle {
      * Logs warning if needed.
      *
      * @param log Logger.
-     * @param e Error (optional).
      * @param msg Message.
-     * @param quite Print warning anyway.
+     * @param quiet Print warning anyway.
      */
-    public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String msg, boolean quite) {
+    public static void warn(@Nullable IgniteLogger log, String msg, boolean quiet) {
         assert !F.isEmpty(msg);
 
-        log(log, e, msg, null, LogLevel.WARN, quite, false);
+        log(log, null, msg, null, LogLevel.WARN, quiet, false);
     }
 
     /**
      * Logs warning if needed.
      *
      * @param log Logger.
-     * @param e Error (optional).
      * @param longMsg Long message (or just message).
-     * @param shortMsg Short message for quite logging.
+     * @param shortMsg Short message for quiet logging.
      */
-    public static void warn(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, @Nullable String shortMsg) {
+    public static void warn(@Nullable IgniteLogger log, String longMsg, @Nullable String shortMsg) {
         assert !F.isEmpty(longMsg);
 
-        log(log, e, longMsg, shortMsg, LogLevel.WARN, false, false);
+        log(log, null, longMsg, shortMsg, LogLevel.WARN, false, false);
     }
 
     /**
@@ -151,12 +148,12 @@ public class GridLogThrottle {
      *
      * @param log Logger.
      * @param msg Message.
-     * @param quite Print info anyway.
+     * @param quiet Print info anyway.
      */
-    public static void info(@Nullable IgniteLogger log, String msg, boolean quite) {
+    public static void info(@Nullable IgniteLogger log, String msg, boolean quiet) {
         assert !F.isEmpty(msg);
 
-        log(log, null, msg, null, LogLevel.INFO, quite, false);
+        log(log, null, msg, null, LogLevel.INFO, quiet, false);
     }
 
     /**
@@ -166,6 +163,8 @@ public class GridLogThrottle {
      * @param msg Message.
      */
     public static void info(@Nullable IgniteLogger log, String msg) {
+        assert !F.isEmpty(msg);
+
         info(log, msg, false);
     }
 
@@ -182,13 +181,13 @@ public class GridLogThrottle {
      * @param log Logger.
      * @param e Error (optional).
      * @param longMsg Long message (or just message).
-     * @param shortMsg Short message for quite logging.
+     * @param shortMsg Short message for quiet logging.
      * @param level Level where messages should appear.
      * @param byMsg Errors group by message, not by tuple(error, msg).
      */
     @SuppressWarnings({"RedundantTypeArguments"})
-    private static void log(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg, @Nullable String shortMsg,
-        LogLevel level, boolean quiet, boolean byMsg) {
+    private static void log(@Nullable IgniteLogger log, @Nullable Throwable e, String longMsg,
+        @Nullable String shortMsg, LogLevel level, boolean quiet, boolean byMsg) {
         assert !F.isEmpty(longMsg);
 
         IgniteBiTuple<Class<? extends Throwable>, String> tup =
@@ -283,4 +282,4 @@ public class GridLogThrottle {
          */
         public abstract void doLog(IgniteLogger log, String longMsg, String shortMsg, Throwable e, boolean quiet);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
index 914b3ec..3fa3f7b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/IgniteUtils.java
@@ -4105,7 +4105,7 @@ public abstract class IgniteUtils {
     }
 
     /**
-     * Logs warning message in both verbose and quite modes.
+     * Logs warning message in both verbose and quiet modes.
      *
      * @param log Logger to use.
      * @param msg Message to log.
@@ -4115,7 +4115,7 @@ public abstract class IgniteUtils {
     }
 
     /**
-     * Logs warning message in both verbose and quite modes.
+     * Logs warning message in both verbose and quiet modes.
      *
      * @param log Logger to use.
      * @param shortMsg Short message.
@@ -4285,7 +4285,7 @@ public abstract class IgniteUtils {
     }
 
     /**
-     * Prints out the message in quite and info modes.
+     * Prints out the message in quiet and info modes.
      *
      * @param log Logger.
      * @param msg Message to print.

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
index 2771d28..02c4de5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryNativeLoader.java
@@ -150,7 +150,7 @@ public class IpcSharedMemoryNativeLoader {
 
             try {
                 if (log != null)
-                    LT.warn(log, null, "Failed to load 'igniteshmem' library from classpath. Will try to load it from IGNITE_HOME.");
+                    LT.warn(log, "Failed to load 'igniteshmem' library from classpath. Will try to load it from IGNITE_HOME.");
 
                 String igniteHome = X.resolveIgniteHome();
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
index 94c3820..178e608 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/ipc/shmem/IpcSharedMemoryServerEndpoint.java
@@ -304,13 +304,13 @@ public class IpcSharedMemoryServerEndpoint implements IpcServerEndpoint {
                     String msg = "Failed to process incoming connection (most probably, shared memory " +
                         "rest endpoint has been configured by mistake).";
 
-                    LT.warn(log, null, msg);
+                    LT.warn(log, msg);
 
                     sendErrorResponse(out, e);
                 }
                 catch (IpcOutOfSystemResourcesException e) {
                     if (!omitOutOfResourcesWarn)
-                        LT.warn(log, null, OUT_OF_RESOURCES_MSG);
+                        LT.warn(log, OUT_OF_RESOURCES_MSG);
 
                     sendErrorResponse(out, e);
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
index 13d7ca7..213fd8d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridConnectionBytesVerifyFilter.java
@@ -115,7 +115,7 @@ public class GridConnectionBytesVerifyFilter extends GridNioFilterAdapter {
             else {
                 ses.close();
 
-                LT.warn(log, null, "Unknown connection detected (is some other software connecting to this " +
+                LT.warn(log, "Unknown connection detected (is some other software connecting to this " +
                     "Ignite port?) [rmtAddr=" + ses.remoteAddress() + ", locAddr=" + ses.localAddress() + ']');
             }
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
index a2f543d..7083ccf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioCodecFilter.java
@@ -110,7 +110,7 @@ public class GridNioCodecFilter extends GridNioFilterAdapter {
                         if (directMode)
                             return;
 
-                        LT.warn(log, null, "Parser returned null but there are still unread data in input buffer (bug in " +
+                        LT.warn(log, "Parser returned null but there are still unread data in input buffer (bug in " +
                             "parser code?) [parser=" + parser + ", ses=" + ses + ']');
 
                         input.position(input.limit());

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
index 8a43e29..a3a74e3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFilterChain.java
@@ -158,7 +158,7 @@ public class GridNioFilterChain<T> extends GridNioFilterAdapter {
             head.onExceptionCaught(ses, e);
         }
         catch (Exception ex) {
-            LT.warn(log, ex, "Failed to forward GridNioException to filter chain [ses=" + ses + ", e=" + e + ']');
+            LT.error(log, ex, "Failed to forward GridNioException to filter chain [ses=" + ses + ", e=" + e + ']');
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
index 24b8fad..c8e2e0b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioServer.java
@@ -768,7 +768,7 @@ public class GridNioServer<T> {
                 filterChain.onMessageReceived(ses, readBuf);
 
                 if (readBuf.remaining() > 0) {
-                    LT.warn(log, null, "Read buffer contains data after filter chain processing (will discard " +
+                    LT.warn(log, "Read buffer contains data after filter chain processing (will discard " +
                         "remaining bytes) [ses=" + ses + ", remainingCnt=" + readBuf.remaining() + ']');
 
                     readBuf.clear();

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
index 0ba6af2..63c9845 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
@@ -227,7 +227,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
 
             if (recovery != null) {
                 if (!recovery.add(last)) {
-                    LT.warn(log, null, "Unacknowledged messages queue size overflow, will attempt to reconnect " +
+                    LT.warn(log, "Unacknowledged messages queue size overflow, will attempt to reconnect " +
                         "[remoteAddr=" + remoteAddress() +
                         ", queueLimit=" + recovery.queueLimit() + ']');
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/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 767490f..1fe437c 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
@@ -331,7 +331,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
     private final GridNioServerListener<Message> srvLsnr =
         new GridNioServerListenerAdapter<Message>() {
             @Override public void onSessionWriteTimeout(GridNioSession ses) {
-                LT.warn(log, null, "Communication SPI Session write timed out (consider increasing " +
+                LT.warn(log, "Communication SPI Session write timed out (consider increasing " +
                     "'socketWriteTimeout' " + "configuration property) [remoteAddr=" + ses.remoteAddress() +
                     ", writeTimeout=" + sockWriteTimeout + ']');
 
@@ -2146,9 +2146,9 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             catch (IgniteCheckedException e) {
                 if (e.hasCause(IpcOutOfSystemResourcesException.class))
                     // Has cause or is itself the IpcOutOfSystemResourcesException.
-                    LT.warn(log, null, OUT_OF_RESOURCES_TCP_MSG);
+                    LT.warn(log, OUT_OF_RESOURCES_TCP_MSG);
                 else if (getSpiContext().node(node.id()) != null)
-                    LT.warn(log, null, e.getMessage());
+                    LT.warn(log, e.getMessage());
                 else if (log.isDebugEnabled())
                     log.debug("Failed to establish shared memory connection with local node (node has left): " +
                         node.id());
@@ -2510,11 +2510,11 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
                     boolean failureDetThrReached = timeoutHelper.checkFailureTimeoutReached(e);
 
                     if (failureDetThrReached)
-                        LT.warn(log, null, "Connect timed out (consider increasing 'failureDetectionTimeout' " +
+                        LT.warn(log, "Connect timed out (consider increasing 'failureDetectionTimeout' " +
                             "configuration property) [addr=" + addr + ", failureDetectionTimeout=" +
                             failureDetectionTimeout() + ']');
                     else if (X.hasCause(e, SocketTimeoutException.class))
-                        LT.warn(log, null, "Connect timed out (consider increasing 'connTimeout' " +
+                        LT.warn(log, "Connect timed out (consider increasing 'connTimeout' " +
                             "configuration property) [addr=" + addr + ", connTimeout=" + connTimeout + ']');
 
                     if (errs == null)
@@ -2545,7 +2545,7 @@ public class TcpCommunicationSpi extends IgniteSpiAdapter
             assert errs != null;
 
             if (X.hasCause(errs, ConnectException.class))
-                LT.warn(log, null, "Failed to connect to a remote node " +
+                LT.warn(log, "Failed to connect to a remote node " +
                     "(make sure that destination node is alive and " +
                     "operating system firewall is disabled on local and remote hosts) " +
                     "[addrs=" + addrs + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
index 7b8c029..8928f28 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ClientImpl.java
@@ -64,7 +64,6 @@ import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgniteUuid;
-import org.apache.ignite.marshaller.MarshallerUtils;
 import org.apache.ignite.spi.IgniteSpiContext;
 import org.apache.ignite.spi.IgniteSpiException;
 import org.apache.ignite.spi.IgniteSpiOperationTimeoutHelper;
@@ -483,7 +482,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                     if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout)
                         return null;
 
-                    LT.warn(log, null, "IP finder returned empty addresses list. " +
+                    LT.warn(log, "IP finder returned empty addresses list. " +
                             "Please check IP finder configuration" +
                             (spi.ipFinder instanceof TcpDiscoveryMulticastIpFinder ?
                                 " and make sure multicast works on your network. " : ". ") +
@@ -553,7 +552,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                 if (timeout > 0 && (U.currentTimeMillis() - startTime) > timeout)
                     return null;
 
-                LT.warn(log, null, "Failed to connect to any address from IP finder (will retry to join topology " +
+                LT.warn(log, "Failed to connect to any address from IP finder (will retry to join topology " +
                     "every 2 secs): " + toOrderedList(addrs0), true);
 
                 Thread.sleep(2000);
@@ -917,7 +916,7 @@ class ClientImpl extends TcpDiscoveryImpl {
                             ClassNotFoundException clsNotFoundEx = X.cause(e, ClassNotFoundException.class);
 
                             if (clsNotFoundEx != null)
-                                LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
+                                LT.warn(log, "Failed to read message due to ClassNotFoundException " +
                                     "(make sure same versions of all classes are available on all nodes) " +
                                     "[rmtNodeId=" + rmtNodeId + ", err=" + clsNotFoundEx.getMessage() + ']');
                             else

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index a660ec8..204b685 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -54,7 +54,6 @@ import java.util.concurrent.ConcurrentLinkedDeque;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.LinkedBlockingDeque;
 import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import javax.net.ssl.SSLException;
@@ -529,7 +528,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         boolean res = pingNode(node);
 
         if (!res && !node.isClient() && nodeAlive(nodeId)) {
-            LT.warn(log, null, "Failed to ping node (status check will be initiated): " + nodeId);
+            LT.warn(log, "Failed to ping node (status check will be initiated): " + nodeId);
 
             msgWorker.addMessage(new TcpDiscoveryStatusCheckMessage(locNode, node.id()));
         }
@@ -910,7 +909,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         U.addressesAsString(msg.addresses(), msg.hostNames()) + ']');
                 }
                 else
-                    LT.warn(log, null, "Node has not been connected to topology and will repeat join process. " +
+                    LT.warn(log, "Node has not been connected to topology and will repeat join process. " +
                         "Check remote nodes logs for possible error messages. " +
                         "Note that large topology may require significant time to start. " +
                         "Increase 'TcpDiscoverySpi.networkTimeout' configuration property " +
@@ -1059,7 +1058,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                 }
 
                 if (e != null && X.hasCause(e, ConnectException.class)) {
-                    LT.warn(log, null, "Failed to connect to any address from IP finder " +
+                    LT.warn(log, "Failed to connect to any address from IP finder " +
                         "(make sure IP finder addresses are correct and firewalls are disabled on all host machines): " +
                         toOrderedList(addrs), true);
                 }
@@ -3148,7 +3147,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     }
                 }
 
-                LT.warn(log, null, "Local node has detected failed nodes and started cluster-wide procedure. " +
+                LT.warn(log, "Local node has detected failed nodes and started cluster-wide procedure. " +
                         "To speed up failure detection please see 'Failure Detection' section under javadoc" +
                         " for 'TcpDiscoverySpi'");
             }
@@ -3233,7 +3232,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         "[locNodeAddrs=" + U.addressesAsString(locNode) +
                         ", rmtNodeAddrs=" + U.addressesAsString(node) + ']';
 
-                    LT.warn(log, null, errMsg);
+                    LT.warn(log, errMsg);
 
                     // Always output in debug.
                     if (log.isDebugEnabled())
@@ -3286,7 +3285,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         }
 
                         // Output warning.
-                        LT.warn(log, null, "Ignoring join request from node (duplicate ID) [node=" + node +
+                        LT.warn(log, "Ignoring join request from node (duplicate ID) [node=" + node +
                             ", existingNode=" + existingNode + ']');
 
                         // Ignore join request.
@@ -3341,8 +3340,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                         if (subj == null) {
                             // Node has not pass authentication.
-                            LT.warn(log, null,
-                                "Authentication failed [nodeId=" + node.id() +
+                            LT.warn(log, "Authentication failed [nodeId=" + node.id() +
                                     ", addrs=" + U.addressesAsString(node) + ']',
                                 "Authentication failed [nodeId=" + U.id8(node.id()) + ", addrs=" +
                                     U.addressesAsString(node) + ']');
@@ -3371,8 +3369,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         else {
                             if (!(subj instanceof Serializable)) {
                                 // Node has not pass authentication.
-                                LT.warn(log, null,
-                                    "Authentication subject is not Serializable [nodeId=" + node.id() +
+                                LT.warn(log, "Authentication subject is not Serializable [nodeId=" + node.id() +
                                         ", addrs=" + U.addressesAsString(node) + ']',
                                     "Authentication subject is not Serializable [nodeId=" + U.id8(node.id()) +
                                         ", addrs=" +
@@ -3442,7 +3439,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     return;
                                 }
 
-                                LT.warn(log, null, err.message());
+                                LT.warn(log, err.message());
 
                                 // Always output in debug.
                                 if (log.isDebugEnabled())
@@ -3483,7 +3480,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     ", rmtNodeAddrs=" + U.addressesAsString(node) +
                                     ", locNodeId=" + locNode.id() + ", rmtNodeId=" + msg.creatorNodeId() + ']';
 
-                                LT.warn(log, null, errMsg);
+                                LT.warn(log, errMsg);
 
                                 // Always output in debug.
                                 if (log.isDebugEnabled())
@@ -3771,7 +3768,7 @@ class ServerImpl extends TcpDiscoveryImpl {
          * @param sndMsg Message to send.
          */
         private void nodeCheckError(TcpDiscoveryNode node, String errMsg, String sndMsg) {
-            LT.warn(log, null, errMsg);
+            LT.warn(log, errMsg);
 
             // Always output in debug.
             if (log.isDebugEnabled())
@@ -4056,8 +4053,7 @@ class ServerImpl extends TcpDiscoveryImpl {
 
                             if (!permissionsEqual(coordSubj.subject().permissions(), subj.subject().permissions())) {
                                 // Node has not pass authentication.
-                                LT.warn(log, null,
-                                    "Authentication failed [nodeId=" + node.id() +
+                                LT.warn(log, "Authentication failed [nodeId=" + node.id() +
                                         ", addrs=" + U.addressesAsString(node) + ']',
                                     "Authentication failed [nodeId=" + U.id8(node.id()) + ", addrs=" +
                                         U.addressesAsString(node) + ']');
@@ -4148,7 +4144,6 @@ class ServerImpl extends TcpDiscoveryImpl {
                                         rmCrd.subject().permissions())) {
                                         // Node has not pass authentication.
                                         LT.warn(log,
-                                            null,
                                             "Failed to authenticate local node " +
                                                 "(local authentication result is different from rest of topology) " +
                                                 "[nodeId=" + node.id() + ", addrs=" + U.addressesAsString(node) + ']',
@@ -5593,7 +5588,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                                     "[rmtAddr=" + sock.getRemoteSocketAddress() +
                                     ", locAddr=" + sock.getLocalSocketAddress() + ']');
 
-                            LT.warn(log, null, "Failed to read magic header (too few bytes received) [rmtAddr=" +
+                            LT.warn(log, "Failed to read magic header (too few bytes received) [rmtAddr=" +
                                 sock.getRemoteSocketAddress() + ", locAddr=" + sock.getLocalSocketAddress() + ']');
 
                             return;
@@ -5609,7 +5604,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                                 "[rmtAddr=" + sock.getRemoteSocketAddress() +
                                 ", locAddr=" + sock.getLocalSocketAddress() + ']');
 
-                        LT.warn(log, null, "Unknown connection detected (is some other software connecting to " +
+                        LT.warn(log, "Unknown connection detected (is some other software connecting to " +
                             "this Ignite port?" +
                             (!spi.isSslEnabled() ? " missing SSL configuration on remote node?" : "" ) +
                             ") [rmtAddr=" + sock.getInetAddress() + ']', true);
@@ -5729,7 +5724,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                         U.error(log, "Caught exception on handshake [err=" + e +", sock=" + sock + ']', e);
 
                     if (X.hasCause(e, SSLException.class) && spi.isSslEnabled() && !spi.isNodeStopping0())
-                        LT.warn(log, null, "Failed to initialize connection " +
+                        LT.warn(log, "Failed to initialize connection " +
                             "(missing SSL configuration on remote node?) " +
                             "[rmtAddr=" + sock.getInetAddress() + ']', true);
                     else if ((X.hasCause(e, ObjectStreamException.class) || !sock.isClosed())
@@ -5758,12 +5753,12 @@ class ServerImpl extends TcpDiscoveryImpl {
                     onException("Caught exception on handshake [err=" + e +", sock=" + sock + ']', e);
 
                     if (e.hasCause(SocketTimeoutException.class))
-                        LT.warn(log, null, "Socket operation timed out on handshake " +
+                        LT.warn(log, "Socket operation timed out on handshake " +
                             "(consider increasing 'networkTimeout' configuration property) " +
                             "[netTimeout=" + spi.netTimeout + ']');
 
                     else if (e.hasCause(ClassNotFoundException.class))
-                        LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
+                        LT.warn(log, "Failed to read message due to ClassNotFoundException " +
                             "(make sure same versions of all classes are available on all nodes) " +
                             "[rmtAddr=" + sock.getRemoteSocketAddress() +
                             ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']');
@@ -5995,7 +5990,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                             return;
 
                         if (e.hasCause(ClassNotFoundException.class))
-                            LT.warn(log, null, "Failed to read message due to ClassNotFoundException " +
+                            LT.warn(log, "Failed to read message due to ClassNotFoundException " +
                                 "(make sure same versions of all classes are available on all nodes) " +
                                 "[rmtNodeId=" + nodeId +
                                 ", err=" + X.cause(e, ClassNotFoundException.class).getMessage() + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
index 0816cbc..f199c20 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoveryImpl.java
@@ -329,7 +329,7 @@ abstract class TcpDiscoveryImpl {
      */
     protected boolean checkAckTimeout(long ackTimeout) {
         if (ackTimeout > spi.getMaxAckTimeout()) {
-            LT.warn(log, null, "Acknowledgement timeout is greater than maximum acknowledgement timeout " +
+            LT.warn(log, "Acknowledgement timeout is greater than maximum acknowledgement timeout " +
                 "(consider increasing 'maxAckTimeout' configuration property) " +
                 "[ackTimeout=" + ackTimeout + ", maxAckTimeout=" + spi.getMaxAckTimeout() + ']');
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index a8704e7..45933e1 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -1471,7 +1471,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
         }
         catch (IOException | IgniteCheckedException e) {
             if (X.hasCause(e, SocketTimeoutException.class))
-                LT.warn(log, null, "Timed out waiting for message to be read (most probably, the reason is " +
+                LT.warn(log, "Timed out waiting for message to be read (most probably, the reason is " +
                     "in long GC pauses on remote node) [curTimeout=" + timeout + ']');
 
             throw e;
@@ -1511,7 +1511,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
             return res;
         }
         catch (SocketTimeoutException e) {
-            LT.warn(log, null, "Timed out waiting for message delivery receipt (most probably, the reason is " +
+            LT.warn(log, "Timed out waiting for message delivery receipt (most probably, the reason is " +
                 "in long GC pauses on remote node; consider tuning GC and increasing 'ackTimeout' " +
                 "configuration property). Will retry to send message with increased timeout. " +
                 "Current timeout: " + timeout + '.');
@@ -1575,7 +1575,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
                     res.add(resolved);
             }
             catch (UnknownHostException ignored) {
-                LT.warn(log, null, "Failed to resolve address from IP finder (host is unknown): " + addr);
+                LT.warn(log, "Failed to resolve address from IP finder (host is unknown): " + addr);
 
                 // Add address in any case.
                 res.add(addr);
@@ -2045,7 +2045,7 @@ public class TcpDiscoverySpi extends IgniteSpiAdapter implements DiscoverySpi, T
                 // Close socket - timeout occurred.
                 U.closeQuiet(sock);
 
-                LT.warn(log, null, "Socket write has timed out (consider increasing " +
+                LT.warn(log, "Socket write has timed out (consider increasing " +
                     (failureDetectionTimeoutEnabled() ?
                     "'IgniteConfiguration.failureDetectionTimeout' configuration property) [" +
                     "failureDetectionTimeout=" + failureDetectionTimeout() + ']' :

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
index e96abe9..8fe8a65 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ipfinder/multicast/TcpDiscoveryMulticastIpFinder.java
@@ -598,7 +598,7 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
                                 addrRes = new AddressResponse(data);
                             }
                             catch (IgniteCheckedException e) {
-                                LT.warn(log, e, "Failed to deserialize multicast response.");
+                                LT.error(log, e, "Failed to deserialize multicast response.");
 
                                 continue;
                             }
@@ -876,7 +876,7 @@ public class TcpDiscoveryMulticastIpFinder extends TcpDiscoveryVmIpFinder {
                 }
                 catch (IOException e) {
                     if (!isInterrupted()) {
-                        LT.warn(log, e, "Failed to send/receive address message (will try to reconnect).");
+                        LT.error(log, e, "Failed to send/receive address message (will try to reconnect).");
 
                         synchronized (this) {
                             U.close(sock);

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index b3ce46b..0ae6575 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -1061,7 +1061,7 @@ public final class GridTestUtils {
                     Collection<ClusterNode> nodes = top.nodes(p, AffinityTopologyVersion.NONE);
 
                     if (nodes.size() > backups + 1) {
-                        LT.warn(log, null, "Partition map was not updated yet (will wait) [grid=" + g.name() +
+                        LT.warn(log, "Partition map was not updated yet (will wait) [grid=" + g.name() +
                             ", p=" + p + ", nodes=" + F.nodeIds(nodes) + ']');
 
                         wait = true;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index d5f6b1c..90fabd9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -507,7 +507,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
                                 if (affNodes.size() != owners.size() || !affNodes.containsAll(owners) ||
                                     (waitEvicts && loc != null && loc.state() != GridDhtPartitionState.OWNING)) {
-                                    LT.warn(log(), null, "Waiting for topology map update [" +
+                                    LT.warn(log(), "Waiting for topology map update [" +
                                         "grid=" + g.name() +
                                         ", cache=" + cfg.getName() +
                                         ", cacheId=" + dht.context().cacheId() +
@@ -524,7 +524,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                     match = true;
                             }
                             else {
-                                LT.warn(log(), null, "Waiting for topology map update [" +
+                                LT.warn(log(), "Waiting for topology map update [" +
                                     "grid=" + g.name() +
                                     ", cache=" + cfg.getName() +
                                     ", cacheId=" + dht.context().cacheId() +
@@ -590,7 +590,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
                                     }
 
                                     if (entry.getValue() != GridDhtPartitionState.OWNING) {
-                                        LT.warn(log(), null,
+                                        LT.warn(log(),
                                             "Waiting for correct partition state, should be OWNING [state=" +
                                                 entry.getValue() + "]");
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java
index d9540a8..9eac0cc 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridLogThrottleTest.java
@@ -53,26 +53,20 @@ public class GridLogThrottleTest extends GridCommonAbstractTest {
         // LOGGED.
         LT.error(log, new RuntimeException("Test exception 2."), "Test");
 
-        // OMITTED.
-        LT.warn(log, new RuntimeException("Test exception 1."), "Test");
-
-        // OMITTED.
-        LT.warn(log, new RuntimeException("Test exception 2."), "Test1");
-
-        // OMITTED.
-        LT.warn(log, new RuntimeException("Test exception 2."), "Test3");
-
         // LOGGED.
         LT.error(log, null, "Test - without throwable.");
 
         // OMITTED.
         LT.error(log, null, "Test - without throwable.");
 
+        // OMITTED.
+        LT.warn(log, "Test - without throwable.");
+
         // LOGGED.
-        LT.warn(log, null, "Test - without throwable1.");
+        LT.warn(log, "Test - without throwable1.");
 
         // OMITTED.
-        LT.warn(log, null, "Test - without throwable1.");
+        LT.warn(log, "Test - without throwable1.");
 
         Thread.sleep(LT.throttleTimeout());
 
@@ -90,14 +84,11 @@ public class GridLogThrottleTest extends GridCommonAbstractTest {
         // LOGGED.
         LT.error(log, new RuntimeException("Test exception 2."), "Test");
 
-        // OMITTED.
-        LT.warn(log, new RuntimeException("Test exception 1."), "Test");
-
-        // OMITTED.
-        LT.warn(log, new RuntimeException("Test exception 2."), "Test1");
+        // LOGGED.
+        LT.warn(log, "Test - without throwable.");
 
         // OMITTED.
-        LT.warn(log, new RuntimeException("Test exception 2."), "Test3");
+        LT.warn(log, "Test - without throwable.");
 
         Thread.sleep(LT.throttleTimeout());
 
@@ -121,4 +112,4 @@ public class GridLogThrottleTest extends GridCommonAbstractTest {
         //OMMITED.
         LT.info(log(), "Test info message.");
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
index 6d903d8..bc047e7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
@@ -856,7 +856,7 @@ public class HadoopExternalCommunication {
             catch (IgniteCheckedException e) {
                 if (e.hasCause(IpcOutOfSystemResourcesException.class))
                     // Has cause or is itself the IpcOutOfSystemResourcesException.
-                    LT.warn(log, null, OUT_OF_RESOURCES_TCP_MSG);
+                    LT.warn(log, OUT_OF_RESOURCES_TCP_MSG);
                 else if (log.isDebugEnabled())
                     log.debug("Failed to establish shared memory connection with local hadoop process: " +
                         desc);
@@ -1059,7 +1059,7 @@ public class HadoopExternalCommunication {
                         ", err=" + e + ']');
 
                 if (X.hasCause(e, SocketTimeoutException.class))
-                    LT.warn(log, null, "Connect timed out (consider increasing 'connTimeout' " +
+                    LT.warn(log, "Connect timed out (consider increasing 'connTimeout' " +
                         "configuration property) [addr=" + addr + ", port=" + port + ']');
 
                 if (errs == null)
@@ -1084,7 +1084,7 @@ public class HadoopExternalCommunication {
             assert errs != null;
 
             if (X.hasCause(errs, ConnectException.class))
-                LT.warn(log, null, "Failed to connect to a remote Hadoop process (is process still running?). " +
+                LT.warn(log, "Failed to connect to a remote Hadoop process (is process still running?). " +
                     "Make sure operating system firewall is disabled on local and remote host) " +
                     "[addrs=" + addr + ", port=" + port + ']');
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index c3f701a..6da8758 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -996,7 +996,7 @@ public class IgniteH2Indexing implements GridQueryIndexing {
                 String longMsg = "Query execution is too long [time=" + time + " ms, sql='" + sql + '\'' +
                     ", plan=" + U.nl() + plan.getString(1) + U.nl() + ", parameters=" + params + "]";
 
-                LT.warn(log, null, longMsg, msg);
+                LT.warn(log, longMsg, msg);
             }
 
             return rs;

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
index c24fed4..bddef66 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheOffheapEvictQueryTest.java
@@ -181,7 +181,7 @@ public class IgniteCacheOffheapEvictQueryTest extends GridCommonAbstractTest {
                             }
                         }
 
-                        LT.warn(log, null, e.getMessage());
+                        LT.warn(log, e.getMessage());
 
                         return;
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/bbaa79af/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java
----------------------------------------------------------------------
diff --git a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java
index 48bfd7f..bb7260d 100644
--- a/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java
+++ b/modules/urideploy/src/main/java/org/apache/ignite/spi/deployment/uri/scanners/http/UriDeploymentHttpScanner.java
@@ -343,11 +343,11 @@ public class UriDeploymentHttpScanner implements UriDeploymentScanner {
                     catch (IOException e) {
                         if (!scanCtx.isCancelled()) {
                             if (X.hasCause(e, ConnectException.class)) {
-                                LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server " +
+                                LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server " +
                                     "(connection refused): " + U.hidePassword(url));
                             }
                             else if (X.hasCause(e, UnknownHostException.class)) {
-                                LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server " +
+                                LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server " +
                                     "(host is unknown): " + U.hidePassword(url));
                             }
                             else
@@ -404,11 +404,11 @@ public class UriDeploymentHttpScanner implements UriDeploymentScanner {
             catch (IOException e) {
                 if (!scanCtx.isCancelled()) {
                     if (X.hasCause(e, ConnectException.class)) {
-                        LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server (connection refused): " +
+                        LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server (connection refused): " +
                             U.hidePassword(url.toString()));
                     }
                     else if (X.hasCause(e, UnknownHostException.class)) {
-                        LT.warn(scanCtx.getLogger(), e, "Failed to connect to HTTP server (host is unknown): " +
+                        LT.error(scanCtx.getLogger(), e, "Failed to connect to HTTP server (host is unknown): " +
                             U.hidePassword(url.toString()));
                     }
                     else


[05/25] ignite git commit: ignite-3685 - fixed

Posted by sb...@apache.org.
ignite-3685 - fixed


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

Branch: refs/heads/ignite-4371
Commit: ac8602dbdf2bbf5b16a611eaf6d520a0a7b0010b
Parents: 0d4a1b7
Author: Sergi Vladykin <se...@gmail.com>
Authored: Mon Aug 15 16:46:54 2016 +0300
Committer: Sergi <se...@gmail.com>
Committed: Wed Dec 7 15:08:12 2016 +0300

----------------------------------------------------------------------
 .../internal/processors/query/h2/opt/GridH2Row.java |  2 +-
 .../IgniteCacheAbstractFieldsQuerySelfTest.java     |  2 +-
 .../local/IgniteCacheLocalFieldsQuerySelfTest.java  | 16 ++++++++++++++++
 3 files changed, 18 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/ac8602db/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
index 9486a2e..8e7b161 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/opt/GridH2Row.java
@@ -98,7 +98,7 @@ public abstract class GridH2Row extends Row implements GridSearchRowPointer {
 
     /** {@inheritDoc} */
     @Override public void setKey(long key) {
-        throw new UnsupportedOperationException();
+        // No-op, may be set in H2 INFORMATION_SCHEMA.
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/ac8602db/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
index 926d294..d5f02eb 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheAbstractFieldsQuerySelfTest.java
@@ -71,7 +71,7 @@ public abstract class IgniteCacheAbstractFieldsQuerySelfTest extends GridCommonA
     private static final TcpDiscoveryIpFinder IP_FINDER = new TcpDiscoveryVmIpFinder(true);
 
     /** Cache name. */
-    private static final String CACHE = "cache";
+    protected static final String CACHE = "cache";
 
     /** Empty cache name. */
     private static final String EMPTY_CACHE = "emptyCache";

http://git-wip-us.apache.org/repos/asf/ignite/blob/ac8602db/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java
index be1f196..462118f 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/local/IgniteCacheLocalFieldsQuerySelfTest.java
@@ -18,6 +18,8 @@
 package org.apache.ignite.internal.processors.cache.local;
 
 import org.apache.ignite.cache.CacheMode;
+import org.apache.ignite.cache.query.SqlFieldsQuery;
+import org.apache.ignite.internal.IgniteEx;
 import org.apache.ignite.internal.processors.cache.IgniteCacheAbstractFieldsQuerySelfTest;
 
 import static org.apache.ignite.cache.CacheMode.LOCAL;
@@ -26,6 +28,10 @@ import static org.apache.ignite.cache.CacheMode.LOCAL;
  * Tests for fields queries.
  */
 public class IgniteCacheLocalFieldsQuerySelfTest extends IgniteCacheAbstractFieldsQuerySelfTest {
+//    static {
+//        System.setProperty(IgniteSystemProperties.IGNITE_H2_DEBUG_CONSOLE, "1");
+//    }
+
     /** {@inheritDoc} */
     @Override protected CacheMode cacheMode() {
         return LOCAL;
@@ -35,4 +41,14 @@ public class IgniteCacheLocalFieldsQuerySelfTest extends IgniteCacheAbstractFiel
     @Override protected int gridCount() {
         return 1;
     }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testInformationSchema() throws Exception {
+        IgniteEx ignite = grid(0);
+
+        ignite.cache(CACHE).query(
+            new SqlFieldsQuery("SELECT VALUE FROM INFORMATION_SCHEMA.SETTINGS").setLocal(true)).getAll();
+    }
 }
\ No newline at end of file


[11/25] ignite git commit: IGNITE-4336 Manual rebalance can't be requested twice

Posted by sb...@apache.org.
IGNITE-4336 Manual rebalance can't be requested twice


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

Branch: refs/heads/ignite-4371
Commit: c14561c337cf145aced381de1f1ca9676733b862
Parents: 59e6fec
Author: Anton Vinogradov <av...@apache.org>
Authored: Thu Dec 8 17:58:54 2016 +0300
Committer: Anton Vinogradov <av...@apache.org>
Committed: Thu Dec 8 17:58:54 2016 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      |  6 +-
 .../GridCachePartitionExchangeManager.java      | 11 ++-
 .../processors/cache/GridCachePreloader.java    |  8 ++-
 .../cache/GridCachePreloaderAdapter.java        | 12 ++--
 .../processors/cache/IgniteCacheProxy.java      |  4 +-
 .../dht/preloader/GridDhtPartitionDemander.java | 46 ++++++++++--
 .../GridDhtPartitionsExchangeFuture.java        | 16 ++++-
 .../dht/preloader/GridDhtPreloader.java         | 11 +--
 .../cache/CacheRebalancingSelfTest.java         | 75 ++++++++++++++++++++
 .../testsuites/IgniteCacheTestSuite5.java       |  9 ++-
 10 files changed, 166 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 2e24e67..0f39c69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -84,8 +84,8 @@ import org.apache.ignite.internal.processors.affinity.AffinityTopologyVersion;
 import org.apache.ignite.internal.processors.cache.affinity.GridCacheAffinityImpl;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteExternalizableExpiryPolicy;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtCacheAdapter;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtInvalidPartitionException;
+import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtLocalPartition;
 import org.apache.ignite.internal.processors.cache.dr.GridCacheDrInfo;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteInternalTx;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxAdapter;
@@ -4651,9 +4651,7 @@ public abstract class GridCacheAdapter<K, V> implements IgniteInternalCache<K, V
 
     /** {@inheritDoc} */
     @Override public IgniteInternalFuture<?> rebalance() {
-        ctx.preloader().forcePreload();
-
-        return ctx.preloader().syncFuture();
+        return ctx.preloader().forceRebalance();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index f04a6ce..8ea2169 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -711,9 +711,13 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      *
      * @param exchFut Exchange future.
      */
-    public void forcePreloadExchange(GridDhtPartitionsExchangeFuture exchFut) {
+    public IgniteInternalFuture<Boolean> forceRebalance(GridDhtPartitionsExchangeFuture exchFut) {
+        GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>();
+
         exchWorker.addFuture(
-            new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId()));
+            new GridDhtPartitionsExchangeFuture(cctx, exchFut.discoveryEvent(), exchFut.exchangeId(), fut));
+
+        return fut;
     }
 
     /**
@@ -1771,7 +1775,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                                 Runnable cur = cacheCtx.preloader().addAssignments(assigns,
                                     forcePreload,
                                     cnt,
-                                    r);
+                                    r,
+                                    exchFut.forcedRebalanceFuture());
 
                                 if (cur != null) {
                                     rebList.add(U.maskName(cacheCtx.name()));

http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index 3c4456d..0c28691 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -28,6 +28,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionSupplyMessageV2;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
 
@@ -90,7 +91,8 @@ public interface GridCachePreloader {
     public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
         boolean forcePreload,
         int cnt,
-        Runnable next);
+        Runnable next,
+        @Nullable GridFutureAdapter<Boolean> forcedRebFut);
 
     /**
      * @param p Preload predicate.
@@ -150,9 +152,9 @@ public interface GridCachePreloader {
         AffinityTopologyVersion topVer);
 
     /**
-     * Force preload process.
+     * Force Rebalance process.
      */
-    public void forcePreload();
+    public IgniteInternalFuture<Boolean> forceRebalance();
 
     /**
      * Unwinds undeploys.

http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index 656a960..d7ec288 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -31,6 +31,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPreloaderAssignments;
 import org.apache.ignite.internal.util.future.GridFinishedFuture;
+import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.jetbrains.annotations.Nullable;
 
@@ -88,8 +89,8 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
-    @Override public void forcePreload() {
-        // No-op.
+    @Override public IgniteInternalFuture<Boolean> forceRebalance() {
+        return new GridFinishedFuture<>(true);
     }
 
     /** {@inheritDoc} */
@@ -165,8 +166,11 @@ public class GridCachePreloaderAdapter implements GridCachePreloader {
     }
 
     /** {@inheritDoc} */
-    @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments, boolean forcePreload,
-        int cnt, Runnable next) {
+    @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
+        boolean forcePreload,
+        int cnt,
+        Runnable next,
+        @Nullable GridFutureAdapter<Boolean> forcedRebFut) {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index f87fa1d..f03a3b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -2326,9 +2326,7 @@ public class IgniteCacheProxy<K, V> extends AsyncSupportAdapter<IgniteCache<K, V
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<?> rebalance() {
-        ctx.preloader().forcePreload();
-
-        return new IgniteFutureImpl<>(ctx.preloader().syncFuture());
+        return new IgniteFutureImpl<>(ctx.preloader().forceRebalance());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
index a6808c7..02c31da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
@@ -55,6 +55,7 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartit
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObject;
 import org.apache.ignite.internal.processors.timeout.GridTimeoutObjectAdapter;
 import org.apache.ignite.internal.util.GridLeanSet;
+import org.apache.ignite.internal.util.future.GridFinishedFuture;
 import org.apache.ignite.internal.util.future.GridFutureAdapter;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
@@ -64,6 +65,7 @@ import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
+import org.apache.ignite.lang.IgniteInClosure;
 import org.apache.ignite.lang.IgnitePredicate;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.spi.IgniteSpiException;
@@ -216,9 +218,9 @@ public class GridDhtPartitionDemander {
     }
 
     /**
-     * Force preload.
+     * Force Rebalance.
      */
-    void forcePreload() {
+    IgniteInternalFuture<Boolean> forceRebalance() {
         GridTimeoutObject obj = lastTimeoutObj.getAndSet(null);
 
         if (obj != null)
@@ -230,14 +232,31 @@ public class GridDhtPartitionDemander {
             if (log.isDebugEnabled())
                 log.debug("Forcing rebalance event for future: " + exchFut);
 
+            final GridFutureAdapter<Boolean> fut = new GridFutureAdapter<>();
+
             exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                 @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> t) {
-                    cctx.shared().exchange().forcePreloadExchange(exchFut);
+                    IgniteInternalFuture<Boolean> fut0 = cctx.shared().exchange().forceRebalance(exchFut);
+
+                    fut0.listen(new IgniteInClosure<IgniteInternalFuture<Boolean>>() {
+                        @Override public void apply(IgniteInternalFuture<Boolean> future) {
+                            try {
+                                fut.onDone(future.get());
+                            }
+                            catch (Exception e) {
+                                fut.onDone(e);
+                            }
+                        }
+                    });
                 }
             });
+
+            return fut;
         }
         else if (log.isDebugEnabled())
             log.debug("Ignoring force rebalance request (no topology event happened yet).");
+
+        return new GridFinishedFuture<>(true);
     }
 
     /**
@@ -275,15 +294,19 @@ public class GridDhtPartitionDemander {
      * @param force {@code True} if dummy reassign.
      * @param cnt Counter.
      * @param next Runnable responsible for cache rebalancing start.
+     * @param forcedRebFut External future for forced rebalance.
      * @return Rebalancing runnable.
      */
     Runnable addAssignments(final GridDhtPreloaderAssignments assigns,
         boolean force,
         int cnt,
-        final Runnable next) {
+        final Runnable next,
+        @Nullable final GridFutureAdapter<Boolean> forcedRebFut) {
         if (log.isDebugEnabled())
             log.debug("Adding partition assignments: " + assigns);
 
+        assert force == (forcedRebFut != null);
+
         long delay = cctx.config().getRebalanceDelay();
 
         if (delay == 0 || force) {
@@ -301,6 +324,19 @@ public class GridDhtPartitionDemander {
                 });
             }
 
+             if (forcedRebFut != null) {
+                fut.listen(new CI1<IgniteInternalFuture<Boolean>>() {
+                    @Override public void apply(IgniteInternalFuture<Boolean> future) {
+                        try {
+                            forcedRebFut.onDone(future.get());
+                        }
+                        catch (Exception e) {
+                            forcedRebFut.onDone(e);
+                        }
+                    }
+                });
+            }
+
             rebalanceFut = fut;
 
             fut.sendRebalanceStartedEvent();
@@ -383,7 +419,7 @@ public class GridDhtPartitionDemander {
                 @Override public void onTimeout() {
                     exchFut.listen(new CI1<IgniteInternalFuture<AffinityTopologyVersion>>() {
                         @Override public void apply(IgniteInternalFuture<AffinityTopologyVersion> f) {
-                            cctx.shared().exchange().forcePreloadExchange(exchFut);
+                            cctx.shared().exchange().forceRebalance(exchFut);
                         }
                     });
                 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index e945de9..4f34401 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -54,7 +54,6 @@ import org.apache.ignite.internal.processors.cache.GridCacheContext;
 import org.apache.ignite.internal.processors.cache.GridCacheMvccCandidate;
 import org.apache.ignite.internal.processors.cache.GridCacheSharedContext;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridClientPartitionTopology;
-import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionState;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtPartitionTopology;
 import org.apache.ignite.internal.processors.cache.distributed.dht.GridDhtTopologyFuture;
 import org.apache.ignite.internal.processors.cache.transactions.IgniteTxKey;
@@ -65,7 +64,6 @@ import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.tostring.GridToStringInclude;
 import org.apache.ignite.internal.util.typedef.CI1;
 import org.apache.ignite.internal.util.typedef.F;
-import org.apache.ignite.internal.util.typedef.T2;
 import org.apache.ignite.internal.util.typedef.internal.CU;
 import org.apache.ignite.internal.util.typedef.internal.LT;
 import org.apache.ignite.internal.util.typedef.internal.S;
@@ -194,6 +192,9 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     /** */
     private boolean centralizedAff;
 
+    /** Forced Rebalance future. */
+    private GridFutureAdapter<Boolean> forcedRebFut;
+
     /**
      * Dummy future created to trigger reassignments if partition
      * topology changed while preloading.
@@ -227,15 +228,17 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      * @param cctx Cache context.
      * @param discoEvt Discovery event.
      * @param exchId Exchange id.
+     * @param forcedRebFut Forced Rebalance future.
      */
     public GridDhtPartitionsExchangeFuture(GridCacheSharedContext cctx, DiscoveryEvent discoEvt,
-        GridDhtPartitionExchangeId exchId) {
+        GridDhtPartitionExchangeId exchId, GridFutureAdapter<Boolean> forcedRebFut) {
         dummy = false;
         forcePreload = true;
 
         this.exchId = exchId;
         this.discoEvt = discoEvt;
         this.cctx = cctx;
+        this.forcedRebFut = forcedRebFut;
 
         reassign = true;
 
@@ -398,6 +401,13 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
     }
 
     /**
+     * @return Forced Rebalance future.
+     */
+    @Nullable public GridFutureAdapter<Boolean> forcedRebalanceFuture() {
+        return forcedRebFut;
+    }
+
+    /**
      * @return {@code true} if entered to busy state.
      */
     private boolean enterBusy() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 692e7c0..41bc2fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -413,8 +413,11 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
 
     /** {@inheritDoc} */
     @Override public Runnable addAssignments(GridDhtPreloaderAssignments assignments,
-        boolean forcePreload, int cnt, Runnable next) {
-        return demander.addAssignments(assignments, forcePreload, cnt, next);
+        boolean forceRebalance,
+        int cnt,
+        Runnable next,
+        @Nullable GridFutureAdapter<Boolean> forcedRebFut) {
+        return demander.addAssignments(assignments, forceRebalance, cnt, next, forcedRebFut);
     }
 
     /**
@@ -728,8 +731,8 @@ public class GridDhtPreloader extends GridCachePreloaderAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void forcePreload() {
-        demander.forcePreload();
+    @Override public IgniteInternalFuture<Boolean> forceRebalance() {
+        return demander.forceRebalance();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
new file mode 100644
index 0000000..8d1f67a
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/CacheRebalancingSelfTest.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ *
+ */
+
+package org.apache.ignite.internal.processors.cache;
+
+import org.apache.ignite.IgniteCache;
+import org.apache.ignite.configuration.CacheConfiguration;
+import org.apache.ignite.configuration.IgniteConfiguration;
+import org.apache.ignite.internal.IgniteEx;
+import org.apache.ignite.internal.IgniteInternalFuture;
+import org.apache.ignite.internal.util.future.IgniteFutureImpl;
+import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
+
+/**
+ * Test for rebalancing.
+ */
+public class CacheRebalancingSelfTest extends GridCommonAbstractTest {
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        cfg.setCacheConfiguration(new CacheConfiguration());
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If failed.
+     */
+    public void testRebalanceFuture() throws Exception {
+        IgniteEx ignite0 = startGrid(0);
+        startGrid(1);
+
+        IgniteCache<Object, Object> cache = ignite0.cache(null);
+
+        IgniteFuture fut1 = cache.rebalance();
+
+        fut1.get();
+
+        startGrid(2);
+
+        IgniteFuture fut2 = cache.rebalance();
+
+        assert internalFuture(fut2) != internalFuture(fut1);
+
+        fut2.get();
+    }
+
+    /**
+     * @param future Future.
+     * @return Internal future.
+     */
+    private static IgniteInternalFuture internalFuture(IgniteFuture future) {
+        assert future instanceof IgniteFutureImpl;
+
+        return ((IgniteFutureImpl)future).internalFuture();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/c14561c3/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
index 7f0e23c..144aac6 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite5.java
@@ -19,18 +19,19 @@ package org.apache.ignite.testsuites;
 
 import junit.framework.TestSuite;
 import org.apache.ignite.internal.processors.cache.CacheNearReaderUpdateTest;
+import org.apache.ignite.internal.processors.cache.CacheRebalancingSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheSerializableTransactionsTest;
-import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest;
 import org.apache.ignite.internal.processors.cache.EntryVersionConsistencyReadThroughTest;
+import org.apache.ignite.internal.processors.cache.GridCacheOffHeapCleanupTest;
+import org.apache.ignite.internal.processors.cache.GridCacheSwapSpaceSpiConsistencySelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCachePutStackOverflowSelfTest;
 import org.apache.ignite.internal.processors.cache.IgniteCacheReadThroughEvictionsVariationsSuite;
 import org.apache.ignite.internal.processors.cache.IgniteCacheStoreCollectionTest;
-import org.apache.ignite.internal.processors.cache.GridCacheOffHeapCleanupTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentFairAffinityTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentNodeJoinValidationTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheLateAffinityAssignmentTest;
-import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheTxIteratorSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.replicated.IgniteCacheSyncRebalanceModeSelfTest;
 import org.apache.ignite.internal.processors.cache.store.IgniteCacheWriteBehindNoUpdateSelfTest;
 
 /**
@@ -62,6 +63,8 @@ public class IgniteCacheTestSuite5 extends TestSuite {
 
         suite.addTestSuite(GridCacheOffHeapCleanupTest.class);
 
+        suite.addTestSuite(CacheRebalancingSelfTest.class);
+
         return suite;
     }
 }


[25/25] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-4371

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

# Conflicts:
#	modules/core/src/main/java/org/apache/ignite/internal/managers/communication/GridIoMessageFactory.java
#	modules/core/src/main/java/org/apache/ignite/internal/managers/communication/IgniteIoTestMessage.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemander.java
#	modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
#	modules/core/src/main/java/org/apache/ignite/spi/communication/tcp/TcpCommunicationSpi.java


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

Branch: refs/heads/ignite-4371
Commit: 37a76190d1447d7fbfe15659b8c52d983daa3775
Parents: 697e2fb 27a81c8
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 9 14:27:09 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 9 14:27:09 2016 +0300

----------------------------------------------------------------------
 .../jmh/misc/JmhIncrementBenchmark.java         |  95 +++++
 .../rendezvous/RendezvousAffinityFunction.java  |  80 ++--
 .../communication/GridIoMessageFactory.java     |  12 +-
 .../communication/IgniteIoTestMessage.java      |   2 +-
 .../discovery/GridDiscoveryManager.java         | 118 +-----
 .../processors/cache/GridCacheAdapter.java      |  13 +-
 .../processors/cache/GridCacheEntryEx.java      |   4 +-
 .../processors/cache/GridCacheMapEntry.java     |  17 +-
 .../GridCachePartitionExchangeManager.java      |  17 +-
 .../processors/cache/GridCachePreloader.java    |   8 +-
 .../cache/GridCachePreloaderAdapter.java        |  12 +-
 .../processors/cache/GridCacheUtils.java        |  17 -
 .../processors/cache/IgniteCacheProxy.java      |   4 +-
 .../binary/CacheObjectBinaryProcessorImpl.java  |   3 +-
 .../GridDistributedTxFinishResponse.java        |   6 +
 .../dht/GridClientPartitionTopology.java        | 120 +++---
 .../distributed/dht/GridDhtLockFuture.java      |  23 +-
 .../dht/GridDhtPartitionTopology.java           |  28 +-
 .../dht/GridDhtPartitionTopologyImpl.java       | 284 ++++++++------
 .../dht/atomic/GridDhtAtomicCache.java          |   2 +-
 .../dht/preloader/GridDhtPartitionDemander.java |  46 ++-
 .../dht/preloader/GridDhtPartitionFullMap.java  |  18 +-
 .../GridDhtPartitionsExchangeFuture.java        |  72 +++-
 .../dht/preloader/GridDhtPreloader.java         |  11 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../transactions/IgniteTxLocalAdapter.java      |   2 +-
 .../processors/hadoop/HadoopJobProperty.java    |   7 +
 .../hadoop/HadoopMapperAwareTaskOutput.java     |  32 ++
 .../processors/hadoop/HadoopTaskInfo.java       |  43 ++
 .../shuffle/HadoopDirectShuffleMessage.java     | 243 ++++++++++++
 .../platform/PlatformAbstractTarget.java        | 268 ++-----------
 .../platform/PlatformAsyncTarget.java           |  44 +++
 .../platform/PlatformNoopProcessor.java         |  41 +-
 .../processors/platform/PlatformProcessor.java  |  42 +-
 .../platform/PlatformProcessorImpl.java         |  87 +++--
 .../processors/platform/PlatformTarget.java     | 103 +++--
 .../platform/PlatformTargetProxy.java           | 126 ++++++
 .../platform/PlatformTargetProxyImpl.java       | 222 +++++++++++
 .../binary/PlatformBinaryProcessor.java         |   6 +-
 .../platform/cache/PlatformCache.java           |  15 +-
 .../platform/cache/PlatformCacheIterator.java   |   2 +-
 .../cache/affinity/PlatformAffinity.java        |   4 +-
 .../affinity/PlatformAffinityFunction.java      |   7 +-
 .../PlatformAffinityFunctionTarget.java         |   4 +-
 .../query/PlatformAbstractQueryCursor.java      |   4 +-
 .../query/PlatformContinuousQueryProxy.java     |   3 +-
 .../callback/PlatformCallbackGateway.java       |   6 +-
 .../callback/PlatformCallbackUtils.java         |   6 +-
 .../platform/cluster/PlatformClusterGroup.java  |  18 +-
 .../platform/compute/PlatformCompute.java       |  15 +-
 .../datastreamer/PlatformDataStreamer.java      |   4 +-
 .../PlatformStreamReceiverImpl.java             |   8 +-
 .../datastructures/PlatformAtomicLong.java      |   4 +-
 .../datastructures/PlatformAtomicReference.java |   8 +-
 .../datastructures/PlatformAtomicSequence.java  |   2 +-
 .../platform/events/PlatformEvents.java         |  15 +-
 .../platform/messaging/PlatformMessaging.java   |   9 +-
 .../platform/services/PlatformServices.java     |  27 +-
 .../transactions/PlatformTransactions.java      |   8 +-
 .../platform/utils/PlatformFutureUtils.java     |  14 +-
 .../utils/PlatformListenableTarget.java         |  62 +++
 .../service/GridServiceProcessor.java           |   4 +-
 .../internal/visor/query/VisorQueryJob.java     |   2 +-
 .../communication/tcp/TcpCommunicationSpi.java  |   4 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  41 +-
 .../tcp/internal/TcpDiscoveryStatistics.java    |   4 +
 .../AbstractAffinityFunctionSelfTest.java       |   2 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   2 +-
 .../cache/CacheRebalancingSelfTest.java         |  75 ++++
 .../GridCachePartitionedAffinitySpreadTest.java |   7 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   3 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  | 232 -----------
 ...eCacheExpiryPolicyWithStoreAbstractTest.java |  40 ++
 .../testsuites/IgniteCacheTestSuite5.java       |   9 +-
 .../processors/hadoop/HadoopMapperUtils.java    |  56 +++
 .../hadoop/impl/v2/HadoopV2Context.java         |  11 +
 .../hadoop/impl/v2/HadoopV2MapTask.java         |  10 +
 .../hadoop/jobtracker/HadoopJobTracker.java     |   4 +
 .../hadoop/shuffle/HadoopShuffle.java           |  23 +-
 .../hadoop/shuffle/HadoopShuffleJob.java        | 389 ++++++++++++++-----
 .../shuffle/HadoopShuffleRemoteState.java       |   5 +-
 .../shuffle/direct/HadoopDirectDataInput.java   | 166 ++++++++
 .../shuffle/direct/HadoopDirectDataOutput.java  | 221 +++++++++++
 .../direct/HadoopDirectDataOutputContext.java   | 100 +++++
 .../direct/HadoopDirectDataOutputState.java     |  54 +++
 .../child/HadoopChildProcessRunner.java         |   2 +-
 .../impl/HadoopMapReduceEmbeddedSelfTest.java   |  22 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |  14 +-
 .../cpp/jni/include/ignite/jni/exports.h        |   3 -
 .../platforms/cpp/jni/include/ignite/jni/java.h |   7 -
 modules/platforms/cpp/jni/project/vs/module.def |   2 -
 modules/platforms/cpp/jni/src/exports.cpp       |   8 -
 modules/platforms/cpp/jni/src/java.cpp          |  76 ++--
 .../Services/ServicesTest.cs                    |   9 +-
 .../Apache.Ignite.Core.csproj                   |   1 +
 .../Apache.Ignite.Core/Impl/Common/Future.cs    |  13 +-
 .../Impl/Common/Listenable.cs                   |  49 +++
 .../Impl/Compute/ComputeImpl.cs                 |   4 +-
 .../Apache.Ignite.Core/Impl/PlatformTarget.cs   |   2 +-
 .../Impl/Unmanaged/IgniteJniNativeMethods.cs    |   8 -
 .../Impl/Unmanaged/UnmanagedUtils.cs            |   5 -
 101 files changed, 2904 insertions(+), 1310 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/37a76190/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/37a76190/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/37a76190/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/37a76190/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------


[02/25] ignite git commit: ignite-4296 Optimize GridDhtPartitionsSingleMessage processing - optimized processing of GridDhtPartitionsSingleMessage - minor optimizations for RendezvousAffinityFunction - fixed heartbeats sending in tcp discovery

Posted by sb...@apache.org.
ignite-4296 Optimize GridDhtPartitionsSingleMessage processing
- optimized processing of GridDhtPartitionsSingleMessage
- minor optimizations for RendezvousAffinityFunction
 - fixed heartbeats sending in tcp discovery


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

Branch: refs/heads/ignite-4371
Commit: acf20b32d8fb68e42b904b091fb2b943f4558cef
Parents: b4aedfd
Author: sboikov <sb...@gridgain.com>
Authored: Mon Dec 5 14:01:28 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Mon Dec 5 14:01:28 2016 +0300

----------------------------------------------------------------------
 .../rendezvous/RendezvousAffinityFunction.java  |  80 ++++--
 .../discovery/GridDiscoveryManager.java         | 118 +-------
 .../GridCachePartitionExchangeManager.java      |   6 +-
 .../processors/cache/GridCacheUtils.java        |  17 --
 .../binary/CacheObjectBinaryProcessorImpl.java  |   3 +-
 .../dht/GridClientPartitionTopology.java        | 120 ++++----
 .../distributed/dht/GridDhtLocalPartition.java  |   1 -
 .../dht/GridDhtPartitionTopology.java           |  28 +-
 .../dht/GridDhtPartitionTopologyImpl.java       | 284 +++++++++++--------
 .../dht/preloader/GridDhtPartitionFullMap.java  |  18 +-
 .../GridDhtPartitionsExchangeFuture.java        |  56 +++-
 .../cache/transactions/IgniteTxHandler.java     |   4 +-
 .../service/GridServiceProcessor.java           |   4 +-
 .../ignite/spi/discovery/tcp/ServerImpl.java    |  40 ++-
 .../tcp/internal/TcpDiscoveryStatistics.java    |   4 +
 .../AbstractAffinityFunctionSelfTest.java       |   2 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |   2 +-
 .../GridCachePartitionedAffinitySpreadTest.java |   7 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  | 232 ---------------
 .../h2/twostep/GridReduceQueryExecutor.java     |  14 +-
 20 files changed, 437 insertions(+), 603 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
index ec12973..75e7c92 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/affinity/rendezvous/RendezvousAffinityFunction.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.cache.affinity.rendezvous;
 
-import java.io.ByteArrayOutputStream;
 import java.io.Externalizable;
 import java.io.IOException;
 import java.io.ObjectInput;
@@ -354,46 +353,69 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
 
     /**
      * Returns collection of nodes (primary first) for specified partition.
+     *
+     * @param d Message digest.
+     * @param part Partition.
+     * @param nodes Nodes.
+     * @param nodesHash Serialized nodes hashes.
+     * @param backups Number of backups.
+     * @param neighborhoodCache Neighborhood.
+     * @return Assignment.
      */
-    public List<ClusterNode> assignPartition(int part, List<ClusterNode> nodes, int backups,
+    public List<ClusterNode> assignPartition(MessageDigest d,
+        int part,
+        List<ClusterNode> nodes,
+        Map<ClusterNode, byte[]> nodesHash,
+        int backups,
         @Nullable Map<UUID, Collection<ClusterNode>> neighborhoodCache) {
         if (nodes.size() <= 1)
             return nodes;
 
-        List<IgniteBiTuple<Long, ClusterNode>> lst = new ArrayList<>();
+        if (d == null)
+            d = digest.get();
 
-        MessageDigest d = digest.get();
+        List<IgniteBiTuple<Long, ClusterNode>> lst = new ArrayList<>(nodes.size());
 
-        for (ClusterNode node : nodes) {
-            Object nodeHash = resolveNodeHash(node);
+        try {
+            for (int i = 0; i < nodes.size(); i++) {
+                ClusterNode node = nodes.get(i);
 
-            try {
-                ByteArrayOutputStream out = new ByteArrayOutputStream();
+                byte[] nodeHashBytes = nodesHash.get(node);
 
-                byte[] nodeHashBytes = U.marshal(ignite.configuration().getMarshaller(), nodeHash);
+                if (nodeHashBytes == null) {
+                    Object nodeHash = resolveNodeHash(node);
 
-                out.write(U.intToBytes(part), 0, 4); // Avoid IOException.
-                out.write(nodeHashBytes, 0, nodeHashBytes.length); // Avoid IOException.
+                    byte[] nodeHashBytes0 = U.marshal(ignite.configuration().getMarshaller(), nodeHash);
+
+                    // Add 4 bytes for partition bytes.
+                    nodeHashBytes = new byte[nodeHashBytes0.length + 4];
+
+                    System.arraycopy(nodeHashBytes0, 0, nodeHashBytes, 4, nodeHashBytes0.length);
+
+                    nodesHash.put(node, nodeHashBytes);
+                }
+
+                U.intToBytes(part, nodeHashBytes, 0);
 
                 d.reset();
 
-                byte[] bytes = d.digest(out.toByteArray());
+                byte[] bytes = d.digest(nodeHashBytes);
 
                 long hash =
-                      (bytes[0] & 0xFFL)
-                    | ((bytes[1] & 0xFFL) << 8)
-                    | ((bytes[2] & 0xFFL) << 16)
-                    | ((bytes[3] & 0xFFL) << 24)
-                    | ((bytes[4] & 0xFFL) << 32)
-                    | ((bytes[5] & 0xFFL) << 40)
-                    | ((bytes[6] & 0xFFL) << 48)
-                    | ((bytes[7] & 0xFFL) << 56);
+                    (bytes[0] & 0xFFL)
+                        | ((bytes[1] & 0xFFL) << 8)
+                        | ((bytes[2] & 0xFFL) << 16)
+                        | ((bytes[3] & 0xFFL) << 24)
+                        | ((bytes[4] & 0xFFL) << 32)
+                        | ((bytes[5] & 0xFFL) << 40)
+                        | ((bytes[6] & 0xFFL) << 48)
+                        | ((bytes[7] & 0xFFL) << 56);
 
                 lst.add(F.t(hash, node));
             }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
 
         Collections.sort(lst, COMPARATOR);
@@ -474,8 +496,18 @@ public class RendezvousAffinityFunction implements AffinityFunction, Externaliza
         Map<UUID, Collection<ClusterNode>> neighborhoodCache = exclNeighbors ?
             GridCacheUtils.neighbors(affCtx.currentTopologySnapshot()) : null;
 
+        MessageDigest d = digest.get();
+
+        List<ClusterNode> nodes = affCtx.currentTopologySnapshot();
+
+        Map<ClusterNode, byte[]> nodesHash = U.newHashMap(nodes.size());
+
         for (int i = 0; i < parts; i++) {
-            List<ClusterNode> partAssignment = assignPartition(i, affCtx.currentTopologySnapshot(), affCtx.backups(),
+            List<ClusterNode> partAssignment = assignPartition(d,
+                i,
+                nodes,
+                nodesHash,
+                affCtx.backups(),
                 neighborhoodCache);
 
             assignments.add(partAssignment);

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index d24f900..ddd4ee3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -42,7 +42,7 @@ import java.util.UUID;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListSet;
+import java.util.concurrent.ConcurrentSkipListMap;
 import java.util.concurrent.CopyOnWriteArrayList;
 import java.util.concurrent.CountDownLatch;
 import java.util.concurrent.LinkedBlockingQueue;
@@ -1623,17 +1623,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
     /**
      * Gets cache remote nodes for cache with given name.
      *
-     * @param cacheName Cache name.
-     * @param topVer Topology version.
-     * @return Collection of cache nodes.
-     */
-    public Collection<ClusterNode> remoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(cacheName, topVer).remoteCacheNodes(cacheName, topVer.topologyVersion());
-    }
-
-    /**
-     * Gets cache remote nodes for cache with given name.
-     *
      * @param topVer Topology version.
      * @return Collection of cache nodes.
      */
@@ -1648,7 +1637,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param topVer Topology version.
      * @return Collection of cache nodes.
      */
-    public Collection<ClusterNode> aliveCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
+    Collection<ClusterNode> aliveCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
         return resolveDiscoCache(cacheName, topVer).aliveCacheNodes(cacheName, topVer.topologyVersion());
     }
 
@@ -1659,38 +1648,20 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param topVer Topology version.
      * @return Collection of cache nodes.
      */
-    public Collection<ClusterNode> aliveRemoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
+    Collection<ClusterNode> aliveRemoteCacheNodes(@Nullable String cacheName, AffinityTopologyVersion topVer) {
         return resolveDiscoCache(cacheName, topVer).aliveRemoteCacheNodes(cacheName, topVer.topologyVersion());
     }
 
     /**
-     * Gets alive remote server nodes with at least one cache configured.
-     *
      * @param topVer Topology version (maximum allowed node order).
-     * @return Collection of alive cache nodes.
+     * @return Oldest alive server nodes with at least one cache configured.
      */
-    public Collection<ClusterNode> aliveRemoteServerNodesWithCaches(AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(null, topVer).aliveRemoteServerNodesWithCaches(topVer.topologyVersion());
-    }
+    @Nullable public ClusterNode oldestAliveCacheServerNode(AffinityTopologyVersion topVer) {
+        DiscoCache cache = resolveDiscoCache(null, topVer);
 
-    /**
-     * Gets alive server nodes with at least one cache configured.
-     *
-     * @param topVer Topology version (maximum allowed node order).
-     * @return Collection of alive cache nodes.
-     */
-    public Collection<ClusterNode> aliveServerNodesWithCaches(AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(null, topVer).aliveServerNodesWithCaches(topVer.topologyVersion());
-    }
+        Map.Entry<ClusterNode, Boolean> e = cache.aliveSrvNodesWithCaches.firstEntry();
 
-    /**
-     * Gets alive nodes with at least one cache configured.
-     *
-     * @param topVer Topology version (maximum allowed node order).
-     * @return Collection of alive cache nodes.
-     */
-    public Collection<ClusterNode> aliveNodesWithCaches(AffinityTopologyVersion topVer) {
-        return resolveDiscoCache(null, topVer).aliveNodesWithCaches(topVer.topologyVersion());
+        return e != null ? e.getKey() : null;
     }
 
     /**
@@ -2580,19 +2551,9 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         private final ConcurrentMap<String, Collection<ClusterNode>> aliveRmtCacheNodes;
 
         /**
-         * Cached alive remote nodes with caches.
-         */
-        private final Collection<ClusterNode> aliveNodesWithCaches;
-
-        /**
          * Cached alive server remote nodes with caches.
          */
-        private final Collection<ClusterNode> aliveSrvNodesWithCaches;
-
-        /**
-         * Cached alive remote server nodes with caches.
-         */
-        private final Collection<ClusterNode> aliveRmtSrvNodesWithCaches;
+        private final ConcurrentSkipListMap<ClusterNode, Boolean> aliveSrvNodesWithCaches;
 
         /**
          * @param loc Local node.
@@ -2625,9 +2586,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             aliveCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f);
             aliveRmtCacheNodes = new ConcurrentHashMap8<>(allNodes.size(), 1.0f);
-            aliveNodesWithCaches = new ConcurrentSkipListSet<>();
-            aliveSrvNodesWithCaches = new ConcurrentSkipListSet<>();
-            aliveRmtSrvNodesWithCaches = new ConcurrentSkipListSet<>();
+            aliveSrvNodesWithCaches = new ConcurrentSkipListMap<>(GridNodeOrderComparator.INSTANCE);
             nodesByVer = new TreeMap<>();
 
             long maxOrder0 = 0;
@@ -2681,18 +2640,8 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
                     }
                 }
 
-                if (hasCaches) {
-                    if (alive(node.id())) {
-                        aliveNodesWithCaches.add(node);
-
-                        if (!CU.clientNode(node)) {
-                            aliveSrvNodesWithCaches.add(node);
-
-                            if (!loc.id().equals(node.id()))
-                                aliveRmtSrvNodesWithCaches.add(node);
-                        }
-                    }
-                }
+                if (hasCaches && alive(node.id()) && !CU.clientNode(node))
+                    aliveSrvNodesWithCaches.put(node, Boolean.TRUE);
 
                 IgniteProductVersion nodeVer = U.productVersion(node);
 
@@ -2821,17 +2770,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
 
         /**
-         * Gets all remote nodes that have cache with given name.
-         *
-         * @param cacheName Cache name.
-         * @param topVer Topology version.
-         * @return Collection of nodes.
-         */
-        Collection<ClusterNode> remoteCacheNodes(@Nullable String cacheName, final long topVer) {
-            return filter(topVer, rmtCacheNodes.get(cacheName));
-        }
-
-        /**
          * Gets all remote nodes that have at least one cache configured.
          *
          * @param topVer Topology version.
@@ -2876,36 +2814,6 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
         }
 
         /**
-         * Gets all alive remote server nodes with at least one cache configured.
-         *
-         * @param topVer Topology version.
-         * @return Collection of nodes.
-         */
-        Collection<ClusterNode> aliveRemoteServerNodesWithCaches(final long topVer) {
-            return filter(topVer, aliveRmtSrvNodesWithCaches);
-        }
-
-        /**
-         * Gets all alive server nodes with at least one cache configured.
-         *
-         * @param topVer Topology version.
-         * @return Collection of nodes.
-         */
-        Collection<ClusterNode> aliveServerNodesWithCaches(final long topVer) {
-            return filter(topVer, aliveSrvNodesWithCaches);
-        }
-
-        /**
-         * Gets all alive remote nodes with at least one cache configured.
-         *
-         * @param topVer Topology version.
-         * @return Collection of nodes.
-         */
-        Collection<ClusterNode> aliveNodesWithCaches(final long topVer) {
-            return filter(topVer, aliveNodesWithCaches);
-        }
-
-        /**
          * Checks if cache with given name has at least one node with near cache enabled.
          *
          * @param cacheName Cache name.
@@ -2928,9 +2836,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
 
             filterNodeMap(aliveRmtCacheNodes, leftNode);
 
-            aliveNodesWithCaches.remove(leftNode);
             aliveSrvNodesWithCaches.remove(leftNode);
-            aliveRmtSrvNodesWithCaches.remove(leftNode);
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index 503b334..7a24aa1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -740,7 +740,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * Partition refresh callback.
      */
     private void refreshPartitions() {
-        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, AffinityTopologyVersion.NONE);
+        ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE);
 
         if (oldest == null) {
             if (log.isDebugEnabled())
@@ -1224,7 +1224,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         top = cacheCtx.topology();
 
                     if (top != null)
-                        updated |= top.update(null, entry.getValue(), null) != null;
+                        updated |= top.update(null, entry.getValue(), null);
                 }
 
                 if (!cctx.kernalContext().clientNode() && updated)
@@ -1273,7 +1273,7 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                         top = cacheCtx.topology();
 
                     if (top != null) {
-                        updated |= top.update(null, entry.getValue(), null) != null;
+                        updated |= top.update(null, entry.getValue(), null, true);
 
                         cctx.affinity().checkRebalanceState(top, cacheId);
                     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 90e428c..d32f4c1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -490,23 +490,6 @@ public class GridCacheUtils {
     }
 
     /**
-     * Gets oldest alive server node with at least one cache configured for specified topology version.
-     *
-     * @param ctx Context.
-     * @param topVer Maximum allowed topology version.
-     * @return Oldest alive cache server node.
-     */
-    @Nullable public static ClusterNode oldestAliveCacheServerNode(GridCacheSharedContext ctx,
-        AffinityTopologyVersion topVer) {
-        Collection<ClusterNode> nodes = ctx.discovery().aliveServerNodesWithCaches(topVer);
-
-        if (nodes.isEmpty())
-            return null;
-
-        return oldest(nodes);
-    }
-
-    /**
      * @param nodes Nodes.
      * @return Oldest node for the given topology version.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
index 568a4da..1d60c42 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/binary/CacheObjectBinaryProcessorImpl.java
@@ -304,8 +304,7 @@ public class CacheObjectBinaryProcessorImpl extends IgniteCacheObjectProcessorIm
             assert !metaDataCache.context().affinityNode();
 
             while (true) {
-                ClusterNode oldestSrvNode =
-                    CU.oldestAliveCacheServerNode(ctx.cache().context(), AffinityTopologyVersion.NONE);
+                ClusterNode oldestSrvNode = ctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE);
 
                 if (oldestSrvNode == null)
                     break;

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
index 5efb317..816132d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridClientPartitionTopology.java
@@ -271,7 +271,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
             removeNode(exchId.nodeId());
 
         // In case if node joins, get topology at the time of joining node.
-        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
+        ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer);
 
         assert oldest != null;
 
@@ -536,7 +536,8 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
 
         try {
             assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part +
-                ", locNodeId=" + cctx.localNodeId() + ", gridName=" + cctx.gridName() + ']';
+                ", locNodeId=" + cctx.localNodeId() +
+                ", gridName=" + cctx.gridName() + ']';
 
             GridDhtPartitionFullMap m = node2part;
 
@@ -549,7 +550,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
-    @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+    @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
         GridDhtPartitionFullMap partMap,
         Map<Integer, Long> cntrMap) {
         if (log.isDebugEnabled())
@@ -563,7 +564,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
                     log.debug("Stale exchange id for full partition map update (will ignore) [lastExchId=" +
                         lastExchangeId + ", exchId=" + exchId + ']');
 
-                return null;
+                return false;
             }
 
             if (node2part != null && node2part.compareTo(partMap) >= 0) {
@@ -571,7 +572,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
                     log.debug("Stale partition map for full partition map update (will ignore) [lastExchId=" +
                         lastExchangeId + ", exchId=" + exchId + ", curMap=" + node2part + ", newMap=" + partMap + ']');
 
-                return null;
+                return false;
             }
 
             updateSeq.incrementAndGet();
@@ -634,7 +635,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
             if (log.isDebugEnabled())
                 log.debug("Partition map after full update: " + fullMapString());
 
-            return null;
+            return false;
         }
         finally {
             lock.writeLock().unlock();
@@ -642,10 +643,10 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
-    @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+    @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
         GridDhtPartitionMap2 parts,
-        Map<Integer, Long> cntrMap) {
+        Map<Integer, Long> cntrMap,
+        boolean checkEvictions) {
         if (log.isDebugEnabled())
             log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']');
 
@@ -654,29 +655,27 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
                 log.debug("Received partition update for non-existing node (will ignore) [exchId=" + exchId +
                     ", parts=" + parts + ']');
 
-            return null;
+            return false;
         }
 
         lock.writeLock().lock();
 
         try {
             if (stopping)
-                return null;
+                return false;
 
             if (lastExchangeId != null && exchId != null && lastExchangeId.compareTo(exchId) > 0) {
                 if (log.isDebugEnabled())
                     log.debug("Stale exchange id for single partition map update (will ignore) [lastExchId=" +
                         lastExchangeId + ", exchId=" + exchId + ']');
 
-                return null;
+                return false;
             }
 
             if (exchId != null)
                 lastExchangeId = exchId;
 
             if (node2part == null) {
-                U.dumpStack(log, "Created invalid: " + node2part);
-
                 // Create invalid partition map.
                 node2part = new GridDhtPartitionFullMap();
             }
@@ -688,43 +687,45 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
                     log.debug("Stale update sequence for single partition map update (will ignore) [exchId=" + exchId +
                         ", curSeq=" + cur.updateSequence() + ", newSeq=" + parts.updateSequence() + ']');
 
-                return null;
+                return false;
             }
 
             long updateSeq = this.updateSeq.incrementAndGet();
 
-            node2part = new GridDhtPartitionFullMap(node2part, updateSeq);
-
-            boolean changed = false;
+            node2part.updateSequence(updateSeq);
 
-            if (cur == null || !cur.equals(parts))
-                changed = true;
+            boolean changed = cur == null || !cur.equals(parts);
 
-            node2part.put(parts.nodeId(), parts);
+            if (changed) {
+                node2part.put(parts.nodeId(), parts);
 
-            part2node = new HashMap<>(part2node);
+                // Add new mappings.
+                for (Integer p : parts.keySet()) {
+                    Set<UUID> ids = part2node.get(p);
 
-            // Add new mappings.
-            for (Integer p : parts.keySet()) {
-                Set<UUID> ids = part2node.get(p);
+                    if (ids == null)
+                        // Initialize HashSet to size 3 in anticipation that there won't be
+                        // more than 3 nodes per partition.
+                        part2node.put(p, ids = U.newHashSet(3));
 
-                if (ids == null)
-                    // Initialize HashSet to size 3 in anticipation that there won't be
-                    // more than 3 nodes per partition.
-                    part2node.put(p, ids = U.newHashSet(3));
+                    ids.add(parts.nodeId());
+                }
 
-                changed |= ids.add(parts.nodeId());
-            }
+                // Remove obsolete mappings.
+                if (cur != null) {
+                    for (Integer p : cur.keySet()) {
+                        if (parts.containsKey(p))
+                            continue;
 
-            // Remove obsolete mappings.
-            if (cur != null) {
-                for (Integer p : F.view(cur.keySet(), F0.notIn(parts.keySet()))) {
-                    Set<UUID> ids = part2node.get(p);
+                        Set<UUID> ids = part2node.get(p);
 
-                    if (ids != null)
-                        changed |= ids.remove(parts.nodeId());
+                        if (ids != null)
+                            ids.remove(parts.nodeId());
+                    }
                 }
             }
+            else
+                cur.updateSequence(parts.updateSequence(), parts.topologyVersion());
 
             if (cntrMap != null) {
                 for (Map.Entry<Integer, Long> e : cntrMap.entrySet()) {
@@ -740,13 +741,18 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
             if (log.isDebugEnabled())
                 log.debug("Partition map after single update: " + fullMapString());
 
-            return changed ? localPartitionMap() : null;
+            return changed;
         }
         finally {
             lock.writeLock().unlock();
         }
     }
 
+    /** {@inheritDoc} */
+    @Override public void checkEvictions() {
+        // No-op.
+    }
+
     /**
      * Updates value for single partition.
      *
@@ -755,13 +761,12 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
      * @param state State.
      * @param updateSeq Update sequence.
      */
-    @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
     private void updateLocal(int p, UUID nodeId, GridDhtPartitionState state, long updateSeq) {
         assert lock.isWriteLockedByCurrentThread();
         assert nodeId.equals(cctx.localNodeId());
 
         // In case if node joins, get topology at the time of joining node.
-        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
+        ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer);
 
         // If this node became the oldest node.
         if (oldest.id().equals(cctx.localNodeId())) {
@@ -811,7 +816,7 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
         assert nodeId != null;
         assert lock.writeLock().isHeldByCurrentThread();
 
-        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, topVer);
+        ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer);
 
         ClusterNode loc = cctx.localNode();
 
@@ -877,18 +882,6 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public GridDhtPartitionMap2 partitions(UUID nodeId) {
-        lock.readLock().lock();
-
-        try {
-            return node2part.get(nodeId);
-        }
-        finally {
-            lock.readLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override public Map<Integer, Long> updateCounters(boolean skipZeros) {
         lock.readLock().lock();
 
@@ -919,6 +912,27 @@ public class GridClientPartitionTopology implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean hasMovingPartitions() {
+        lock.readLock().lock();
+
+        try {
+            assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part +
+                ", locNodeId=" + cctx.localNodeId() +
+                ", gridName=" + cctx.gridName() + ']';
+
+            for (GridDhtPartitionMap2 map : node2part.values()) {
+                if (map.hasMovingPartitions())
+                    return true;
+            }
+
+            return false;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public void printMemoryStats(int threshold) {
         X.println(">>>  Cache partition topology stats [grid=" + cctx.gridName() + ", cacheId=" + cacheId + ']');
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 39a3e08..668a1cd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -479,7 +479,6 @@ public class GridDhtLocalPartition implements Comparable<GridDhtLocalPartition>,
 
         if ((reservations & 0xFFFF) == 0 && casState(reservations, RENTING)) {
                 shouldBeRenting = false;
-
             if (log.isDebugEnabled())
                 log.debug("Moved partition to RENTING state: " + this);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
index 0f75a5d..14ce1f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopology.java
@@ -195,6 +195,11 @@ public interface GridDhtPartitionTopology {
     public GridDhtPartitionFullMap partitionMap(boolean onlyActive);
 
     /**
+     * @return {@code True} If one of cache nodes has partitions in {@link GridDhtPartitionState#MOVING} state.
+     */
+    public boolean hasMovingPartitions();
+
+    /**
      * @param e Entry removed from cache.
      */
     public void onRemoved(GridDhtCacheEntry e);
@@ -203,9 +208,9 @@ public interface GridDhtPartitionTopology {
      * @param exchId Exchange ID.
      * @param partMap Update partition map.
      * @param cntrMap Partition update counters.
-     * @return Local partition map if there were evictions or {@code null} otherwise.
+     * @return {@code True} if topology state changed.
      */
-    public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+    public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
         GridDhtPartitionFullMap partMap,
         @Nullable Map<Integer, Long> cntrMap);
 
@@ -213,11 +218,18 @@ public interface GridDhtPartitionTopology {
      * @param exchId Exchange ID.
      * @param parts Partitions.
      * @param cntrMap Partition update counters.
-     * @return Local partition map if there were evictions or {@code null} otherwise.
+     * @param checkEvictions Check evictions flag.
+     * @return {@code True} if topology state changed.
      */
-    @Nullable public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+    @Nullable public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
         GridDhtPartitionMap2 parts,
-        @Nullable Map<Integer, Long> cntrMap);
+        @Nullable Map<Integer, Long> cntrMap,
+        boolean checkEvictions);
+
+    /**
+     *
+     */
+    public void checkEvictions();
 
     /**
      * @param skipZeros If {@code true} then filters out zero counters.
@@ -238,12 +250,6 @@ public interface GridDhtPartitionTopology {
     public void onEvicted(GridDhtLocalPartition part, boolean updateSeq);
 
     /**
-     * @param nodeId Node to get partitions for.
-     * @return Partitions for node.
-     */
-    @Nullable public GridDhtPartitionMap2 partitions(UUID nodeId);
-
-    /**
      * Prints memory stats.
      *
      * @param threshold Threshold for number of entries.

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
index ab573bd..1b4dcc9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtPartitionTopologyImpl.java
@@ -44,7 +44,6 @@ import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.Gri
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionFullMap;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionMap2;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.GridDhtPartitionsExchangeFuture;
-import org.apache.ignite.internal.util.F0;
 import org.apache.ignite.internal.util.GridAtomicLong;
 import org.apache.ignite.internal.util.StripedCompositeReadWriteLock;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
@@ -330,8 +329,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Override public void initPartitions(
-        GridDhtPartitionsExchangeFuture exchFut) throws IgniteInterruptedCheckedException {
+    @Override public void initPartitions(GridDhtPartitionsExchangeFuture exchFut)
+        throws IgniteInterruptedCheckedException
+    {
         U.writeLock(lock);
 
         try {
@@ -356,9 +356,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     private void initPartitions0(GridDhtPartitionsExchangeFuture exchFut, long updateSeq) {
         ClusterNode loc = cctx.localNode();
 
-        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer);
-
-        assert oldest != null || cctx.kernalContext().clientNode();
+        ClusterNode oldest = currentCoordinator();
 
         GridDhtPartitionExchangeId exchId = exchFut.exchangeId();
 
@@ -397,7 +395,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         if (log.isDebugEnabled())
                             log.debug("Owned partition for oldest node: " + locPart);
 
-                        updateLocal(p, loc.id(), locPart.state(), updateSeq);
+                        updateSeq = updateLocal(p, locPart.state(), updateSeq);
                     }
                 }
             }
@@ -419,7 +417,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         if (state.active()) {
                             locPart.rent(false);
 
-                            updateLocal(p, loc.id(), locPart.state(), updateSeq);
+                            updateSeq = updateLocal(p, locPart.state(), updateSeq);
 
                             if (log.isDebugEnabled())
                                 log.debug("Evicting partition with rebalancing disabled " +
@@ -443,8 +441,6 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
      * @param updateSeq Update sequence.
      */
     private void createPartitions(List<List<ClusterNode>> aff, long updateSeq) {
-        ClusterNode loc = cctx.localNode();
-
         int num = cctx.affinity().partitions();
 
         for (int p = 0; p < num; p++) {
@@ -454,7 +450,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     // will be created in MOVING state.
                     GridDhtLocalPartition locPart = createPartition(p);
 
-                    updateLocal(p, loc.id(), locPart.state(), updateSeq);
+                    updateSeq = updateLocal(p, locPart.state(), updateSeq);
                 }
             }
             // If this node's map is empty, we pre-create local partitions,
@@ -485,10 +481,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             if (exchId.isLeft())
                 removeNode(exchId.nodeId());
 
-            // In case if node joins, get topology at the time of joining node.
-            ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer);
-
-            assert oldest != null || cctx.kernalContext().clientNode();
+            ClusterNode oldest = currentCoordinator();
 
             if (log.isDebugEnabled())
                 log.debug("Partition map beforeExchange [exchId=" + exchId + ", fullMap=" + fullMapString() + ']');
@@ -548,8 +541,6 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     @Override public boolean afterExchange(GridDhtPartitionsExchangeFuture exchFut) throws IgniteCheckedException {
         boolean changed = waitForRent();
 
-        ClusterNode loc = cctx.localNode();
-
         int num = cctx.affinity().partitions();
 
         AffinityTopologyVersion topVer = exchFut.topologyVersion();
@@ -600,7 +591,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                                 assert owned : "Failed to own partition [cacheName" + cctx.name() + ", locPart=" +
                                     locPart + ']';
 
-                                updateLocal(p, loc.id(), locPart.state(), updateSeq);
+                                updateSeq = updateLocal(p, locPart.state(), updateSeq);
 
                                 changed = true;
 
@@ -620,7 +611,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                                     locPart + ", owners = " + owners + ']');
                         }
                         else
-                            updateLocal(p, loc.id(), locPart.state(), updateSeq);
+                            updateSeq = updateLocal(p, locPart.state(), updateSeq);
                     }
                 }
                 else {
@@ -630,7 +621,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                         if (state == MOVING) {
                             locPart.rent(false);
 
-                            updateLocal(p, loc.id(), locPart.state(), updateSeq);
+                            updateSeq = updateLocal(p, locPart.state(), updateSeq);
 
                             changed = true;
 
@@ -803,8 +794,11 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 map.put(i, part.state());
             }
 
-            return new GridDhtPartitionMap2(cctx.nodeId(), updateSeq.get(), topVer,
-                Collections.unmodifiableMap(map), true);
+            return new GridDhtPartitionMap2(cctx.nodeId(),
+                updateSeq.get(),
+                topVer,
+                Collections.unmodifiableMap(map),
+                true);
         }
         finally {
             lock.readLock().unlock();
@@ -985,7 +979,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
-    @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+    @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
         GridDhtPartitionFullMap partMap,
         @Nullable Map<Integer, Long> cntrMap) {
         if (log.isDebugEnabled())
@@ -997,7 +991,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
         try {
             if (stopping)
-                return null;
+                return false;
 
             if (cntrMap != null) {
                 for (Map.Entry<Integer, Long> e : cntrMap.entrySet()) {
@@ -1025,7 +1019,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     log.debug("Stale exchange id for full partition map update (will ignore) [lastExchId=" +
                         lastExchangeId + ", exchId=" + exchId + ']');
 
-                return null;
+                return false;
             }
 
             if (node2part != null && node2part.compareTo(partMap) >= 0) {
@@ -1033,7 +1027,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     log.debug("Stale partition map for full partition map update (will ignore) [lastExchId=" +
                         lastExchangeId + ", exchId=" + exchId + ", curMap=" + node2part + ", newMap=" + partMap + ']');
 
-                return null;
+                return false;
             }
 
             long updateSeq = this.updateSeq.incrementAndGet();
@@ -1076,7 +1070,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
             node2part = partMap;
 
-            Map<Integer, Set<UUID>> p2n = new HashMap<>(cctx.affinity().partitions(), 1.0f);
+            Map<Integer, Set<UUID>> p2n = U.newHashMap(cctx.affinity().partitions());
 
             for (Map.Entry<UUID, GridDhtPartitionMap2> e : partMap.entrySet()) {
                 for (Integer p : e.getValue().keySet()) {
@@ -1110,7 +1104,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             if (log.isDebugEnabled())
                 log.debug("Partition map after full update: " + fullMapString());
 
-            return changed ? localPartitionMap() : null;
+            return changed;
         }
         finally {
             lock.writeLock().unlock();
@@ -1118,10 +1112,10 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
-    @Nullable @Override public GridDhtPartitionMap2 update(@Nullable GridDhtPartitionExchangeId exchId,
+    @Nullable @Override public boolean update(@Nullable GridDhtPartitionExchangeId exchId,
         GridDhtPartitionMap2 parts,
-        @Nullable Map<Integer, Long> cntrMap) {
+        @Nullable Map<Integer, Long> cntrMap,
+        boolean checkEvictions) {
         if (log.isDebugEnabled())
             log.debug("Updating single partition map [exchId=" + exchId + ", parts=" + mapString(parts) + ']');
 
@@ -1130,33 +1124,28 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                 log.debug("Received partition update for non-existing node (will ignore) [exchId=" + exchId +
                     ", parts=" + parts + ']');
 
-            return null;
+            return false;
         }
 
         lock.writeLock().lock();
 
         try {
             if (stopping)
-                return null;
+                return false;
 
             if (cntrMap != null) {
                 for (Map.Entry<Integer, Long> e : cntrMap.entrySet()) {
-                    Long cntr = this.cntrMap.get(e.getKey());
+                    Integer p = e.getKey();
 
-                    if (cntr == null || cntr < e.getValue())
-                        this.cntrMap.put(e.getKey(), e.getValue());
-                }
-
-                for (int i = 0; i < locParts.length(); i++) {
-                    GridDhtLocalPartition part = locParts.get(i);
+                    Long cntr = this.cntrMap.get(p);
 
-                    if (part == null)
-                        continue;
+                    if (cntr == null || cntr < e.getValue())
+                        this.cntrMap.put(p, e.getValue());
 
-                    Long cntr = cntrMap.get(part.id());
+                    GridDhtLocalPartition part = locParts.get(p);
 
-                    if (cntr != null)
-                        part.updateCounter(cntr);
+                    if (part != null)
+                        part.updateCounter(e.getValue());
                 }
             }
 
@@ -1165,7 +1154,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     log.debug("Stale exchange id for single partition map update (will ignore) [lastExchId=" +
                         lastExchangeId + ", exchId=" + exchId + ']');
 
-                return null;
+                return false;
             }
 
             if (exchId != null)
@@ -1182,60 +1171,91 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     log.debug("Stale update sequence for single partition map update (will ignore) [exchId=" + exchId +
                         ", curSeq=" + cur.updateSequence() + ", newSeq=" + parts.updateSequence() + ']');
 
-                return null;
+                return false;
             }
 
             long updateSeq = this.updateSeq.incrementAndGet();
 
-            node2part = new GridDhtPartitionFullMap(node2part, updateSeq);
-
-            boolean changed = false;
+            node2part.newUpdateSequence(updateSeq);
 
-            if (cur == null || !cur.equals(parts))
-                changed = true;
+            boolean changed = cur == null || !cur.equals(parts);
 
-            node2part.put(parts.nodeId(), parts);
+            if (changed) {
+                node2part.put(parts.nodeId(), parts);
 
-            part2node = new HashMap<>(part2node);
+                // Add new mappings.
+                for (Integer p : parts.keySet()) {
+                    Set<UUID> ids = part2node.get(p);
 
-            // Add new mappings.
-            for (Integer p : parts.keySet()) {
-                Set<UUID> ids = part2node.get(p);
+                    if (ids == null)
+                        // Initialize HashSet to size 3 in anticipation that there won't be
+                        // more than 3 nodes per partition.
+                        part2node.put(p, ids = U.newHashSet(3));
 
-                if (ids == null)
-                    // Initialize HashSet to size 3 in anticipation that there won't be
-                    // more than 3 nodes per partition.
-                    part2node.put(p, ids = U.newHashSet(3));
+                    ids.add(parts.nodeId());
+                }
 
-                changed |= ids.add(parts.nodeId());
-            }
+                // Remove obsolete mappings.
+                if (cur != null) {
+                    for (Integer p : cur.keySet()) {
+                        if (parts.containsKey(p))
+                            continue;
 
-            // Remove obsolete mappings.
-            if (cur != null) {
-                for (Integer p : F.view(cur.keySet(), F0.notIn(parts.keySet()))) {
-                    Set<UUID> ids = part2node.get(p);
+                        Set<UUID> ids = part2node.get(p);
 
-                    if (ids != null)
-                        changed |= ids.remove(parts.nodeId());
+                        if (ids != null)
+                            ids.remove(parts.nodeId());
+                    }
                 }
             }
+            else
+                cur.updateSequence(parts.updateSequence(), parts.topologyVersion());
 
-            AffinityTopologyVersion affVer = cctx.affinity().affinityTopologyVersion();
-
-            if (!affVer.equals(AffinityTopologyVersion.NONE) && affVer.compareTo(topVer) >= 0) {
-                List<List<ClusterNode>> aff = cctx.affinity().assignments(topVer);
-
-                changed |= checkEvictions(updateSeq, aff);
-
-                updateRebalanceVersion(aff);
-            }
+            if (checkEvictions)
+                changed |= checkEvictions(updateSeq);
 
             consistencyCheck();
 
             if (log.isDebugEnabled())
                 log.debug("Partition map after single update: " + fullMapString());
 
-            return changed ? localPartitionMap() : null;
+            return changed;
+        }
+        finally {
+            lock.writeLock().unlock();
+        }
+    }
+
+    /**
+     * @param updateSeq Update sequence.
+     * @return {@code True} if state changed.
+     */
+    private boolean checkEvictions(long updateSeq) {
+        AffinityTopologyVersion affVer = cctx.affinity().affinityTopologyVersion();
+
+        boolean changed = false;
+
+        if (!affVer.equals(AffinityTopologyVersion.NONE) && affVer.compareTo(topVer) >= 0) {
+            List<List<ClusterNode>> aff = cctx.affinity().assignments(topVer);
+
+            changed = checkEvictions(updateSeq, aff);
+
+            updateRebalanceVersion(aff);
+        }
+
+        return changed;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void checkEvictions() {
+        lock.writeLock().lock();
+
+        try {
+            long updateSeq = this.updateSeq.incrementAndGet();
+
+            node2part.newUpdateSequence(updateSeq);
+
+            checkEvictions(updateSeq);
         }
         finally {
             lock.writeLock().unlock();
@@ -1270,7 +1290,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                     if (nodeIds.containsAll(F.nodeIds(affNodes))) {
                         part.rent(false);
 
-                        updateLocal(part.id(), locId, part.state(), updateSeq);
+                        updateSeq = updateLocal(part.id(), part.state(), updateSeq);
 
                         changed = true;
 
@@ -1295,7 +1315,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
                                 if (locId.equals(n.id())) {
                                     part.rent(false);
 
-                                    updateLocal(part.id(), locId, part.state(), updateSeq);
+                                    updateSeq = updateLocal(part.id(), part.state(), updateSeq);
 
                                     changed = true;
 
@@ -1316,19 +1336,27 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /**
+     * @return Current coordinator node.
+     */
+    @Nullable private ClusterNode currentCoordinator() {
+        ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(topVer);
+
+        assert oldest != null || cctx.kernalContext().clientNode();
+
+        return oldest;
+    }
+
+    /**
      * Updates value for single partition.
      *
      * @param p Partition.
-     * @param nodeId Node ID.
      * @param state State.
      * @param updateSeq Update sequence.
+     * @return Update sequence.
      */
     @SuppressWarnings({"MismatchedQueryAndUpdateOfCollection"})
-    private void updateLocal(int p, UUID nodeId, GridDhtPartitionState state, long updateSeq) {
-        assert nodeId.equals(cctx.nodeId());
-
-        // In case if node joins, get topology at the time of joining node.
-        ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx.shared(), topVer);
+    private long updateLocal(int p, GridDhtPartitionState state, long updateSeq) {
+        ClusterNode oldest = currentCoordinator();
 
         assert oldest != null || cctx.kernalContext().clientNode();
 
@@ -1338,12 +1366,16 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
             if (seq != updateSeq) {
                 if (seq > updateSeq) {
-                    if (this.updateSeq.get() < seq) {
+                    long seq0 = this.updateSeq.get();
+
+                    if (seq0 < seq) {
                         // Update global counter if necessary.
-                        boolean b = this.updateSeq.compareAndSet(this.updateSeq.get(), seq + 1);
+                        boolean b = this.updateSeq.compareAndSet(seq0, seq + 1);
 
-                        assert b : "Invalid update sequence [updateSeq=" + updateSeq + ", seq=" + seq +
-                            ", curUpdateSeq=" + this.updateSeq.get() + ", node2part=" + node2part.toFullString() + ']';
+                        assert b : "Invalid update sequence [updateSeq=" + updateSeq +
+                            ", seq=" + seq +
+                            ", curUpdateSeq=" + this.updateSeq.get() +
+                            ", node2part=" + node2part.toFullString() + ']';
 
                         updateSeq = seq + 1;
                     }
@@ -1355,11 +1387,19 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             }
         }
 
-        GridDhtPartitionMap2 map = node2part.get(nodeId);
+        UUID locNodeId = cctx.localNodeId();
+
+        GridDhtPartitionMap2 map = node2part.get(locNodeId);
 
-        if (map == null)
-            node2part.put(nodeId, map = new GridDhtPartitionMap2(nodeId, updateSeq, topVer,
-                Collections.<Integer, GridDhtPartitionState>emptyMap(), false));
+        if (map == null) {
+            map = new GridDhtPartitionMap2(locNodeId,
+                updateSeq,
+                topVer,
+                Collections.<Integer, GridDhtPartitionState>emptyMap(),
+                false);
+
+            node2part.put(locNodeId, map);
+        }
 
         map.updateSequence(updateSeq, topVer);
 
@@ -1370,7 +1410,9 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
         if (ids == null)
             part2node.put(p, ids = U.newHashSet(3));
 
-        ids.add(nodeId);
+        ids.add(locNodeId);
+
+        return updateSeq;
     }
 
     /**
@@ -1395,8 +1437,6 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             else
                 node2part = new GridDhtPartitionFullMap(node2part, node2part.updateSequence());
 
-            part2node = new HashMap<>(part2node);
-
             GridDhtPartitionMap2 parts = node2part.remove(nodeId);
 
             if (parts != null) {
@@ -1418,13 +1458,11 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
     /** {@inheritDoc} */
     @Override public boolean own(GridDhtLocalPartition part) {
-        ClusterNode loc = cctx.localNode();
-
         lock.writeLock().lock();
 
         try {
             if (part.own()) {
-                updateLocal(part.id(), loc.id(), part.state(), updateSeq.incrementAndGet());
+                updateLocal(part.id(), part.state(), updateSeq.incrementAndGet());
 
                 consistencyCheck();
 
@@ -1452,7 +1490,7 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
 
             long seq = updateSeq ? this.updateSeq.incrementAndGet() : this.updateSeq.get();
 
-            updateLocal(part.id(), cctx.localNodeId(), part.state(), seq);
+            updateLocal(part.id(), part.state(), seq);
 
             consistencyCheck();
         }
@@ -1462,18 +1500,6 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public GridDhtPartitionMap2 partitions(UUID nodeId) {
-        lock.readLock().lock();
-
-        try {
-            return node2part.get(nodeId);
-        }
-        finally {
-            lock.readLock().unlock();
-        }
-    }
-
-    /** {@inheritDoc} */
     @Override public Map<Integer, Long> updateCounters(boolean skipZeros) {
         lock.readLock().lock();
 
@@ -1526,6 +1552,30 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
     }
 
     /** {@inheritDoc} */
+    @Override public boolean hasMovingPartitions() {
+        lock.readLock().lock();
+
+        try {
+            assert node2part != null && node2part.valid() : "Invalid node2part [node2part: " + node2part +
+                ", cache=" + cctx.name() +
+                ", started=" + cctx.started() +
+                ", stopping=" + stopping +
+                ", locNodeId=" + cctx.localNode().id() +
+                ", locName=" + cctx.gridName() + ']';
+
+            for (GridDhtPartitionMap2 map : node2part.values()) {
+                if (map.hasMovingPartitions())
+                    return true;
+            }
+
+            return false;
+        }
+        finally {
+            lock.readLock().unlock();
+        }
+    }
+
+    /** {@inheritDoc} */
     @Override public void printMemoryStats(int threshold) {
         X.println(">>>  Cache partition topology stats [grid=" + cctx.gridName() + ", cache=" + cctx.name() + ']');
 
@@ -1607,10 +1657,12 @@ class GridDhtPartitionTopologyImpl implements GridDhtPartitionTopology {
             if (state == match)
                 return true;
 
-            if (matches != null && matches.length > 0)
-                for (GridDhtPartitionState s : matches)
+            if (matches != null && matches.length > 0) {
+                for (GridDhtPartitionState s : matches) {
                     if (state == s)
                         return true;
+                }
+            }
         }
 
         return false;

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
index 8f5ad17..e8860f2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionFullMap.java
@@ -103,10 +103,13 @@ public class GridDhtPartitionFullMap extends HashMap<UUID, GridDhtPartitionMap2>
         for (Map.Entry<UUID, GridDhtPartitionMap2> e : m.entrySet()) {
             GridDhtPartitionMap2 part = e.getValue();
 
-            if (onlyActive)
-                put(e.getKey(), new GridDhtPartitionMap2(part.nodeId(), part.updateSequence(), part.topologyVersion(), part.map(), true));
-            else
-                put(e.getKey(), part);
+            GridDhtPartitionMap2 cpy = new GridDhtPartitionMap2(part.nodeId(),
+                part.updateSequence(),
+                part.topologyVersion(),
+                part.map(),
+                onlyActive);
+
+            put(e.getKey(), cpy);
         }
     }
 
@@ -177,6 +180,13 @@ public class GridDhtPartitionFullMap extends HashMap<UUID, GridDhtPartitionMap2>
 
     /**
      * @param updateSeq New update sequence value.
+     */
+    public void newUpdateSequence(long updateSeq) {
+        this.updateSeq = updateSeq;
+    }
+
+    /**
+     * @param updateSeq New update sequence value.
      * @return Old update sequence value.
      */
     public long updateSequence(long updateSeq) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index f391265..e945de9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -112,6 +112,10 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
     /** */
     @GridToStringExclude
+    private int pendingSingleUpdates;
+
+    /** */
+    @GridToStringExclude
     private List<ClusterNode> srvNodes;
 
     /** */
@@ -1162,13 +1166,16 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
      */
     private void processMessage(ClusterNode node, GridDhtPartitionsSingleMessage msg) {
         boolean allReceived = false;
+        boolean updateSingleMap = false;
 
         synchronized (mux) {
             assert crd != null;
 
             if (crd.isLocal()) {
                 if (remaining.remove(node.id())) {
-                    updatePartitionSingleMap(msg);
+                    updateSingleMap = true;
+
+                    pendingSingleUpdates++;
 
                     allReceived = remaining.isEmpty();
                 }
@@ -1177,8 +1184,42 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 singleMsgs.put(node, msg);
         }
 
-        if (allReceived)
+        if (updateSingleMap) {
+            try {
+                updatePartitionSingleMap(msg);
+            }
+            finally {
+                synchronized (mux) {
+                    assert pendingSingleUpdates > 0;
+
+                    pendingSingleUpdates--;
+
+                    if (pendingSingleUpdates == 0)
+                        mux.notifyAll();
+                }
+            }
+        }
+
+        if (allReceived) {
+            awaitSingleMapUpdates();
+
             onAllReceived(false);
+        }
+    }
+
+    /**
+     *
+     */
+    private void awaitSingleMapUpdates() {
+        synchronized (mux) {
+            try {
+                while (pendingSingleUpdates > 0)
+                    U.wait(mux);
+            }
+            catch (IgniteInterruptedCheckedException e) {
+                U.warn(log, "Failed to wait for partition map updates, thread was interrupted: " + e);
+            }
+        }
     }
 
     /**
@@ -1218,6 +1259,11 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
                 }
             }
 
+            for (GridCacheContext cacheCtx : cctx.cacheContexts()) {
+                if (!cacheCtx.isLocal())
+                    cacheCtx.topology().checkEvictions();
+            }
+
             updateLastVersion(cctx.versions().last());
 
             cctx.versions().onExchange(lastVer.get().order());
@@ -1374,7 +1420,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             if (cacheCtx != null)
                 cacheCtx.topology().update(exchId, entry.getValue(), cntrMap);
             else {
-                ClusterNode oldest = CU.oldestAliveCacheServerNode(cctx, AffinityTopologyVersion.NONE);
+                ClusterNode oldest = cctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE);
 
                 if (oldest != null && oldest.isLocal())
                     cctx.exchange().clientTopology(cacheId, this).update(exchId, entry.getValue(), cntrMap);
@@ -1395,7 +1441,7 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
             GridDhtPartitionTopology top = cacheCtx != null ? cacheCtx.topology() :
                 cctx.exchange().clientTopology(cacheId, this);
 
-            top.update(exchId, entry.getValue(), msg.partitionUpdateCounters(cacheId));
+            top.update(exchId, entry.getValue(), msg.partitionUpdateCounters(cacheId), false);
         }
     }
 
@@ -1557,6 +1603,8 @@ public class GridDhtPartitionsExchangeFuture extends GridFutureAdapter<AffinityT
 
                         if (crd0.isLocal()) {
                             if (allReceived) {
+                                awaitSingleMapUpdates();
+
                                 onAllReceived(true);
 
                                 return;

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index fbd8ce5..cf69264 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -679,14 +679,14 @@ public class IgniteTxHandler {
      * @param req Request.
      * @return Future.
      */
-    @Nullable public IgniteInternalFuture<IgniteInternalTx> processNearTxFinishRequest(UUID nodeId,
+    @Nullable private IgniteInternalFuture<IgniteInternalTx> processNearTxFinishRequest(UUID nodeId,
         GridNearTxFinishRequest req) {
         if (txFinishMsgLog.isDebugEnabled())
             txFinishMsgLog.debug("Received near finish request [txId=" + req.version() + ", node=" + nodeId + ']');
 
         IgniteInternalFuture<IgniteInternalTx> fut = finish(nodeId, null, req);
 
-        assert req.txState() != null || fut.error() != null ||
+        assert req.txState() != null || (fut != null && fut.error() != null) ||
             (ctx.tm().tx(req.version()) == null && ctx.tm().nearTx(req.version()) == null);
 
         return fut;

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 6c26363..b9b92b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -1265,7 +1265,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         try {
             if (!cache.context().affinityNode()) {
                 ClusterNode oldestSrvNode =
-                    CU.oldestAliveCacheServerNode(cache.context().shared(), AffinityTopologyVersion.NONE);
+                    ctx.discovery().oldestAliveCacheServerNode(AffinityTopologyVersion.NONE);
 
                 if (oldestSrvNode == null)
                     return new GridEmptyIterator<>();
@@ -1589,7 +1589,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
 
                 depExe.submit(new BusyRunnable() {
                     @Override public void run0() {
-                        ClusterNode oldest = CU.oldestAliveCacheServerNode(cache.context().shared(), topVer);
+                        ClusterNode oldest = ctx.discovery().oldestAliveCacheServerNode(topVer);
 
                         if (oldest != null && oldest.isLocal()) {
                             final Collection<GridServiceDeployment> retries = new ConcurrentLinkedQueue<>();

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
index 8814745..a660ec8 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
@@ -129,6 +129,7 @@ import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingRequest;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryPingResponse;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryRedirectToClient;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryStatusCheckMessage;
+import org.apache.ignite.thread.IgniteThreadPoolExecutor;
 import org.jetbrains.annotations.Nullable;
 import org.jsr166.ConcurrentHashMap8;
 
@@ -174,8 +175,7 @@ class ServerImpl extends TcpDiscoveryImpl {
         IgniteProductVersion.fromString("1.5.0");
 
     /** */
-    private final ThreadPoolExecutor utilityPool = new ThreadPoolExecutor(0, 1, 2000, TimeUnit.MILLISECONDS,
-        new LinkedBlockingQueue<Runnable>());
+    private IgniteThreadPoolExecutor utilityPool;
 
     /** Nodes ring. */
     @GridToStringExclude
@@ -297,6 +297,13 @@ class ServerImpl extends TcpDiscoveryImpl {
             spiState = DISCONNECTED;
         }
 
+        utilityPool = new IgniteThreadPoolExecutor("disco-pool",
+            spi.ignite().name(),
+            0,
+            1,
+            2000,
+            new LinkedBlockingQueue<Runnable>());
+
         if (debugMode) {
             if (!log.isInfoEnabled())
                 throw new IgniteSpiException("Info log level should be enabled for TCP discovery to work " +
@@ -2403,9 +2410,12 @@ class ServerImpl extends TcpDiscoveryImpl {
         /** Connection check threshold. */
         private long connCheckThreshold;
 
+        /** */
+        private long lastRingMsgTime;
+
         /**
          */
-        protected RingMessageWorker() {
+        RingMessageWorker() {
             super("tcp-disco-msg-worker", 10);
 
             initConnectionCheckFrequency();
@@ -2500,6 +2510,8 @@ class ServerImpl extends TcpDiscoveryImpl {
          * @param msg Message to process.
          */
         @Override protected void processMessage(TcpDiscoveryAbstractMessage msg) {
+            sendHeartbeatMessage();
+
             DebugLogger log = messageLogger(msg);
 
             if (log.isDebugEnabled())
@@ -2508,6 +2520,11 @@ class ServerImpl extends TcpDiscoveryImpl {
             if (debugMode)
                 debugLog(msg, "Processing message [cls=" + msg.getClass().getSimpleName() + ", id=" + msg.id() + ']');
 
+            boolean ensured = spi.ensured(msg);
+
+            if (!locNode.id().equals(msg.senderNodeId()) && ensured)
+                lastRingMsgTime = U.currentTimeMillis();
+
             if (locNode.internalOrder() == 0) {
                 boolean proc = false;
 
@@ -2564,7 +2581,7 @@ class ServerImpl extends TcpDiscoveryImpl {
             else
                 assert false : "Unknown message type: " + msg.getClass().getSimpleName();
 
-            if (spi.ensured(msg) && redirectToClients(msg))
+            if (ensured && redirectToClients(msg))
                 msgHist.add(msg);
 
             if (msg.senderNodeId() != null && !msg.senderNodeId().equals(getLocalNodeId())) {
@@ -5336,12 +5353,9 @@ class ServerImpl extends TcpDiscoveryImpl {
          * Sends heartbeat message if needed.
          */
         private void sendHeartbeatMessage() {
-            if (!isLocalNodeCoordinator())
-                return;
-
             long elapsed = (lastTimeHbMsgSent + spi.hbFreq) - U.currentTimeMillis();
 
-            if (elapsed > 0)
+            if (elapsed > 0 || !isLocalNodeCoordinator())
                 return;
 
             TcpDiscoveryHeartbeatMessage msg = new TcpDiscoveryHeartbeatMessage(getConfiguredNodeId());
@@ -5361,7 +5375,9 @@ class ServerImpl extends TcpDiscoveryImpl {
             if (lastTimeStatusMsgSent < locNode.lastUpdateTime())
                 lastTimeStatusMsgSent = locNode.lastUpdateTime();
 
-            long elapsed = (lastTimeStatusMsgSent + hbCheckFreq) - U.currentTimeMillis();
+            long updateTime = Math.max(lastTimeStatusMsgSent, lastRingMsgTime);
+
+            long elapsed = (updateTime + hbCheckFreq) - U.currentTimeMillis();
 
             if (elapsed > 0)
                 return;
@@ -6062,11 +6078,11 @@ class ServerImpl extends TcpDiscoveryImpl {
 
             TcpDiscoverySpiState state = spiStateCopy();
 
-            long socketTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() :
+            long sockTimeout = spi.failureDetectionTimeoutEnabled() ? spi.failureDetectionTimeout() :
                 spi.getSocketTimeout();
 
             if (state == CONNECTED) {
-                spi.writeToSocket(msg, sock, RES_OK, socketTimeout);
+                spi.writeToSocket(msg, sock, RES_OK, sockTimeout);
 
                 if (log.isDebugEnabled())
                     log.debug("Responded to join request message [msg=" + msg + ", res=" + RES_OK + ']');
@@ -6103,7 +6119,7 @@ class ServerImpl extends TcpDiscoveryImpl {
                     // Local node is stopping. Remote node should try next one.
                     res = RES_CONTINUE_JOIN;
 
-                spi.writeToSocket(msg, sock, res, socketTimeout);
+                spi.writeToSocket(msg, sock, res, sockTimeout);
 
                 if (log.isDebugEnabled())
                     log.debug("Responded to join request message [msg=" + msg + ", res=" + res + ']');

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
index 9e73632..c790644 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/internal/TcpDiscoveryStatistics.java
@@ -28,6 +28,8 @@ import org.apache.ignite.internal.util.typedef.internal.S;
 import org.apache.ignite.internal.util.typedef.internal.U;
 import org.apache.ignite.lang.IgniteUuid;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryAbstractMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryCustomEventMessage;
+import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddFinishedMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeAddedMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeFailedMessage;
 import org.apache.ignite.spi.discovery.tcp.messages.TcpDiscoveryNodeLeftMessage;
@@ -317,7 +319,9 @@ public class TcpDiscoveryStatistics {
         assert time >= 0 : time;
 
         if (crdSinceTs.get() > 0 &&
+            (msg instanceof TcpDiscoveryCustomEventMessage) ||
             (msg instanceof TcpDiscoveryNodeAddedMessage) ||
+            (msg instanceof TcpDiscoveryNodeAddFinishedMessage) ||
             (msg instanceof TcpDiscoveryNodeLeftMessage) ||
             (msg instanceof TcpDiscoveryNodeFailedMessage)) {
             ringMsgsSndTs.put(msg.id(), U.currentTimeMillis());

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
index 878d7d1..43017db 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/AbstractAffinityFunctionSelfTest.java
@@ -104,6 +104,7 @@ public abstract class AbstractAffinityFunctionSelfTest extends GridCommonAbstrac
     }
 
     /**
+     * @param backups Number of backups.
      * @throws Exception If failed.
      */
     protected void checkNodeRemoved(int backups) throws Exception {
@@ -247,7 +248,6 @@ public abstract class AbstractAffinityFunctionSelfTest extends GridCommonAbstrac
         }
     }
 
-
     /**
      * @param assignment Assignment to verify.
      */

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index 390c83e..31b4bc7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -239,7 +239,7 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
                 GridCacheSharedContext<?, ?> ctx = k.context().cache().context();
 
                 ClusterNode oldest =
-                    GridCacheUtils.oldestAliveCacheServerNode(ctx, new AffinityTopologyVersion(currVer));
+                    ctx.discovery().oldestAliveCacheServerNode(new AffinityTopologyVersion(currVer));
 
                 assertNotNull(oldest);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/acf20b32/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
index a59ca8b..2d46cf4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedAffinitySpreadTest.java
@@ -76,7 +76,12 @@ public class GridCachePartitionedAffinitySpreadTest extends GridCommonAbstractTe
         Map<ClusterNode, Integer> parts = new HashMap<>(nodes.size());
 
         for (int part = 0; part < aff.getPartitions(); part++) {
-            Collection<ClusterNode> affNodes = aff.assignPartition(part, new ArrayList(nodes), 0, null);
+            Collection<ClusterNode> affNodes = aff.assignPartition(null,
+                part,
+                new ArrayList<>(nodes),
+                new HashMap<ClusterNode, byte[]>(),
+                0,
+                null);
 
             assertEquals(1, affNodes.size());
 


[22/25] ignite git commit: Added benchmark for atomic increments w/ and w/o thread-local batching.

Posted by sb...@apache.org.
Added benchmark for atomic increments w/ and w/o thread-local batching.


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

Branch: refs/heads/ignite-4371
Commit: b65802b9d32d6c52bcf35b3a52a39b12ea34ec0a
Parents: 065ca4a
Author: devozerov <vo...@gridgain.com>
Authored: Fri Dec 9 13:10:22 2016 +0300
Committer: devozerov <vo...@gridgain.com>
Committed: Fri Dec 9 13:10:22 2016 +0300

----------------------------------------------------------------------
 .../jmh/misc/JmhIncrementBenchmark.java         | 95 ++++++++++++++++++++
 1 file changed, 95 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ignite/blob/b65802b9/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/JmhIncrementBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/JmhIncrementBenchmark.java b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/JmhIncrementBenchmark.java
new file mode 100644
index 0000000..629fdd3
--- /dev/null
+++ b/modules/benchmarks/src/main/java/org/apache/ignite/internal/benchmarks/jmh/misc/JmhIncrementBenchmark.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.benchmarks.jmh.misc;
+
+import org.apache.ignite.internal.benchmarks.jmh.JmhAbstractBenchmark;
+import org.apache.ignite.internal.benchmarks.jmh.runner.JmhIdeBenchmarkRunner;
+import org.openjdk.jmh.annotations.Benchmark;
+import org.openjdk.jmh.annotations.Scope;
+import org.openjdk.jmh.annotations.State;
+
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Test various increment strategies.
+ */
+@State(Scope.Benchmark)
+public class JmhIncrementBenchmark extends JmhAbstractBenchmark {
+    /** Batch size. */
+    private static final int BATCH_SIZE = 1024;
+
+    /** Thread-local value. */
+    private static final ThreadLocal<Long> VAL = new ThreadLocal<Long>() {
+        @Override protected Long initialValue() {
+            return 0L;
+        }
+    };
+
+    /** Atomic variable. */
+    private final AtomicLong atomic = new AtomicLong();
+
+    /**
+     * Test standard increment.
+     */
+    @Benchmark
+    public long increment() {
+        return atomic.incrementAndGet();
+    }
+
+    /**
+     * Test increment with thread-local batching.
+     */
+    @Benchmark
+    public long threadLocalIncrement() {
+        long val = VAL.get();
+
+        if ((val & (BATCH_SIZE - 1)) == 0)
+            val = atomic.addAndGet(BATCH_SIZE);
+
+        VAL.set(++val);
+
+        return val;
+    }
+
+    /**
+     * Run benchmarks.
+     *
+     * @param args Arguments.
+     * @throws Exception If failed.
+     */
+    public static void main(String[] args) throws Exception {
+        run(8);
+    }
+
+    /**
+     * Run benchmark.
+     *
+     * @param threads Amount of threads.
+     * @throws Exception If failed.
+     */
+    private static void run(int threads) throws Exception {
+        JmhIdeBenchmarkRunner.create()
+            .forks(1)
+            .threads(threads)
+            .warmupIterations(30)
+            .measurementIterations(30)
+            .benchmarks(JmhIncrementBenchmark.class.getSimpleName())
+            .jvmArguments("-Xms4g", "-Xmx4g")
+            .run();
+    }
+}


[18/25] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NIO sessio

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
index 63c9845..66f9176 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridSelectorNioSessionImpl.java
@@ -20,9 +20,11 @@ package org.apache.ignite.internal.util.nio;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.nio.channels.SelectionKey;
+import java.util.ArrayList;
 import java.util.Collection;
+import java.util.List;
 import java.util.concurrent.Semaphore;
-import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicBoolean;
 import org.apache.ignite.IgniteLogger;
 import org.apache.ignite.internal.util.tostring.GridToStringExclude;
 import org.apache.ignite.internal.util.typedef.internal.LT;
@@ -37,17 +39,14 @@ import org.jsr166.ConcurrentLinkedDeque8;
  */
 class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     /** Pending write requests. */
-    private final ConcurrentLinkedDeque8<GridNioFuture<?>> queue = new ConcurrentLinkedDeque8<>();
+    private final ConcurrentLinkedDeque8<SessionWriteRequest> queue = new ConcurrentLinkedDeque8<>();
 
     /** Selection key associated with this session. */
     @GridToStringExclude
     private SelectionKey key;
 
-    /** Worker index for server */
-    private final int selectorIdx;
-
-    /** Size counter. */
-    private final AtomicInteger queueSize = new AtomicInteger();
+    /** Current worker thread. */
+    private volatile GridNioWorker worker;
 
     /** Semaphore. */
     @GridToStringExclude
@@ -59,17 +58,29 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     /** Read buffer. */
     private ByteBuffer readBuf;
 
-    /** Recovery data. */
-    private GridNioRecoveryDescriptor recovery;
+    /** Incoming recovery data. */
+    private GridNioRecoveryDescriptor inRecovery;
+
+    /** Outgoing recovery data. */
+    private GridNioRecoveryDescriptor outRecovery;
 
     /** Logger. */
     private final IgniteLogger log;
 
+    /** */
+    private List<GridNioServer.SessionChangeRequest> pendingStateChanges;
+
+    /** */
+    final AtomicBoolean procWrite = new AtomicBoolean();
+
+    /** */
+    private Object sysMsg;
+
     /**
      * Creates session instance.
      *
      * @param log Logger.
-     * @param selectorIdx Selector index for this session.
+     * @param worker NIO worker thread.
      * @param filterChain Filter chain that will handle requests.
      * @param locAddr Local address.
      * @param rmtAddr Remote address.
@@ -80,7 +91,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      */
     GridSelectorNioSessionImpl(
         IgniteLogger log,
-        int selectorIdx,
+        GridNioWorker worker,
         GridNioFilterChain filterChain,
         InetSocketAddress locAddr,
         InetSocketAddress rmtAddr,
@@ -91,7 +102,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     ) {
         super(filterChain, locAddr, rmtAddr, accepted);
 
-        assert selectorIdx >= 0;
+        assert worker != null;
         assert sndQueueLimit >= 0;
 
         assert locAddr != null : "GridSelectorNioSessionImpl should have local socket address.";
@@ -101,7 +112,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
 
         this.log = log;
 
-        this.selectorIdx = selectorIdx;
+        this.worker = worker;
 
         sem = sndQueueLimit > 0 ? new Semaphore(sndQueueLimit) : null;
 
@@ -119,12 +130,19 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     }
 
     /**
+     * @return Worker.
+     */
+    GridNioWorker worker() {
+        return worker;
+    }
+
+    /**
      * Sets selection key for this session.
      *
      * @param key Selection key.
      */
     void key(SelectionKey key) {
-        assert this.key == null;
+        assert key != null;
 
         this.key = key;
     }
@@ -151,10 +169,88 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     }
 
     /**
-     * @return Selector index.
+     * @param from Current session worker.
+     * @param fut Move future.
+     * @return {@code True} if session move was scheduled.
+     */
+    boolean offerMove(GridNioWorker from, GridNioServer.SessionChangeRequest fut) {
+        synchronized (this) {
+            if (log.isDebugEnabled())
+                log.debug("Offered move [ses=" + this + ", fut=" + fut + ']');
+
+            GridNioWorker worker0 = worker;
+
+            if (worker0 != from)
+                return false;
+
+            worker.offer(fut);
+        }
+
+        return true;
+    }
+
+    /**
+     * @param fut Future.
+     */
+    void offerStateChange(GridNioServer.SessionChangeRequest fut) {
+        synchronized (this) {
+            if (log.isDebugEnabled())
+                log.debug("Offered move [ses=" + this + ", fut=" + fut + ']');
+
+            GridNioWorker worker0 = worker;
+
+            if (worker0 == null) {
+                if (pendingStateChanges == null)
+                    pendingStateChanges = new ArrayList<>();
+
+                pendingStateChanges.add(fut);
+            }
+            else
+                worker0.offer(fut);
+        }
+    }
+
+    /**
+     * @param moveFrom Current session worker.
      */
-    int selectorIndex() {
-        return selectorIdx;
+    void startMoveSession(GridNioWorker moveFrom) {
+        synchronized (this) {
+            assert this.worker == moveFrom;
+
+            if (log.isDebugEnabled())
+                log.debug("Started moving [ses=" + this + ", from=" + moveFrom + ']');
+
+            List<GridNioServer.SessionChangeRequest> sesReqs = moveFrom.clearSessionRequests(this);
+
+            worker = null;
+
+            if (sesReqs != null) {
+                if (pendingStateChanges == null)
+                    pendingStateChanges = new ArrayList<>();
+
+                pendingStateChanges.addAll(sesReqs);
+            }
+        }
+    }
+
+    /**
+     * @param moveTo New session worker.
+     */
+    void finishMoveSession(GridNioWorker moveTo) {
+        synchronized (this) {
+            assert worker == null;
+
+            if (log.isDebugEnabled())
+                log.debug("Finishing moving [ses=" + this + ", to=" + moveTo + ']');
+
+            worker = moveTo;
+
+            if (pendingStateChanges != null) {
+                moveTo.offer(pendingStateChanges);
+
+                pendingStateChanges = null;
+            }
+        }
     }
 
     /**
@@ -163,14 +259,14 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      * @param writeFut Write request.
      * @return Updated size of the queue.
      */
-    int offerSystemFuture(GridNioFuture<?> writeFut) {
+    int offerSystemFuture(SessionWriteRequest writeFut) {
         writeFut.messageThread(true);
 
         boolean res = queue.offerFirst(writeFut);
 
         assert res : "Future was not added to queue";
 
-        return queueSize.incrementAndGet();
+        return queue.sizex();
     }
 
     /**
@@ -183,7 +279,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      * @param writeFut Write request to add.
      * @return Updated size of the queue.
      */
-    int offerFuture(GridNioFuture<?> writeFut) {
+    int offerFuture(SessionWriteRequest writeFut) {
         boolean msgThread = GridNioBackPressureControl.threadProcessingMessage();
 
         if (sem != null && !msgThread)
@@ -195,47 +291,41 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
 
         assert res : "Future was not added to queue";
 
-        return queueSize.incrementAndGet();
+        return queue.sizex();
     }
 
     /**
      * @param futs Futures to resend.
      */
-    void resend(Collection<GridNioFuture<?>> futs) {
+    void resend(Collection<SessionWriteRequest> 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() {
-        GridNioFuture<?> last = queue.poll();
+    @Nullable SessionWriteRequest pollFuture() {
+        SessionWriteRequest last = queue.poll();
 
         if (last != null) {
-            queueSize.decrementAndGet();
-
             if (sem != null && !last.messageThread())
                 sem.release();
 
-            if (recovery != null) {
-                if (!recovery.add(last)) {
+            if (outRecovery != null) {
+                if (!outRecovery.add(last)) {
                     LT.warn(log, "Unacknowledged messages queue size overflow, will attempt to reconnect " +
                         "[remoteAddr=" + remoteAddress() +
-                        ", queueLimit=" + recovery.queueLimit() + ']');
+                        ", queueLimit=" + outRecovery.queueLimit() + ']');
 
                     if (log.isDebugEnabled())
                         log.debug("Unacknowledged messages queue size overflow, will attempt to reconnect " +
                             "[remoteAddr=" + remoteAddress() +
-                            ", queueSize=" + recovery.messagesFutures().size() +
-                            ", queueLimit=" + recovery.queueLimit() + ']');
+                            ", queueSize=" + outRecovery.messagesRequests().size() +
+                            ", queueLimit=" + outRecovery.queueLimit() + ']');
 
                     close();
                 }
@@ -249,7 +339,7 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      * @param fut Future.
      * @return {@code True} if future was removed from queue.
      */
-    boolean removeFuture(GridNioFuture<?> fut) {
+    boolean removeFuture(SessionWriteRequest fut) {
         assert closed();
 
         return queue.removeLastOccurrence(fut);
@@ -261,35 +351,49 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
      * @return Number of write requests.
      */
     int writeQueueSize() {
-        return queueSize.get();
+        return queue.sizex();
     }
 
     /**
      * @return Write requests.
      */
-    Collection<GridNioFuture<?>> writeQueue() {
+    Collection<SessionWriteRequest> writeQueue() {
         return queue;
     }
 
     /** {@inheritDoc} */
-    @Override public void recoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+    @Override public void outRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
         assert recoveryDesc != null;
 
-        recovery = recoveryDesc;
+        outRecovery = recoveryDesc;
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public GridNioRecoveryDescriptor recoveryDescriptor() {
-        return recovery;
+    @Nullable @Override public GridNioRecoveryDescriptor outRecoveryDescriptor() {
+        return outRecovery;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void inRecoveryDescriptor(GridNioRecoveryDescriptor recoveryDesc) {
+        assert recoveryDesc != null;
+
+        inRecovery = recoveryDesc;
+    }
+
+    /** {@inheritDoc} */
+    @Nullable @Override public GridNioRecoveryDescriptor inRecoveryDescriptor() {
+        return inRecovery;
     }
 
     /** {@inheritDoc} */
     @Override public <T> T addMeta(int key, @Nullable T val) {
-        if (val instanceof GridNioRecoveryDescriptor) {
-            recovery = (GridNioRecoveryDescriptor)val;
+        if (!accepted() && val instanceof GridNioRecoveryDescriptor) {
+            outRecovery = (GridNioRecoveryDescriptor)val;
+
+            if (!outRecovery.pairedConnections())
+                inRecovery = outRecovery;
 
-            if (!accepted())
-                recovery.connected();
+            outRecovery.onConnected();
 
             return null;
         }
@@ -313,6 +417,31 @@ class GridSelectorNioSessionImpl extends GridNioSessionImpl {
     }
 
     /** {@inheritDoc} */
+    @Override public void systemMessage(Object sysMsg) {
+        this.sysMsg = sysMsg;
+    }
+
+    /**
+     * @return {@code True} if have pending system message to send.
+     */
+    boolean hasSystemMessage() {
+        return sysMsg != null;
+    }
+
+    /**
+     * Gets and clears pending system message.
+     *
+     * @return Pending system message.
+     */
+    Object systemMessage() {
+        Object ret = sysMsg;
+
+        sysMsg = null;
+
+        return ret;
+    }
+
+    /** {@inheritDoc} */
     @Override public String toString() {
         return S.toString(GridSelectorNioSessionImpl.class, this, super.toString());
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
index ebe86fb..d941bae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridShmemCommunicationClient.java
@@ -48,6 +48,7 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
     private final MessageFormatter formatter;
 
     /**
+     * @param connIdx Connection index.
      * @param metricsLsnr Metrics listener.
      * @param port Shared memory IPC server port.
      * @param connTimeout Connection timeout.
@@ -55,14 +56,16 @@ public class GridShmemCommunicationClient extends GridAbstractCommunicationClien
      * @param formatter Message formatter.
      * @throws IgniteCheckedException If failed.
      */
-    public GridShmemCommunicationClient(GridNioMetricsListener metricsLsnr,
+    public GridShmemCommunicationClient(
+        int connIdx,
+        GridNioMetricsListener metricsLsnr,
         int port,
         long connTimeout,
         IgniteLogger log,
         MessageFormatter formatter)
         throws IgniteCheckedException
     {
-        super(metricsLsnr);
+        super(connIdx, metricsLsnr);
 
         assert metricsLsnr != null;
         assert port > 0 && port < 0xffff;

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
index 5fe521d..3397772 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridTcpNioCommunicationClient.java
@@ -45,11 +45,16 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
     private final IgniteLogger log;
 
     /**
+     * @param connIdx Connection index.
      * @param ses Session.
      * @param log Logger.
      */
-    public GridTcpNioCommunicationClient(GridNioSession ses, IgniteLogger log) {
-        super(null);
+    public GridTcpNioCommunicationClient(
+        int connIdx,
+        GridNioSession ses,
+        IgniteLogger log
+    ) {
+        super(connIdx, null);
 
         assert ses != null;
         assert log != null;
@@ -104,40 +109,36 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
     }
 
     /** {@inheritDoc} */
-    @Override public boolean sendMessage(@Nullable UUID nodeId, Message msg, IgniteInClosure<IgniteException> closure)
+    @Override public boolean sendMessage(@Nullable UUID nodeId, Message msg, IgniteInClosure<IgniteException> c)
         throws IgniteCheckedException {
-        // Node ID is never provided in asynchronous send mode.
-        assert nodeId == null;
+        try {
+            // Node ID is never provided in asynchronous send mode.
+            assert nodeId == null;
 
-        if (closure != null)
-            ses.addMeta(ACK_CLOSURE.ordinal(), closure);
+            if (c != null)
+                ses.addMeta(ACK_CLOSURE.ordinal(), c);
 
-        GridNioFuture<?> fut = ses.send(msg);
+            ses.sendNoFuture(msg);
 
-        if (fut.isDone()) {
-            try {
-                fut.get();
-            }
-            catch (IgniteCheckedException e) {
-                if (closure != null)
-                    ses.removeMeta(ACK_CLOSURE.ordinal());
+            if (c != null)
+                ses.removeMeta(ACK_CLOSURE.ordinal());
+        }
+        catch (IgniteCheckedException e) {
+            if (c != null)
+                ses.removeMeta(ACK_CLOSURE.ordinal());
 
-                if (log.isDebugEnabled())
-                    log.debug("Failed to send message [client=" + this + ", err=" + e + ']');
+            if (log.isDebugEnabled())
+                log.debug("Failed to send message [client=" + this + ", err=" + e + ']');
 
-                if (e.getCause() instanceof IOException) {
-                    ses.close();
+            if (e.getCause() instanceof IOException) {
+                ses.close();
 
-                    return true;
-                }
-                else
-                    throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
+                return true;
             }
+            else
+                throw new IgniteCheckedException("Failed to send message [client=" + this + ']', e);
         }
 
-        if (closure != null)
-            ses.removeMeta(ACK_CLOSURE.ordinal());
-
         return false;
     }
 
@@ -159,4 +160,4 @@ public class GridTcpNioCommunicationClient extends GridAbstractCommunicationClie
     @Override public String toString() {
         return S.toString(GridTcpNioCommunicationClient.class, this, super.toString());
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
new file mode 100644
index 0000000..508c791
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/SessionWriteRequest.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.util.nio;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.lang.IgniteInClosure;
+
+/**
+ *
+ */
+public interface SessionWriteRequest {
+    /**
+     * Sets flag indicating that message send future was created in thread that was processing a message.
+     *
+     * @param msgThread {@code True} if future was created in thread that is processing message.
+     */
+    public void messageThread(boolean msgThread);
+
+    /**
+     * @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();
+
+    /**
+     * Sets ack closure which will be applied when ack received.
+     *
+     * @param c Ack closure.
+     */
+    public void ackClosure(IgniteInClosure<IgniteException> c);
+
+    /**
+     * The method will be called when ack received.
+     */
+    public void onAckReceived();
+
+    /**
+     * @return Ack closure.
+     */
+    public IgniteInClosure<IgniteException> ackClosure();
+
+    /**
+     * @return Session.
+     */
+    public GridNioSession session();
+
+    /**
+     * @param ses Session.
+     */
+    public void resetSession(GridNioSession ses);
+
+    /**
+     *
+     */
+    public void onError(Exception e);
+
+    /**
+     * @return Message.
+     */
+    public Object message();
+
+    /**
+     *
+     */
+    public void onMessageWritten();
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
index d6f9d10..8ed7db0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslFilter.java
@@ -282,9 +282,13 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+    @Override public GridNioFuture<?> onSessionWrite(
+        GridNioSession ses,
+        Object msg,
+        boolean fut
+    ) throws IgniteCheckedException {
         if (directMode)
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
 
         ByteBuffer input = checkMessage(ses, msg);
 
@@ -441,4 +445,4 @@ public class GridNioSslFilter extends GridNioFilterAdapter {
 
         return (ByteBuffer)msg;
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
index eb8dad4..269e8b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/ssl/GridNioSslHandler.java
@@ -437,7 +437,7 @@ class GridNioSslHandler extends ReentrantLock {
         while (!deferredWriteQueue.isEmpty()) {
             WriteRequest req = deferredWriteQueue.poll();
 
-            req.future().onDone((GridNioFuture<Object>)parent.proceedSessionWrite(ses, req.buffer()));
+            req.future().onDone((GridNioFuture<Object>)parent.proceedSessionWrite(ses, req.buffer(), true));
         }
     }
 
@@ -482,7 +482,7 @@ class GridNioSslHandler extends ReentrantLock {
 
         ByteBuffer cp = copy(outNetBuf);
 
-        return parent.proceedSessionWrite(ses, cp);
+        return parent.proceedSessionWrite(ses, cp, true);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
index b29d7cd..86aa7a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/tostring/GridToStringBuilder.java
@@ -598,4 +598,4 @@ public class GridToStringBuilder {
 
         return cd;
     }
-}
\ No newline at end of file
+}


[15/25] ignite git commit: IGNITE-3220 I/O bottleneck on server/client cluster configuration Communications optimizations: - possibility to open separate in/out connections - possibility to have multiple connections between nodes - implemented NIO sessio

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/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 b0353a6..5345a9b 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
@@ -18,8 +18,6 @@
 package org.apache.ignite.spi.communication.tcp;
 
 import org.apache.ignite.configuration.IgniteConfiguration;
-import org.apache.ignite.spi.discovery.tcp.TcpDiscoverySpi;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.TcpDiscoveryVmIpFinder;
 import org.apache.ignite.testframework.junits.spi.GridSpiAbstractConfigTest;
 import org.apache.ignite.testframework.junits.spi.GridSpiTest;
 
@@ -47,6 +45,9 @@ public class GridTcpCommunicationSpiConfigSelfTest extends GridSpiAbstractConfig
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "ackSendThreshold", 0);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "ackSendThreshold", -1);
         checkNegativeSpiProperty(new TcpCommunicationSpi(), "unacknowledgedMessagesBufferSize", -1);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionsPerNode", 0);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionsPerNode", -1);
+        checkNegativeSpiProperty(new TcpCommunicationSpi(), "connectionsPerNode", Integer.MAX_VALUE);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/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 a74a07a..0c57105 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
@@ -109,7 +109,7 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
     /**
      * @param useShmem Use shared mem.
      */
-    protected GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
+    GridTcpCommunicationSpiMultithreadedSelfTest(boolean useShmem) {
         super(false);
 
         this.useShmem = useShmem;
@@ -370,17 +370,17 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
             Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
 
             for (GridNioSession ses : sessions) {
-                final GridNioRecoveryDescriptor snd = ses.recoveryDescriptor();
+                final GridNioRecoveryDescriptor snd = ses.outRecoveryDescriptor();
 
                 if (snd != null) {
                     GridTestUtils.waitForCondition(new GridAbsPredicate() {
                         @Override public boolean apply() {
-                            return snd.messagesFutures().isEmpty();
+                            return snd.messagesRequests().isEmpty();
                         }
                     }, 10_000);
 
-                    assertEquals("Unexpected messages: " + snd.messagesFutures(), 0,
-                        snd.messagesFutures().size());
+                    assertEquals("Unexpected messages: " + snd.messagesRequests(), 0,
+                        snd.messagesRequests().size());
                 }
             }
         }
@@ -547,11 +547,18 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
         }
 
         for (CommunicationSpi spi : spis.values()) {
-            final ConcurrentMap<UUID, GridCommunicationClient> clients = U.field(spi, "clients");
+            final ConcurrentMap<UUID, GridCommunicationClient[]> clients = U.field(spi, "clients");
 
             assert GridTestUtils.waitForCondition(new PA() {
                 @Override public boolean apply() {
-                    return clients.isEmpty();
+                    for (GridCommunicationClient[] clients0 : clients.values()) {
+                        for (GridCommunicationClient client : clients0) {
+                            if (client != null)
+                                return false;
+                        }
+                    }
+
+                    return true;
                 }
             }, getTestTimeout()) : "Clients: " + clients;
         }
@@ -583,4 +590,4 @@ public class GridTcpCommunicationSpiMultithreadedSelfTest extends GridSpiAbstrac
         spis.clear();
         nodes.clear();
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/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 34872c6..12c2edb 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
@@ -173,7 +173,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
                     boolean found = false;
 
                     for (GridNioSession ses : sessions) {
-                        final GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+                        final GridNioRecoveryDescriptor recoveryDesc = ses.outRecoveryDescriptor();
 
                         if (recoveryDesc != null) {
                             found = true;
@@ -188,12 +188,12 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
 
                             GridTestUtils.waitForCondition(new GridAbsPredicate() {
                                 @Override public boolean apply() {
-                                    return recoveryDesc.messagesFutures().isEmpty();
+                                    return recoveryDesc.messagesRequests().isEmpty();
                                 }
                             }, 10_000);
 
-                            assertEquals("Unexpected messages: " + recoveryDesc.messagesFutures(), 0,
-                                recoveryDesc.messagesFutures().size());
+                            assertEquals("Unexpected messages: " + recoveryDesc.messagesRequests(), 0,
+                                recoveryDesc.messagesRequests().size());
 
                             break;
                         }
@@ -361,6 +361,7 @@ public class GridTcpCommunicationSpiRecoveryAckSelfTest<T extends CommunicationS
         spi.setAckSendThreshold(ackCnt);
         spi.setMessageQueueLimit(queueLimit);
         spi.setSharedMemoryPort(-1);
+        spi.setConnectionsPerNode(1);
 
         return spi;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
index 95c9e40..b1aa119 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest.java
@@ -33,6 +33,7 @@ public class GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest extends Gri
         spi.setAckSendThreshold(5);
         spi.setSocketSendBuffer(512);
         spi.setSocketReceiveBuffer(512);
+        spi.setConnectionsPerNode(1);
 
         return spi;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java
new file mode 100644
index 0000000..8e43937
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.java
@@ -0,0 +1,28 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.spi.communication.tcp;
+
+/**
+ *
+ */
+public class GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest extends GridTcpCommunicationSpiRecoverySelfTest {
+    /** {@inheritDoc} */
+    @Override protected boolean usePairedConnections() {
+        return false;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/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 3234d74..065a3d7 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
@@ -56,6 +56,7 @@ import org.eclipse.jetty.util.ConcurrentHashSet;
 /**
  *
  */
+@SuppressWarnings("unchecked")
 @GridSpiTest(spi = TcpCommunicationSpi.class, group = "Communication SPI")
 public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi> extends GridSpiAbstractTest<T> {
     /** */
@@ -305,14 +306,14 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
                 log.info("Iteration: " + i);
 
                 try {
-                    final GridNioSession ses0 = communicationSession(spi0);
-                    final GridNioSession ses1 = communicationSession(spi1);
+                    final GridNioSession ses0 = communicationSession(spi0, false);
+                    final GridNioSession ses1 = communicationSession(spi1, true);
 
                     ses1.pauseReads().get();
 
                     IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
                         @Override public Void call() throws Exception {
-                            for (int i = 0; i < 5000; i++) {
+                            for (int i = 0; i < 6000; i++) {
                                 spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
 
                                 sentCnt.incrementAndGet();
@@ -331,7 +332,12 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
 
-                    ses1.resumeReads().get();
+                    try {
+                        ses1.resumeReads().get();
+                    }
+                    catch (IgniteCheckedException ignore) {
+                        // Can fail is ses1 was closed.
+                    }
 
                     for (int j = 0; j < 100; j++) {
                         spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
@@ -415,14 +421,14 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
                 log.info("Iteration: " + i);
 
                 try {
-                    final GridNioSession ses0 = communicationSession(spi0);
-                    final GridNioSession ses1 = communicationSession(spi1);
+                    final GridNioSession ses0 = communicationSession(spi0, false);
+                    final GridNioSession ses1 = communicationSession(spi1, true);
 
                     ses1.pauseReads().get();
 
                     IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
                         @Override public Void call() throws Exception {
-                            for (int i = 0; i < 5000; i++) {
+                            for (int i = 0; i < 6000; i++) {
                                 spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
 
                                 expCnt1.incrementAndGet();
@@ -441,12 +447,16 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
 
-                    ses1.resumeReads().get();
+                    try {
+                        ses1.resumeReads().get();
+                    }
+                    catch (IgniteCheckedException ignore) {
+                        // Can fail is ses1 was closed.
+                    }
 
                     // Wait when session is closed, then try to open new connection from node1.
                     GridTestUtils.waitForCondition(new GridAbsPredicate() {
-                        @Override
-                        public boolean apply() {
+                        @Override public boolean apply() {
                             return ses1.closeTime() != 0;
                         }
                     }, awaitForSocketWriteTimeout());
@@ -532,14 +542,14 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
                 log.info("Iteration: " + i);
 
                 try {
-                    final GridNioSession ses0 = communicationSession(spi0);
-                    final GridNioSession ses1 = communicationSession(spi1);
+                    final GridNioSession ses0 = communicationSession(spi0, false);
+                    final GridNioSession ses1 = communicationSession(spi1, true);
 
                     ses1.pauseReads().get();
 
                     IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
                         @Override public Void call() throws Exception {
-                            for (int i = 0; i < 5000; i++) {
+                            for (int i = 0; i < 6000; i++) {
                                 spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
 
                                 sentCnt.incrementAndGet();
@@ -558,7 +568,12 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     assertTrue("Failed to wait for session close", ses0.closeTime() != 0);
 
-                    ses1.resumeReads().get();
+                    try {
+                        ses1.resumeReads().get();
+                    }
+                    catch (IgniteCheckedException ignore) {
+                        // Can fail is ses1 was closed.
+                    }
 
                     sndFut.get();
 
@@ -605,11 +620,12 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
     /**
      * @param spi SPI.
+     * @param in {@code True} if need find inbound session.
      * @return Session.
      * @throws Exception If failed.
      */
     @SuppressWarnings("unchecked")
-    private GridNioSession communicationSession(TcpCommunicationSpi spi) throws Exception {
+    private GridNioSession communicationSession(TcpCommunicationSpi spi, boolean in) throws Exception {
         final GridNioServer srv = U.field(spi, "nioSrvr");
 
         GridTestUtils.waitForCondition(new GridAbsPredicate() {
@@ -622,9 +638,21 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
         Collection<? extends GridNioSession> sessions = GridTestUtils.getFieldValue(srv, "sessions");
 
-        assertEquals(1, sessions.size());
+        for (GridNioSession ses : sessions) {
+            if (in == ses.accepted())
+                return ses;
+        }
+
+        fail("Failed to find session");
 
-        return sessions.iterator().next();
+        return null;
+    }
+
+    /**
+     * @return {@code True}.
+     */
+    protected boolean usePairedConnections() {
+        return true;
     }
 
     /**
@@ -642,6 +670,8 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
         spi.setSocketWriteTimeout(1000);
         spi.setSocketSendBuffer(512);
         spi.setSocketReceiveBuffer(512);
+        spi.setConnectionsPerNode(1);
+        spi.setUsePairedConnections(usePairedConnections());
 
         return spi;
     }
@@ -748,9 +778,8 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
             spi.spiStop();
         }
 
-        for (IgniteTestResources rsrcs : spiRsrcs) {
+        for (IgniteTestResources rsrcs : spiRsrcs)
             rsrcs.stopThreads();
-        }
 
         spis.clear();
         nodes.clear();

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
index 25e3611..c4930a0 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/IgniteTcpCommunicationRecoveryAckClosureSelfTest.java
@@ -187,7 +187,7 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
                     boolean found = false;
 
                     for (GridNioSession ses : sessions) {
-                        final GridNioRecoveryDescriptor recoveryDesc = ses.recoveryDescriptor();
+                        final GridNioRecoveryDescriptor recoveryDesc = ses.outRecoveryDescriptor();
 
                         if (recoveryDesc != null) {
                             found = true;
@@ -202,12 +202,12 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
 
                             GridTestUtils.waitForCondition(new GridAbsPredicate() {
                                 @Override public boolean apply() {
-                                    return recoveryDesc.messagesFutures().isEmpty();
+                                    return recoveryDesc.messagesRequests().isEmpty();
                                 }
                             }, 10_000);
 
-                            assertEquals("Unexpected messages: " + recoveryDesc.messagesFutures(), 0,
-                                recoveryDesc.messagesFutures().size());
+                            assertEquals("Unexpected messages: " + recoveryDesc.messagesRequests(), 0,
+                                recoveryDesc.messagesRequests().size());
 
                             break;
                         }
@@ -397,6 +397,7 @@ public class IgniteTcpCommunicationRecoveryAckClosureSelfTest<T extends Communic
         spi.setAckSendThreshold(ackCnt);
         spi.setMessageQueueLimit(queueLimit);
         spi.setSharedMemoryPort(-1);
+        spi.setConnectionsPerNode(1);
 
         return spi;
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index f9e2ff4..143159d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -63,7 +63,9 @@ public class GridTestKernalContext extends GridKernalContextImpl {
                 null,
                 null,
                 null,
-                U.allPluginProviders());
+                null,
+                U.allPluginProviders()
+        );
 
         GridTestUtils.setFieldValue(grid(), "cfg", config());
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
index 14e5833..17757ab 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheFullApiSelfTestSuite.java
@@ -57,6 +57,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAto
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderFairAffinityMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderMultiNodeP2PDisabledFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOffHeapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWriteOrderOffHeapTieredFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCacheAtomicPrimaryWrityOrderOffHeapMultiNodeFullApiSelfTest;
@@ -77,6 +78,7 @@ import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePar
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedMultiNodeP2PDisabledFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.CachePartitionedNearEnabledMultiNodeLongTxTimeoutFullApiTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNearOnlyNoPrimaryFullApiSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapMultiNodeFullApiSelfTest;
 import org.apache.ignite.internal.processors.cache.distributed.near.GridCachePartitionedOffHeapTieredFullApiSelfTest;
@@ -228,6 +230,10 @@ public class IgniteCacheFullApiSelfTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheReplicatedFullApiMultithreadedSelfTest.class);
         suite.addTestSuite(GridCachePartitionedFullApiMultithreadedSelfTest.class);
 
+        // Disabled striped pool.
+        suite.addTestSuite(GridCacheAtomicPrimaryWriteOrderNoStripedPoolMultiNodeFullApiSelfTest.class);
+        suite.addTestSuite(GridCachePartitionedNoStripedPoolMultiNodeFullApiSelfTest.class);
+
         // Other.
         suite.addTestSuite(GridCacheClearSelfTest.class);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
index 435fcfb..554bb3d 100755
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteCacheTestSuite.java
@@ -39,6 +39,10 @@ import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreOptimizedMarshallerW
 import org.apache.ignite.cache.store.jdbc.CacheJdbcPojoStoreTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreMultithreadedSelfTest;
 import org.apache.ignite.cache.store.jdbc.GridCacheJdbcBlobStoreSelfTest;
+import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceMultipleConnectionsTest;
+import org.apache.ignite.internal.managers.communication.IgniteCommunicationBalanceTest;
+import org.apache.ignite.internal.managers.communication.IgniteIoTestMessagesTest;
+import org.apache.ignite.internal.managers.communication.IgniteVariousConnectionNumberTest;
 import org.apache.ignite.internal.processors.cache.CacheAffinityCallSelfTest;
 import org.apache.ignite.internal.processors.cache.CacheDeferredDeleteSanitySelfTest;
 import org.apache.ignite.internal.processors.cache.CacheEntryProcessorCopySelfTest;
@@ -128,7 +132,10 @@ import org.apache.ignite.internal.processors.cache.context.IgniteCacheTxExecutio
 import org.apache.ignite.internal.processors.cache.distributed.CacheAtomicNearUpdateTopologyChangeTest;
 import org.apache.ignite.internal.processors.cache.distributed.CacheTxNearUpdateTopologyChangeTest;
 import org.apache.ignite.internal.processors.cache.distributed.GridCacheEntrySetIterationPreloadingSelfTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecovery10ConnectionsTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheAtomicMessageRecoveryTest;
+import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheConnectionRecovery10ConnectionsTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheConnectionRecoveryTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheMessageRecoveryIdleConnectionTest;
 import org.apache.ignite.internal.processors.cache.distributed.IgniteCacheMessageWriteTimeoutTest;
@@ -259,7 +266,7 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheStoreValueBytesSelfTest.class);
         GridTestUtils.addTestIfNeeded(suite, DataStreamProcessorSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, DataStreamerUpdateAfterLoadTest.class, ignoredTests);
-            suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
+        suite.addTestSuite(DataStreamerMultiThreadedSelfTest.class);
         suite.addTestSuite(DataStreamerMultinodeCreateCacheTest.class);
         suite.addTestSuite(DataStreamerImplSelfTest.class);
         suite.addTestSuite(DataStreamerTimeoutTest.class);
@@ -292,10 +299,13 @@ public class IgniteCacheTestSuite extends TestSuite {
         suite.addTestSuite(GridCacheEntrySetIterationPreloadingSelfTest.class);
         suite.addTestSuite(GridCacheMixedPartitionExchangeSelfTest.class);
         suite.addTestSuite(IgniteCacheAtomicMessageRecoveryTest.class);
+        suite.addTestSuite(IgniteCacheAtomicMessageRecoveryNoPairedConnectionsTest.class);
+        suite.addTestSuite(IgniteCacheAtomicMessageRecovery10ConnectionsTest.class);
         suite.addTestSuite(IgniteCacheTxMessageRecoveryTest.class);
         suite.addTestSuite(IgniteCacheMessageWriteTimeoutTest.class);
         suite.addTestSuite(IgniteCacheMessageRecoveryIdleConnectionTest.class);
         suite.addTestSuite(IgniteCacheConnectionRecoveryTest.class);
+        suite.addTestSuite(IgniteCacheConnectionRecovery10ConnectionsTest.class);
         GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionAtomicSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredEvictionSelfTest.class, ignoredTests);
         GridTestUtils.addTestIfNeeded(suite, GridCacheOffHeapTieredAtomicSelfTest.class, ignoredTests);
@@ -325,6 +335,11 @@ public class IgniteCacheTestSuite extends TestSuite {
 
         suite.addTestSuite(CacheTxFastFinishTest.class);
 
+        suite.addTestSuite(IgniteVariousConnectionNumberTest.class);
+        suite.addTestSuite(IgniteCommunicationBalanceTest.class);
+        suite.addTestSuite(IgniteCommunicationBalanceMultipleConnectionsTest.class);
+        suite.addTestSuite(IgniteIoTestMessagesTest.class);
+
         return suite;
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
index c557fbb..11fcfda 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteSpiCommunicationSelfTestSuite.java
@@ -25,6 +25,7 @@ import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiMultithrea
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiMultithreadedShmemTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryAckSelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryFailureDetectionSelfTest;
+import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiRecoverySslSelfTest;
 import org.apache.ignite.spi.communication.tcp.GridTcpCommunicationSpiShmemSelfTest;
@@ -50,6 +51,7 @@ public class IgniteSpiCommunicationSelfTestSuite extends TestSuite {
         suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoveryAckSelfTest.class));
         suite.addTest(new TestSuite(IgniteTcpCommunicationRecoveryAckClosureSelfTest.class));
         suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoverySelfTest.class));
+        suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoveryNoPairedConnectionsTest.class));
         suite.addTest(new TestSuite(GridTcpCommunicationSpiRecoverySslSelfTest.class));
 
         suite.addTest(new TestSuite(GridTcpCommunicationSpiConcurrentConnectSelfTest.class));

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
index a725534..9542372 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/HadoopJobTracker.java
@@ -240,7 +240,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
                     try {
                         // Must process query callback in a separate thread to avoid deadlocks.
-                        evtProcSvc.submit(new EventHandler() {
+                        evtProcSvc.execute(new EventHandler() {
                             @Override protected void body() throws IgniteCheckedException {
                                 processJobMetadataUpdates(evts);
                             }
@@ -264,7 +264,7 @@ public class HadoopJobTracker extends HadoopComponent {
 
                 try {
                     // Must process discovery callback in a separate thread to avoid deadlock.
-                    evtProcSvc.submit(new EventHandler() {
+                    evtProcSvc.execute(new EventHandler() {
                         @Override protected void body() {
                             processNodeLeft((DiscoveryEvent)evt);
                         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
index bc047e7..ff58509 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopExternalCommunication.java
@@ -639,6 +639,7 @@ public class HadoopExternalCommunication {
                         .logger(log.getLogger(GridNioServer.class))
                         .selectorCount(selectorsCnt)
                         .gridName(gridName)
+                        .serverName("hadoop")
                         .tcpNoDelay(tcpNoDelay)
                         .directBuffer(directBuf)
                         .byteOrder(ByteOrder.nativeOrder())
@@ -1305,11 +1306,11 @@ public class HadoopExternalCommunication {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
             if (ses.meta(PROCESS_META) == null && !(msg instanceof ProcessHandshakeMessage))
                 log.warning("Writing message before handshake has finished [ses=" + ses + ", msg=" + msg + ']');
 
-            return proceedSessionWrite(ses, msg);
+            return proceedSessionWrite(ses, msg, fut);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
index a8de999..3f33fb7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopIpcToNioAdapter.java
@@ -190,7 +190,7 @@ public class HadoopIpcToNioAdapter<T> {
         }
 
         /** {@inheritDoc} */
-        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) {
+        @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) {
             assert ses == HadoopIpcToNioAdapter.this.ses : "ses=" + ses +
                 ", this.ses=" + HadoopIpcToNioAdapter.this.ses;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
index eeca564..24bba88 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/communication/HadoopMarshallerFilter.java
@@ -57,10 +57,10 @@ public class HadoopMarshallerFilter extends GridNioFilterAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg) throws IgniteCheckedException {
+    @Override public GridNioFuture<?> onSessionWrite(GridNioSession ses, Object msg, boolean fut) throws IgniteCheckedException {
         assert msg instanceof HadoopMessage : "Invalid message type: " + msg;
 
-        return proceedSessionWrite(ses, U.marshal(marsh, msg));
+        return proceedSessionWrite(ses, U.marshal(marsh, msg), fut);
     }
 
     @Override public void onMessageReceived(GridNioSession ses, Object msg) throws IgniteCheckedException {
@@ -84,4 +84,4 @@ public class HadoopMarshallerFilter extends GridNioFilterAdapter {
     @Override public void onSessionWriteTimeout(GridNioSession ses) throws IgniteCheckedException {
         proceedSessionWriteTimeout(ses);
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
----------------------------------------------------------------------
diff --git a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
index 5767790..73820b7 100644
--- a/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
+++ b/modules/kafka/src/main/java/org/apache/ignite/stream/kafka/KafkaStreamer.java
@@ -143,7 +143,7 @@ public class KafkaStreamer<K, V> extends StreamAdapter<MessageAndMetadata<byte[]
 
         // Now create an object to consume the messages.
         for (final KafkaStream<byte[], byte[]> stream : streams) {
-            executor.submit(new Runnable() {
+            executor.execute(new Runnable() {
                 @Override public void run() {
                     while (!stopped) {
                         try {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
----------------------------------------------------------------------
diff --git a/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java b/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
index 0b10a75..369e318 100644
--- a/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
+++ b/modules/tools/src/main/java/org/apache/ignite/tools/classgen/ClassesGenerator.java
@@ -122,7 +122,11 @@ public class ClassesGenerator {
             for (String err : errs)
                 sb.append("    ").append(err).append('\n');
 
-            throw new Exception(sb.toString().trim());
+            String msg = sb.toString().trim();
+
+            System.out.println(msg);
+
+            throw new Exception(msg);
         }
 
         PrintStream out = new PrintStream(new File(basePath,
@@ -246,4 +250,4 @@ public class ClassesGenerator {
             }
         }
     }
-}
\ No newline at end of file
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
index 54ef00d..07549d5 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/IgniteBenchmarkUtils.java
@@ -30,7 +30,7 @@ import org.apache.ignite.transactions.TransactionConcurrency;
 import org.apache.ignite.transactions.TransactionIsolation;
 import org.apache.ignite.transactions.TransactionOptimisticException;
 import org.apache.ignite.transactions.TransactionRollbackException;
-import org.apache.ignite.yardstick.cache.IgniteSqlQueryBenchmark;
+import org.apache.ignite.yardstick.cache.IgnitePutBenchmark;
 import org.yardstickframework.BenchmarkDriver;
 import org.yardstickframework.BenchmarkDriverStartUp;
 
@@ -89,13 +89,13 @@ public class IgniteBenchmarkUtils {
     public static void main(String[] args) throws Exception {
         final String cfg = "modules/yardstick/config/ignite-localhost-config.xml";
 
-        final Class<? extends BenchmarkDriver> benchmark = IgniteSqlQueryBenchmark.class;
+        final Class<? extends BenchmarkDriver> benchmark = IgnitePutBenchmark.class;
 
         final int threads = 1;
 
         final boolean clientDriverNode = true;
 
-        final int extraNodes = 2;
+        final int extraNodes = 4;
 
         final int warmUp = 5;
         final int duration = 5;

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
index a25f975..c8022e2 100644
--- a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/CacheEntryEventProbe.java
@@ -72,7 +72,7 @@ public class CacheEntryEventProbe implements BenchmarkProbe {
 
                 buildingService = Executors.newSingleThreadExecutor();
 
-                buildingService.submit(new Runnable() {
+                buildingService.execute(new Runnable() {
                     @Override public void run() {
                         try {
                             while (!Thread.currentThread().isInterrupted()) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
new file mode 100644
index 0000000..bee45e0
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/cache/IgniteIoTestBenchmark.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.cache;
+
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+/**
+ *
+ */
+public class IgniteIoTestBenchmark extends IgniteAbstractBenchmark {
+    /** */
+    private List<ClusterNode> targetNodes;
+
+    /** */
+    private IgniteKernal ignite;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        ignite = (IgniteKernal)ignite();
+
+        targetNodes = new ArrayList<>();
+
+        ClusterNode loc = ignite().cluster().localNode();
+
+        Collection<ClusterNode> nodes = ignite().cluster().forServers().nodes();
+
+        for (ClusterNode node : nodes) {
+            if (!loc.equals(node))
+                targetNodes.add(node);
+        }
+
+        if (targetNodes.isEmpty())
+            throw new IgniteException("Failed to find remote server nodes [nodes=" + nodes + ']');
+
+        BenchmarkUtils.println(cfg, "Initialized target nodes: " + targetNodes + ']');
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ClusterNode node = targetNodes.get(nextRandom(targetNodes.size()));
+
+        ignite.sendIoTest(node, null, false).get();
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java
new file mode 100644
index 0000000..8791c83
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestAbstractBenchmark.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.io;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import org.apache.ignite.IgniteException;
+import org.apache.ignite.cluster.ClusterNode;
+import org.apache.ignite.internal.IgniteKernal;
+import org.apache.ignite.internal.util.typedef.F;
+import org.apache.ignite.yardstick.IgniteAbstractBenchmark;
+import org.yardstickframework.BenchmarkConfiguration;
+import org.yardstickframework.BenchmarkUtils;
+
+/**
+ *
+ */
+public abstract class IgniteIoTestAbstractBenchmark extends IgniteAbstractBenchmark {
+    /** */
+    protected final List<ClusterNode> targetNodes = new ArrayList<>();
+
+    /** */
+    protected IgniteKernal ignite;
+
+    /** {@inheritDoc} */
+    @Override public void setUp(BenchmarkConfiguration cfg) throws Exception {
+        super.setUp(cfg);
+
+        ignite = (IgniteKernal)ignite();
+
+        ClusterNode loc = ignite().cluster().localNode();
+
+        Collection<ClusterNode> nodes = ignite().cluster().forServers().nodes();
+
+        for (ClusterNode node : nodes) {
+            if (!loc.equals(node))
+                targetNodes.add(node);
+        }
+
+        if (targetNodes.isEmpty())
+            throw new IgniteException("Failed to find remote server nodes [nodes=" + nodes + ']');
+
+        BenchmarkUtils.println(cfg, "Initialized target nodes: " + F.nodeIds(targetNodes) + ']');
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java
new file mode 100644
index 0000000..9011910
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendAllBenchmark.java
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.io;
+
+import java.util.Map;
+
+/**
+ *
+ */
+public class IgniteIoTestSendAllBenchmark extends IgniteIoTestAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ignite.sendIoTest(targetNodes, null, false).get();
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/10ade284/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java
new file mode 100644
index 0000000..88368e0
--- /dev/null
+++ b/modules/yardstick/src/main/java/org/apache/ignite/yardstick/io/IgniteIoTestSendRandomBenchmark.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.yardstick.io;
+
+import java.util.Map;
+import org.apache.ignite.cluster.ClusterNode;
+
+/**
+ *
+ */
+public class IgniteIoTestSendRandomBenchmark extends IgniteIoTestAbstractBenchmark {
+    /** {@inheritDoc} */
+    @Override public boolean test(Map<Object, Object> ctx) throws Exception {
+        ClusterNode node = targetNodes.get(nextRandom(targetNodes.size()));
+
+        ignite.sendIoTest(node, null, false).get();
+
+        return true;
+    }
+}


[09/25] ignite git commit: IGNITE-4027 Extract PlatformTargetProxy interface

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
index 8ff15d5..fd1c2d4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
@@ -29,6 +29,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
 import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
 import org.apache.ignite.internal.processors.platform.utils.PlatformListenable;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
@@ -98,7 +99,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader)
+    @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_UNICAST:
@@ -120,7 +121,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
             }
 
             case OP_EXEC_ASYNC:
-                return executeJavaTask(reader, true);
+                return wrapListenable((PlatformListenable) executeJavaTask(reader, true));
 
             default:
                 return super.processInStreamOutObject(type, reader);
@@ -128,7 +129,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_WITH_TIMEOUT: {
                 compute.withTimeout(val);
@@ -154,7 +155,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
      * @param reader Reader.
      * @param broadcast broadcast flag.
      */
-    private PlatformListenable processClosures(long taskPtr, BinaryRawReaderEx reader, boolean broadcast,
+    private PlatformTarget processClosures(long taskPtr, BinaryRawReaderEx reader, boolean broadcast,
         boolean affinity) {
         PlatformAbstractTask task;
 
@@ -221,7 +222,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_EXEC:
@@ -239,7 +240,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
      *
      * @param task Task.
      */
-    private PlatformListenable executeNative0(final PlatformAbstractTask task) {
+    private PlatformTarget executeNative0(final PlatformAbstractTask task) {
         IgniteInternalFuture fut = computeForPlatform.executeAsync(task, null);
 
         fut.listen(new IgniteInClosure<IgniteInternalFuture>() {
@@ -257,7 +258,7 @@ public class PlatformCompute extends PlatformAbstractTarget {
             }
         });
 
-        return PlatformFutureUtils.getListenable(fut);
+        return wrapListenable(PlatformFutureUtils.getListenable(fut));
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
index cd5fba0..7d71a9e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformDataStreamer.java
@@ -114,7 +114,7 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc}  */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_UPDATE:
                 int plc = reader.readInt();
@@ -169,7 +169,7 @@ public class PlatformDataStreamer extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc}  */
-    @Override protected long processInLongOutLong(int type, final long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, final long val) throws IgniteCheckedException {
         switch (type) {
             case OP_SET_ALLOW_OVERWRITE:
                 ldr.allowOverwrite(val == TRUE);

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
index add11ed..d0992fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastreamer/PlatformStreamReceiverImpl.java
@@ -23,6 +23,8 @@ import org.apache.ignite.IgniteException;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractPredicate;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxy;
+import org.apache.ignite.internal.processors.platform.PlatformTargetProxyImpl;
 import org.apache.ignite.internal.processors.platform.cache.PlatformCache;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
@@ -89,8 +91,10 @@ public class PlatformStreamReceiverImpl extends PlatformAbstractPredicate implem
 
             out.synchronize();
 
-            ctx.gateway().dataStreamerStreamReceiverInvoke(ptr, new PlatformCache(ctx, cache, keepBinary),
-                mem.pointer(), keepBinary);
+            PlatformCache cache0 = new PlatformCache(ctx, cache, keepBinary);
+            PlatformTargetProxy cacheProxy = new PlatformTargetProxyImpl(cache0, ctx);
+
+            ctx.gateway().dataStreamerStreamReceiverInvoke(ptr, cacheProxy, mem.pointer(), keepBinary);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java
index 811e38b..b57b140 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicLong.java
@@ -80,7 +80,7 @@ public class PlatformAtomicLong extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_COMPARE_AND_SET:
                 long cmp = reader.readLong();
@@ -99,7 +99,7 @@ public class PlatformAtomicLong extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_ADD_AND_GET:
                 return atomicLong.addAndGet(val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
index 63b5b86..a644259 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicReference.java
@@ -94,7 +94,7 @@ public class PlatformAtomicReference extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         if (type == OP_GET)
             writer.writeObject(atomicRef.get());
         else
@@ -102,7 +102,7 @@ public class PlatformAtomicReference extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader)
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader)
         throws IgniteCheckedException {
         if (type == OP_SET) {
             atomicRef.set(reader.readObjectDetached());
@@ -114,7 +114,7 @@ public class PlatformAtomicReference extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader,
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader,
         BinaryRawWriterEx writer) throws IgniteCheckedException {
         if (type == OP_COMPARE_AND_SET_AND_GET) {
             Object val = reader.readObjectDetached();
@@ -134,7 +134,7 @@ public class PlatformAtomicReference extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_CLOSE:
                 atomicRef.close();

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
index c352731..6d17a72 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/datastructures/PlatformAtomicSequence.java
@@ -71,7 +71,7 @@ public class PlatformAtomicSequence extends PlatformAbstractTarget {
 
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_ADD_AND_GET:
                 return atomicSeq.addAndGet(val);

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
index 383e7ab..9ddcc37 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/events/PlatformEvents.java
@@ -27,6 +27,7 @@ import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
 import org.apache.ignite.internal.processors.platform.PlatformEventFilterListener;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
 import org.apache.ignite.internal.processors.platform.utils.PlatformFutureUtils;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
 import org.apache.ignite.internal.util.typedef.F;
@@ -117,7 +118,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader)
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_RECORD_LOCAL:
@@ -168,7 +169,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
-    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_LOCAL_QUERY: {
@@ -271,7 +272,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_GET_ENABLED_EVENTS:
                 writeEventTypes(events.enabledEvents(), writer);
@@ -284,7 +285,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+    @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
         switch (type) {
             case OP_WITH_ASYNC:
                 if (events.isAsync())
@@ -297,7 +298,7 @@ public class PlatformEvents extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_IS_ENABLED:
                 return events.isEnabled((int)val) ? TRUE : FALSE;
@@ -310,12 +311,12 @@ public class PlatformEvents extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+    @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
         return ((IgniteFutureImpl)eventsAsync.future()).internalFuture();
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override protected PlatformFutureUtils.Writer futureWriter(int opId) {
+    @Nullable @Override public PlatformFutureUtils.Writer futureWriter(int opId) {
         switch (opId) {
             case OP_WAIT_FOR_LOCAL:
                 return eventResWriter;

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
index 216427a..6fe109e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/messaging/PlatformMessaging.java
@@ -24,6 +24,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
 import org.apache.ignite.internal.processors.platform.message.PlatformMessageFilter;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
 import org.apache.ignite.internal.util.future.IgniteFutureImpl;
@@ -86,7 +87,7 @@ public class PlatformMessaging extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader)
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_SEND:
@@ -149,7 +150,7 @@ public class PlatformMessaging extends PlatformAbstractTarget {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"IfMayBeConditional", "ConstantConditions", "unchecked"})
-    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_REMOTE_LISTEN:{
@@ -181,12 +182,12 @@ public class PlatformMessaging extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+    @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
         return ((IgniteFutureImpl)messagingAsync.future()).internalFuture();
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+    @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
         switch (type) {
             case OP_WITH_ASYNC:
                 if (messaging.isAsync())

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
index 962a4c0..22a7fa2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/services/PlatformServices.java
@@ -25,6 +25,7 @@ import org.apache.ignite.internal.binary.BinaryRawReaderEx;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
 import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetService;
 import org.apache.ignite.internal.processors.platform.dotnet.PlatformDotNetServiceImpl;
 import org.apache.ignite.internal.processors.platform.utils.PlatformUtils;
@@ -144,7 +145,7 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader)
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader)
         throws IgniteCheckedException {
         switch (type) {
             case OP_DOTNET_DEPLOY: {
@@ -195,7 +196,7 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer)
         throws IgniteCheckedException {
         switch (type) {
             case OP_DOTNET_SERVICES: {
@@ -223,8 +224,8 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInObjectStreamOutObjectStream(int type, Object arg, BinaryRawReaderEx reader,
-        BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public PlatformTarget processInObjectStreamOutObjectStream(int type, PlatformTarget arg,
+        BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_INVOKE: {
                 assert arg != null;
@@ -260,7 +261,7 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_DESCRIPTORS: {
                 Collection<ServiceDescriptor> descs = services.serviceDescriptors();
@@ -299,7 +300,7 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processOutObject(int type) throws IgniteCheckedException {
+    @Override public PlatformTarget processOutObject(int type) throws IgniteCheckedException {
         switch (type) {
             case OP_WITH_ASYNC:
                 if (services.isAsync())
@@ -315,7 +316,7 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_CANCEL_ALL:
                 services.cancelAll();
@@ -327,7 +328,7 @@ public class PlatformServices extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected Object processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    @Override public PlatformTarget processInStreamOutObject(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
         switch (type) {
             case OP_SERVICE_PROXY: {
                 String name = reader.readString();
@@ -343,14 +344,14 @@ public class PlatformServices extends PlatformAbstractTarget {
                     : new GridServiceProxy<>(services.clusterGroup(), name, Service.class, sticky,
                         platformCtx.kernalContext());
 
-                return new ServiceProxyHolder(proxy, d.serviceClass());
+                return new ServiceProxyHolder(proxy, d.serviceClass(), platformContext());
             }
         }
         return super.processInStreamOutObject(type, reader);
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteInternalFuture currentFuture() throws IgniteCheckedException {
+    @Override public IgniteInternalFuture currentFuture() throws IgniteCheckedException {
         return ((IgniteFutureImpl)servicesAsync.future()).internalFuture();
     }
 
@@ -392,7 +393,7 @@ public class PlatformServices extends PlatformAbstractTarget {
      * Proxy holder.
      */
     @SuppressWarnings("unchecked")
-    private static class ServiceProxyHolder {
+    private static class ServiceProxyHolder extends PlatformAbstractTarget {
         /** */
         private final Object proxy;
 
@@ -422,7 +423,9 @@ public class PlatformServices extends PlatformAbstractTarget {
          * @param proxy Proxy object.
          * @param clazz Proxy class.
          */
-        private ServiceProxyHolder(Object proxy, Class clazz) {
+        private ServiceProxyHolder(Object proxy, Class clazz, PlatformContext ctx) {
+            super(ctx);
+
             assert proxy != null;
             assert clazz != null;
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
index 1b41712..3cee2b1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/transactions/PlatformTransactions.java
@@ -153,7 +153,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
         switch (type) {
             case OP_COMMIT:
                 tx(val).commit();
@@ -184,7 +184,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
+    @Override public long processInStreamOutLong(int type, BinaryRawReaderEx reader) throws IgniteCheckedException {
         long txId = reader.readLong();
 
         final Transaction asyncTx = (Transaction)tx(txId).withAsync();
@@ -220,7 +220,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processInStreamOutStream(int type, BinaryRawReaderEx reader, BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_START: {
                 TransactionConcurrency txConcurrency = TransactionConcurrency.fromOrdinal(reader.readInt());
@@ -245,7 +245,7 @@ public class PlatformTransactions extends PlatformAbstractTarget {
     }
 
     /** {@inheritDoc} */
-    @Override protected void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
+    @Override public void processOutStream(int type, BinaryRawWriterEx writer) throws IgniteCheckedException {
         switch (type) {
             case OP_CACHE_CONFIG_PARAMETERS:
                 TransactionConfiguration txCfg = platformCtx.kernalContext().config().getTransactionConfiguration();

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
index 5985d22..e81f4c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformFutureUtils.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.platform.utils;
 import org.apache.ignite.IgniteCheckedException;
 import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.binary.BinaryRawWriterEx;
-import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
 import org.apache.ignite.internal.processors.platform.PlatformContext;
+import org.apache.ignite.internal.processors.platform.PlatformTarget;
 import org.apache.ignite.internal.processors.platform.callback.PlatformCallbackGateway;
 import org.apache.ignite.internal.processors.platform.memory.PlatformMemory;
 import org.apache.ignite.internal.processors.platform.memory.PlatformOutputStream;
@@ -71,7 +71,7 @@ public class PlatformFutureUtils {
      * @return Resulting listenable.
      */
     public static PlatformListenable listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr,
-        final int typ, PlatformAbstractTarget target) {
+        final int typ, PlatformTarget target) {
         PlatformListenable listenable = getListenable(fut);
 
         listen(ctx, listenable, futPtr, typ, null, target);
@@ -88,7 +88,7 @@ public class PlatformFutureUtils {
      * @return Resulting listenable.
      */
     public static PlatformListenable listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr,
-        final int typ, PlatformAbstractTarget target) {
+        final int typ, PlatformTarget target) {
         PlatformListenable listenable = getListenable(fut);
 
         listen(ctx, listenable, futPtr, typ, null, target);
@@ -107,7 +107,7 @@ public class PlatformFutureUtils {
      * @return Resulting listenable.
      */
     public static PlatformListenable listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr,
-        final int typ, Writer writer, PlatformAbstractTarget target) {
+        final int typ, Writer writer, PlatformTarget target) {
         PlatformListenable listenable = getListenable(fut);
 
         listen(ctx, listenable, futPtr, typ, writer, target);
@@ -126,7 +126,7 @@ public class PlatformFutureUtils {
      * @return Resulting listenable.
      */
     public static PlatformListenable listen(final PlatformContext ctx, IgniteFuture fut, final long futPtr,
-        final int typ, Writer writer, PlatformAbstractTarget target) {
+        final int typ, Writer writer, PlatformTarget target) {
         PlatformListenable listenable = getListenable(fut);
 
         listen(ctx, listenable, futPtr, typ, writer, target);
@@ -144,7 +144,7 @@ public class PlatformFutureUtils {
      * @return Resulting listenable.
      */
     public static PlatformListenable listen(final PlatformContext ctx, IgniteInternalFuture fut, final long futPtr,
-        Writer writer, PlatformAbstractTarget target) {
+        Writer writer, PlatformTarget target) {
         PlatformListenable listenable = getListenable(fut);
 
         listen(ctx, listenable, futPtr, TYP_OBJ, writer, target);
@@ -183,7 +183,7 @@ public class PlatformFutureUtils {
      */
     @SuppressWarnings("unchecked")
     public static void listen(final PlatformContext ctx, PlatformListenable listenable, final long futPtr, final
-        int typ, @Nullable final Writer writer, final PlatformAbstractTarget target) {
+        int typ, @Nullable final Writer writer, final PlatformTarget target) {
         final PlatformCallbackGateway gate = ctx.gateway();
 
         listenable.listen(new IgniteBiInClosure<Object, Throwable>() {

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformListenableTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformListenableTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformListenableTarget.java
new file mode 100644
index 0000000..7d65913
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/utils/PlatformListenableTarget.java
@@ -0,0 +1,62 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.platform.utils;
+
+import org.apache.ignite.IgniteCheckedException;
+import org.apache.ignite.internal.processors.platform.PlatformAbstractTarget;
+import org.apache.ignite.internal.processors.platform.PlatformContext;
+
+/**
+ * Wraps listenable in a platform target.
+ */
+public class PlatformListenableTarget extends PlatformAbstractTarget {
+    /** */
+    private static final int OP_CANCEL = 1;
+
+    /** */
+    private static final int OP_IS_CANCELLED = 2;
+
+    /** Wrapped listenable */
+    private final PlatformListenable listenable;
+
+    /**
+     * Constructor.
+     *
+     * @param platformCtx Context.
+     */
+    public PlatformListenableTarget(PlatformListenable listenable, PlatformContext platformCtx) {
+        super(platformCtx);
+
+        assert listenable != null;
+
+        this.listenable = listenable;
+    }
+
+    /** {@inheritDoc} */
+    @Override public long processInLongOutLong(int type, long val) throws IgniteCheckedException {
+        switch (type) {
+            case OP_CANCEL:
+                return listenable.cancel() ? TRUE : FALSE;
+
+            case OP_IS_CANCELLED:
+                return listenable.isCancelled() ? TRUE : FALSE;
+        }
+
+        return super.processInLongOutLong(type, val);
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/include/ignite/jni/exports.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/exports.h b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
index 3052435..a2e5cbb 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/exports.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/exports.h
@@ -80,9 +80,6 @@ extern "C" {
 
     void IGNITE_CALL IgniteDestroyJvm(gcj::JniContext* ctx);
 
-    bool IGNITE_CALL IgniteListenableCancel(gcj::JniContext* ctx, void* obj);
-    bool IGNITE_CALL IgniteListenableIsCancelled(gcj::JniContext* ctx, void* obj);
-
     void IGNITE_CALL IgniteSetConsoleHandler(gcj::ConsoleWriteHandler consoleHandler);
     void IGNITE_CALL IgniteRemoveConsoleHandler(gcj::ConsoleWriteHandler consoleHandler);
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/include/ignite/jni/java.h
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/include/ignite/jni/java.h b/modules/platforms/cpp/jni/include/ignite/jni/java.h
index 07df001..97e4412 100644
--- a/modules/platforms/cpp/jni/include/ignite/jni/java.h
+++ b/modules/platforms/cpp/jni/include/ignite/jni/java.h
@@ -286,10 +286,6 @@ namespace ignite
                 jmethodID m_PlatformUtils_reallocate;
                 jmethodID m_PlatformUtils_errData;
 
-                jclass c_PlatformListenable;
-                jmethodID m_PlatformListenable_cancel;
-                jmethodID m_PlatformListenable_isCancelled;
-
                 /**
                  * Constructor.
                  */
@@ -465,9 +461,6 @@ namespace ignite
                 jobject CacheOutOpQueryCursor(jobject obj, int type, long long memPtr, JniErrorInfo* errInfo = NULL);
                 jobject CacheOutOpContinuousQuery(jobject obj, int type, long long memPtr);
 
-                bool ListenableCancel(jobject obj);
-                bool ListenableIsCancelled(jobject obj);
-
                 jobject Acquire(jobject obj);
 
                 void DestroyJvm();

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/project/vs/module.def
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/project/vs/module.def b/modules/platforms/cpp/jni/project/vs/module.def
index e58ac3b..fb56dca 100644
--- a/modules/platforms/cpp/jni/project/vs/module.def
+++ b/modules/platforms/cpp/jni/project/vs/module.def
@@ -36,8 +36,6 @@ IgniteDestroyJvm @86
 IgniteTargetOutObject @91 
 IgniteProcessorExtensions @97
 IgniteProcessorAtomicLong @98
-IgniteListenableCancel @110
-IgniteListenableIsCancelled @111
 IgniteProcessorCreateCacheFromConfig @114
 IgniteProcessorGetOrCreateCacheFromConfig @115
 IgniteProcessorGetIgniteConfiguration @116

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/src/exports.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/exports.cpp b/modules/platforms/cpp/jni/src/exports.cpp
index dde98fb..b842c03 100644
--- a/modules/platforms/cpp/jni/src/exports.cpp
+++ b/modules/platforms/cpp/jni/src/exports.cpp
@@ -214,14 +214,6 @@ extern "C" {
         ctx->DestroyJvm();
     }
 
-    bool IGNITE_CALL IgniteListenableCancel(gcj::JniContext* ctx, void* obj) {
-        return ctx->ListenableCancel(static_cast<jobject>(obj));
-    }
-
-    bool IGNITE_CALL IgniteListenableIsCancelled(gcj::JniContext* ctx, void* obj) {
-        return ctx->ListenableIsCancelled(static_cast<jobject>(obj));
-    }
-
     void IGNITE_CALL IgniteSetConsoleHandler(gcj::ConsoleWriteHandler consoleHandler) {
         gcj::JniContext::SetConsoleHandler(consoleHandler);
     }

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/cpp/jni/src/java.cpp
----------------------------------------------------------------------
diff --git a/modules/platforms/cpp/jni/src/java.cpp b/modules/platforms/cpp/jni/src/java.cpp
index 2d3cf72..9626fbb 100644
--- a/modules/platforms/cpp/jni/src/java.cpp
+++ b/modules/platforms/cpp/jni/src/java.cpp
@@ -194,33 +194,33 @@ namespace ignite
 
             const char* C_PLATFORM_PROCESSOR = "org/apache/ignite/internal/processors/platform/PlatformProcessor";
             JniMethod M_PLATFORM_PROCESSOR_RELEASE_START = JniMethod("releaseStart", "()V", false);
-            JniMethod M_PLATFORM_PROCESSOR_PROJECTION = JniMethod("projection", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_CACHE = JniMethod("cache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE = JniMethod("createCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE = JniMethod("getOrCreateCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE_FROM_CONFIG = JniMethod("createCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE_FROM_CONFIG = JniMethod("getOrCreateCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_CREATE_NEAR_CACHE = JniMethod("createNearCache", "(Ljava/lang/String;J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_NEAR_CACHE = JniMethod("getOrCreateNearCache", "(Ljava/lang/String;J)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_PROJECTION = JniMethod("projection", "()Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_CACHE = JniMethod("cache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE = JniMethod("createCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE = JniMethod("getOrCreateCache", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_CREATE_CACHE_FROM_CONFIG = JniMethod("createCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_CACHE_FROM_CONFIG = JniMethod("getOrCreateCacheFromConfig", "(J)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_CREATE_NEAR_CACHE = JniMethod("createNearCache", "(Ljava/lang/String;J)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_GET_OR_CREATE_NEAR_CACHE = JniMethod("getOrCreateNearCache", "(Ljava/lang/String;J)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
             JniMethod M_PLATFORM_PROCESSOR_DESTROY_CACHE = JniMethod("destroyCache", "(Ljava/lang/String;)V", false);
-            JniMethod M_PLATFORM_PROCESSOR_AFFINITY = JniMethod("affinity", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_DATA_STREAMER = JniMethod("dataStreamer", "(Ljava/lang/String;Z)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_TRANSACTIONS = JniMethod("transactions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_COMPUTE = JniMethod("compute", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_MESSAGE = JniMethod("message", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_EVENTS = JniMethod("events", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_SERVICES = JniMethod("services", "(Lorg/apache/ignite/internal/processors/platform/PlatformTarget;)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_EXTENSIONS = JniMethod("extensions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_ATOMIC_LONG = JniMethod("atomicLong", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_ATOMIC_SEQUENCE = JniMethod("atomicSequence", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
-            JniMethod M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE = JniMethod("atomicReference", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_AFFINITY = JniMethod("affinity", "(Ljava/lang/String;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_DATA_STREAMER = JniMethod("dataStreamer", "(Ljava/lang/String;Z)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_TRANSACTIONS = JniMethod("transactions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_COMPUTE = JniMethod("compute", "(Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_MESSAGE = JniMethod("message", "(Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_EVENTS = JniMethod("events", "(Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_SERVICES = JniMethod("services", "(Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_EXTENSIONS = JniMethod("extensions", "()Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_ATOMIC_LONG = JniMethod("atomicLong", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_ATOMIC_SEQUENCE = JniMethod("atomicSequence", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
+            JniMethod M_PLATFORM_PROCESSOR_ATOMIC_REFERENCE = JniMethod("atomicReference", "(Ljava/lang/String;JZ)Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
             JniMethod M_PLATFORM_PROCESSOR_GET_IGNITE_CONFIGURATION = JniMethod("getIgniteConfiguration", "(J)V", false);
             JniMethod M_PLATFORM_PROCESSOR_GET_CACHE_NAMES = JniMethod("getCacheNames", "(J)V", false);
             JniMethod M_PLATFORM_PROCESSOR_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(I)Z", false);
             JniMethod M_PLATFORM_PROCESSOR_LOGGER_LOG = JniMethod("loggerLog", "(ILjava/lang/String;Ljava/lang/String;Ljava/lang/String;)V", false);
-            JniMethod M_PLATFORM_PROCESSOR_BINARY_PROCESSOR = JniMethod("binaryProcessor", "()Lorg/apache/ignite/internal/processors/platform/PlatformTarget;", false);
+            JniMethod M_PLATFORM_PROCESSOR_BINARY_PROCESSOR = JniMethod("binaryProcessor", "()Lorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;", false);
 
-            const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTarget";
+            const char* C_PLATFORM_TARGET = "org/apache/ignite/internal/processors/platform/PlatformTargetProxy";
             JniMethod M_PLATFORM_TARGET_IN_LONG_OUT_LONG = JniMethod("inLongOutLong", "(IJ)J", false);
             JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_LONG = JniMethod("inStreamOutLong", "(IJ)J", false);
             JniMethod M_PLATFORM_TARGET_IN_STREAM_OUT_OBJECT = JniMethod("inStreamOutObject", "(IJ)Ljava/lang/Object;", false);
@@ -260,7 +260,7 @@ namespace ignite
             JniMethod M_PLATFORM_CALLBACK_UTILS_CONTINUOUS_QUERY_FILTER_RELEASE = JniMethod("continuousQueryFilterRelease", "(JJ)V", true);
 
             JniMethod M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_TOPOLOGY_UPDATE = JniMethod("dataStreamerTopologyUpdate", "(JJJI)V", true);
-            JniMethod M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_STREAM_RECEIVER_INVOKE = JniMethod("dataStreamerStreamReceiverInvoke", "(JJLjava/lang/Object;JZ)V", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_DATA_STREAMER_STREAM_RECEIVER_INVOKE = JniMethod("dataStreamerStreamReceiverInvoke", "(JJLorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;JZ)V", true);
 
             JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_BYTE_RES = JniMethod("futureByteResult", "(JJI)V", true);
             JniMethod M_PLATFORM_CALLBACK_UTILS_FUTURE_BOOL_RES = JniMethod("futureBoolResult", "(JJI)V", true);
@@ -307,7 +307,7 @@ namespace ignite
             JniMethod M_PLATFORM_CALLBACK_UTILS_LOGGER_LOG = JniMethod("loggerLog", "(JILjava/lang/String;Ljava/lang/String;Ljava/lang/String;J)V", true);
             JniMethod M_PLATFORM_CALLBACK_UTILS_LOGGER_IS_LEVEL_ENABLED = JniMethod("loggerIsLevelEnabled", "(JI)Z", true);
 
-            JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_INIT = JniMethod("affinityFunctionInit", "(JJLorg/apache/ignite/internal/processors/platform/cache/affinity/PlatformAffinityFunctionTarget;)J", true);
+            JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_INIT = JniMethod("affinityFunctionInit", "(JJLorg/apache/ignite/internal/processors/platform/PlatformTargetProxy;)J", true);
             JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_PARTITION = JniMethod("affinityFunctionPartition", "(JJJ)I", true);
             JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_ASSIGN_PARTITIONS = JniMethod("affinityFunctionAssignPartitions", "(JJJJ)V", true);
             JniMethod M_PLATFORM_CALLBACK_UTILS_AFFINITY_FUNCTION_REMOVE_NODE = JniMethod("affinityFunctionRemoveNode", "(JJJ)V", true);
@@ -327,10 +327,6 @@ namespace ignite
             JniMethod M_PLATFORM_IGNITION_STOP = JniMethod("stop", "(Ljava/lang/String;Z)Z", true);
             JniMethod M_PLATFORM_IGNITION_STOP_ALL = JniMethod("stopAll", "(Z)V", true);
 
-            const char* C_PLATFORM_LISTENABLE = "org/apache/ignite/internal/processors/platform/utils/PlatformListenable";
-            JniMethod M_PLATFORM_LISTENABLE_CANCEL = JniMethod("cancel", "()Z", false);
-            JniMethod M_PLATFORM_LISTENABLE_IS_CANCELED = JniMethod("isCancelled", "()Z", false);
-
             /* STATIC STATE. */
             gcc::CriticalSection JVM_LOCK;
             gcc::CriticalSection CONSOLE_LOCK;
@@ -552,10 +548,6 @@ namespace ignite
                 m_PlatformUtils_reallocate = FindMethod(env, c_PlatformUtils, M_PLATFORM_UTILS_REALLOC);
                 m_PlatformUtils_errData = FindMethod(env, c_PlatformUtils, M_PLATFORM_UTILS_ERR_DATA);
 
-                c_PlatformListenable = FindClass(env, C_PLATFORM_LISTENABLE);
-                m_PlatformListenable_cancel = FindMethod(env, c_PlatformListenable, M_PLATFORM_LISTENABLE_CANCEL);                    
-                m_PlatformListenable_isCancelled = FindMethod(env, c_PlatformListenable, M_PLATFORM_LISTENABLE_IS_CANCELED);
-
                 // Find utility classes which are not used from context, but are still required in other places.
                 CheckClass(env, C_PLATFORM_NO_CALLBACK_EXCEPTION);
             }
@@ -1447,28 +1439,6 @@ namespace ignite
                 return LocalToGlobal(env, res);
             }
 
-            bool JniContext::ListenableCancel(jobject obj)
-            {
-                JNIEnv* env = Attach();
-
-                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformListenable_cancel);
-
-                ExceptionCheck(env);
-
-                return res != 0;;
-            }
-
-            bool JniContext::ListenableIsCancelled(jobject obj)
-            {
-                JNIEnv* env = Attach();
-
-                jboolean res = env->CallBooleanMethod(obj, jvm->GetMembers().m_PlatformListenable_isCancelled);
-
-                ExceptionCheck(env);
-
-                return res != 0;;
-            }
-
 			jobject JniContext::Acquire(jobject obj)
             {
                 if (obj) {

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
index f945efe..6421b8c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Apache.Ignite.Core.csproj
@@ -181,6 +181,7 @@
     <Compile Include="Impl\Cache\Expiry\ExpiryPolicyFactory.cs" />
     <Compile Include="Impl\Cache\Expiry\ExpiryPolicySerializer.cs" />
     <Compile Include="Impl\Cache\ICacheLockInternal.cs" />
+    <Compile Include="Impl\Common\Listenable.cs" />
     <Compile Include="Impl\Common\Platform.cs" />
     <Compile Include="Impl\Binary\UserSerializerProxy.cs" />
     <Compile Include="Impl\Cache\Affinity\AffinityFunctionSerializer.cs" />

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
index 68bd9d4..50102a7 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Future.cs
@@ -24,7 +24,6 @@ namespace Apache.Ignite.Core.Impl.Common
     using System.Threading.Tasks;
     using Apache.Ignite.Core.Common;
     using Apache.Ignite.Core.Impl.Binary.IO;
-    using Apache.Ignite.Core.Impl.Unmanaged;
 
     /// <summary>
     /// Grid future implementation.
@@ -40,7 +39,7 @@ namespace Apache.Ignite.Core.Impl.Common
         private readonly TaskCompletionSource<T> _taskCompletionSource = new TaskCompletionSource<T>();
 
         /** */
-        private volatile IUnmanagedTarget _unmanagedTarget;
+        private volatile Listenable _listenable;
 
         /// <summary>
         /// Constructor.
@@ -84,7 +83,7 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <param name="cancellationToken">The cancellation token.</param>
         public Task<T> GetTask(CancellationToken cancellationToken)
         {
-            Debug.Assert(_unmanagedTarget != null);
+            Debug.Assert(_listenable != null);
 
             // OnTokenCancel will fire even if cancellationToken is already cancelled.
             cancellationToken.Register(OnTokenCancel);
@@ -169,11 +168,11 @@ namespace Apache.Ignite.Core.Impl.Common
         /// <summary>
         /// Sets unmanaged future target for cancellation.
         /// </summary>
-        internal void SetTarget(IUnmanagedTarget target)
+        internal void SetTarget(Listenable target)
         {
             Debug.Assert(target != null);
 
-            _unmanagedTarget = target;
+            _listenable = target;
         }
 
         /// <summary>
@@ -181,8 +180,8 @@ namespace Apache.Ignite.Core.Impl.Common
         /// </summary>
         private void OnTokenCancel()
         {
-            if (_unmanagedTarget != null)
-                UnmanagedUtils.ListenableCancel(_unmanagedTarget);
+            if (_listenable != null)
+                _listenable.Cancel();
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Listenable.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Listenable.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Listenable.cs
new file mode 100644
index 0000000..6da98ab
--- /dev/null
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Common/Listenable.cs
@@ -0,0 +1,49 @@
+\ufeff/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+namespace Apache.Ignite.Core.Impl.Common
+{
+    using Apache.Ignite.Core.Impl.Binary;
+    using Apache.Ignite.Core.Impl.Unmanaged;
+
+    /// <summary>
+    /// Platform listenable.
+    /// </summary>
+    internal class Listenable : PlatformTarget
+    {
+        /** */
+        private const int OpCancel = 1;
+
+        /// <summary>
+        /// Initializes a new instance of the <see cref="Listenable"/> class.
+        /// </summary>
+        /// <param name="target">Target.</param>
+        /// <param name="marsh">Marshaller.</param>
+        public Listenable(IUnmanagedTarget target, Marshaller marsh) : base(target, marsh)
+        {
+            // No-op.
+        }
+
+        /// <summary>
+        /// Cancels the listenable.
+        /// </summary>
+        public void Cancel()
+        {
+            DoOutInOp(OpCancel);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
index bc7c7d9..d36caf3 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Compute/ComputeImpl.cs
@@ -195,7 +195,7 @@ namespace Apache.Ignite.Core.Impl.Compute
 
             var future = holder.Future;
 
-            future.SetTarget(futTarget);
+            future.SetTarget(new Listenable(futTarget, Marshaller));
 
             return future;
         }
@@ -550,7 +550,7 @@ namespace Apache.Ignite.Core.Impl.Compute
                             writeAction(writer);
                     });
 
-                    holder.Future.SetTarget(futTarget);
+                    holder.Future.SetTarget(new Listenable(futTarget, Marshaller));
                 }
                 catch (Exception e)
                 {

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
index f4a07f6..9cf2a6c 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/PlatformTarget.cs
@@ -869,7 +869,7 @@ namespace Apache.Ignite.Core.Impl
                 throw;
             }
 
-            fut.SetTarget(futTarget);
+            fut.SetTarget(new Listenable(futTarget, _marsh));
 
             return fut;
         }

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
index c746866..c4f3e19 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/IgniteJniNativeMethods.cs
@@ -167,14 +167,6 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
         [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteDestroyJvm")]
         public static extern void DestroyJvm(void* ctx);
 
-        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteListenableCancel")]
-        [return: MarshalAs(UnmanagedType.U1)]
-        public static extern bool ListenableCancel(void* ctx, void* target);
-
-        [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteListenableIsCancelled")]
-        [return: MarshalAs(UnmanagedType.U1)]
-        public static extern bool ListenableIsCancelled(void* ctx, void* target);
-
         [DllImport(IgniteUtils.FileIgniteJniDll, EntryPoint = "IgniteSetConsoleHandler")]
         public static extern void SetConsoleHandler(void* consoleHandler);
 

http://git-wip-us.apache.org/repos/asf/ignite/blob/59e6fec0/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
----------------------------------------------------------------------
diff --git a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
index f36c35f..0a2a1f0 100644
--- a/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
+++ b/modules/platforms/dotnet/Apache.Ignite.Core/Impl/Unmanaged/UnmanagedUtils.cs
@@ -516,11 +516,6 @@ namespace Apache.Ignite.Core.Impl.Unmanaged
             JNI.DestroyJvm(ctx);
         }
 
-        internal static bool ListenableCancel(IUnmanagedTarget target)
-        {
-            return JNI.ListenableCancel(target.Context, target.Target);
-        }
-
         #endregion
     }
 }