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 08:57:30 UTC

[01/14] 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-comm-balance-master 03593023d -> 3b0c302e4


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();
     }
 
     /**


[14/14] ignite git commit: Merge remote-tracking branch 'remotes/origin/master' into ignite-comm-balance-master

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

# Conflicts:
#	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


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

Branch: refs/heads/ignite-comm-balance-master
Commit: 3b0c302e4bba18c6b517fc388d2deec0b834c1a8
Parents: 0359302 2f51b4a
Author: sboikov <sb...@gridgain.com>
Authored: Fri Dec 9 11:57:14 2016 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Fri Dec 9 11:57:14 2016 +0300

----------------------------------------------------------------------
 .../rendezvous/RendezvousAffinityFunction.java  |  80 ++++--
 .../discovery/GridDiscoveryManager.java         | 118 +-------
 .../processors/cache/GridCacheAdapter.java      |   6 +-
 .../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 +-
 .../dht/GridClientPartitionTopology.java        | 120 ++++----
 .../dht/GridDhtPartitionTopology.java           |  28 +-
 .../dht/GridDhtPartitionTopologyImpl.java       | 284 +++++++++++--------
 .../dht/preloader/GridDhtPartitionDemander.java |  46 ++-
 .../dht/preloader/GridDhtPartitionFullMap.java  |  18 +-
 .../GridDhtPartitionsExchangeFuture.java        |  72 ++++-
 .../dht/preloader/GridDhtPreloader.java         |  11 +-
 .../cache/transactions/IgniteTxHandler.java     |   2 +-
 .../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 +-
 .../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 +-
 .../distributed/dht/GridCacheDhtTestUtils.java  | 232 ---------------
 .../testsuites/IgniteCacheTestSuite5.java       |   9 +-
 .../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 -
 72 files changed, 1423 insertions(+), 1186 deletions(-)
----------------------------------------------------------------------


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

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

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b0c302e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b0c302e/modules/core/src/main/java/org/apache/ignite/internal/processors/platform/compute/PlatformCompute.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/ignite/blob/3b0c302e/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --cc modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index aeb3ef4,b9b92b8..9a5f077
--- 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
@@@ -1587,9 -1587,9 +1587,9 @@@ public class GridServiceProcessor exten
                  else
                      topVer = new AffinityTopologyVersion(((DiscoveryEvent)evt).topologyVersion(), 0);
  
 -                depExe.submit(new BusyRunnable() {
 +                depExe.execute(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/3b0c302e/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/ServerImpl.java
----------------------------------------------------------------------


[10/14] 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-comm-balance-master
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.


[04/14] 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-comm-balance-master
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);
         }
     }


[02/14] 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-comm-balance-master
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());
 


[07/14] 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-comm-balance-master
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
----------------------------------------------------------------------


[08/14] 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-comm-balance-master
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");


[09/14] 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
     }
 }


[06/14] 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-comm-balance-master
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/14] 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-comm-balance-master
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


[03/14] 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-comm-balance-master
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;


[11/14] 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-comm-balance-master
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;
     }
 }


[13/14] 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-comm-balance-master
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

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

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



[12/14] 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-comm-balance-master
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;
     }
 }