You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by yz...@apache.org on 2015/03/04 19:16:26 UTC

[4/5] incubator-ignite git commit: futures: api cleanup

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index add51bd..cd3eb59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -598,7 +598,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         @Override public IgniteInternalFuture<T> apply(T t, Exception e) {
                             return op.apply();
                         }
-                    }, ctx.kernalContext());
+                    });
 
                 saveFuture(holder, f);
 
@@ -796,7 +796,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             conflictRmvMap != null ? conflictRmvMap.values() : null,
             retval,
             rawRetval,
-            cached,
             prj != null ? prj.expiry() : null,
             filter,
             subjId,
@@ -859,7 +858,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             keys != null ? null : conflictMap.values(),
             retval,
             rawRetval,
-            cached,
             (filter != null && prj != null) ? prj.expiry() : null,
             filter,
             subjId,
@@ -901,7 +899,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         ctx.checkSecurity(GridSecurityPermission.CACHE_READ);
 
         if (F.isEmpty(keys))
-            return new GridFinishedFuture<>(ctx.kernalContext(), Collections.<K, V>emptyMap());
+            return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
 
         if (keyCheck)
             validateCacheKeys(keys);
@@ -980,7 +978,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         break; // While.
                     }
                     catch (IgniteCheckedException e) {
-                        return new GridFinishedFuture<>(ctx.kernalContext(), e);
+                        return new GridFinishedFuture<>(e);
                     }
                     finally {
                         if (entry != null)
@@ -997,7 +995,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             if (success) {
                 sendTtlUpdateRequest(expiry);
 
-                return ctx.wrapCloneMap(new GridFinishedFuture<>(ctx.kernalContext(), locVals));
+                return ctx.wrapCloneMap(new GridFinishedFuture<>(locVals));
             }
         }
 
@@ -2320,7 +2318,6 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             drRmvVals,
             req.returnValue(),
             false,
-            null,
             req.expiry(),
             req.filter(),
             req.subjectId(),
@@ -2787,7 +2784,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /**
      *
      */
-    private static class FinishedLockFuture extends GridFinishedFutureEx<Boolean> implements GridDhtFuture<Boolean> {
+    private static class FinishedLockFuture extends GridFinishedFuture<Boolean> implements GridDhtFuture<Boolean> {
         /** */
         private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
index 92fe74b..b284b79 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicUpdateFuture.java
@@ -33,7 +33,6 @@ import org.jdk8.backport.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.processor.*;
-import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -90,13 +89,6 @@ public class GridDhtAtomicUpdateFuture<K, V> extends GridFutureAdapter<Void>
     private volatile long mapTime;
 
     /**
-     * Empty constructor required by {@link Externalizable}.
-     */
-    public GridDhtAtomicUpdateFuture() {
-        // No-op.
-    }
-
-    /**
      * @param cctx Cache context.
      * @param completionCb Callback to invoke when future is completed.
      * @param writeVer Write version.
@@ -110,7 +102,7 @@ public class GridDhtAtomicUpdateFuture<K, V> extends GridFutureAdapter<Void>
         GridNearAtomicUpdateRequest<K, V> updateReq,
         GridNearAtomicUpdateResponse<K, V> updateRes
     ) {
-        super(cctx.kernalContext());
+        super();
 
         this.cctx = cctx;
         this.writeVer = writeVer;
@@ -122,7 +114,8 @@ public class GridDhtAtomicUpdateFuture<K, V> extends GridFutureAdapter<Void>
 
         forceTransformBackups = updateReq.forceTransformBackups();
 
-        log = U.logger(ctx, logRef, GridDhtAtomicUpdateFuture.class);
+        if (log == null)
+            log = U.logger(cctx.kernalContext(), logRef, GridDhtAtomicUpdateFuture.class);
 
         keys = new ArrayList<>(updateReq.keys().size());
     }
@@ -234,7 +227,7 @@ public class GridDhtAtomicUpdateFuture<K, V> extends GridFutureAdapter<Void>
         for (ClusterNode node : dhtNodes) {
             UUID nodeId = node.id();
 
-            if (!nodeId.equals(ctx.localNodeId())) {
+            if (!nodeId.equals(cctx.localNodeId())) {
                 GridDhtAtomicUpdateRequest<K, V> updateReq = mappings.get(nodeId);
 
                 if (updateReq == null) {
@@ -291,7 +284,7 @@ public class GridDhtAtomicUpdateFuture<K, V> extends GridFutureAdapter<Void>
             GridDhtAtomicUpdateRequest<K, V> updateReq = mappings.get(nodeId);
 
             if (updateReq == null) {
-                ClusterNode node = ctx.discovery().node(nodeId);
+                ClusterNode node = cctx.discovery().node(nodeId);
 
                 // Node left the grid.
                 if (node == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index c3cc50a..bf3ec95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -38,8 +38,8 @@ import org.jdk8.backport.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
+import javax.cache.Cache.*;
 import javax.cache.expiry.*;
-import java.io.*;
 import java.util.*;
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -106,9 +106,6 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
     /** Return value require flag. */
     private final boolean retval;
 
-    /** Cached entry if keys size is 1. */
-    private GridCacheEntryEx<K, V> cached;
-
     /** Expiry policy. */
     private final ExpiryPolicy expiryPlc;
 
@@ -149,28 +146,10 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
     private volatile long mapTime;
 
     /**
-     * Empty constructor required by {@link Externalizable}.
-     */
-    public GridNearAtomicUpdateFuture() {
-        cctx = null;
-        mappings = null;
-        futVer = null;
-        retval = false;
-        fastMap = false;
-        expiryPlc = null;
-        filter = null;
-        syncMode = null;
-        op = null;
-        nearEnabled = false;
-        subjId = null;
-        taskNameHash = 0;
-    }
-
-    /**
      * @param cctx Cache context.
      * @param cache Cache instance.
-     * @param op Update operation.
      * @param syncMode Write synchronization mode.
+     * @param op Update operation.
      * @param keys Keys to update.
      * @param vals Values or transform closure.
      * @param invokeArgs Optional arguments for entry processor.
@@ -178,7 +157,6 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
      * @param conflictRmvVals Conflict remove values (optional).
      * @param retval Return value require flag.
      * @param rawRetval {@code True} if should return {@code GridCacheReturn} as future result.
-     * @param cached Cached entry if keys size is 1.
      * @param expiryPlc Expiry policy explicitly specified for cache operation.
      * @param filter Entry filter.
      * @param subjId Subject ID.
@@ -196,20 +174,18 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
         @Nullable Collection<GridCacheVersion> conflictRmvVals,
         final boolean retval,
         final boolean rawRetval,
-        @Nullable GridCacheEntryEx<K, V> cached,
         @Nullable ExpiryPolicy expiryPlc,
-        final IgnitePredicate<Cache.Entry<K, V>>[] filter,
+        final IgnitePredicate<Entry<K, V>>[] filter,
         UUID subjId,
         int taskNameHash
     ) {
-        super(cctx.kernalContext());
+        super();
 
         this.rawRetval = rawRetval;
 
         assert vals == null || vals.size() == keys.size();
         assert conflictPutVals == null || conflictPutVals.size() == keys.size();
         assert conflictRmvVals == null || conflictRmvVals.size() == keys.size();
-        assert cached == null || keys.size() == 1;
         assert subjId != null;
 
         this.cctx = cctx;
@@ -222,13 +198,13 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
         this.conflictPutVals = conflictPutVals;
         this.conflictRmvVals = conflictRmvVals;
         this.retval = retval;
-        this.cached = cached;
         this.expiryPlc = expiryPlc;
         this.filter = filter;
         this.subjId = subjId;
         this.taskNameHash = taskNameHash;
 
-        log = U.logger(ctx, logRef, GridFutureAdapter.class);
+        if (log == null)
+            log = U.logger(cctx.kernalContext(), logRef, GridFutureAdapter.class);
 
         mappings = new ConcurrentHashMap8<>(keys.size(), 1.0f);
 
@@ -781,7 +757,7 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
         singleNodeId = nodeId;
         singleReq = req;
 
-        if (ctx.localNodeId().equals(nodeId)) {
+        if (cctx.localNodeId().equals(nodeId)) {
             cache.updateAllAsyncInternal(nodeId, req,
                 new CI2<GridNearAtomicUpdateRequest<K, V>, GridNearAtomicUpdateResponse<K, V>>() {
                     @Override public void apply(GridNearAtomicUpdateRequest<K, V> req,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index cdb1759..f15c66e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -167,7 +167,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         ctx.checkSecurity(GridSecurityPermission.CACHE_READ);
 
         if (F.isEmpty(keys))
-            return new GridFinishedFuture<>(ctx.kernalContext(), Collections.<K, V>emptyMap());
+            return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
 
         IgniteTxLocalAdapter<K, V> tx = ctx.tm().threadLocalTx();
 
@@ -232,7 +232,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         boolean skipVals
     ) {
         if (keys == null || keys.isEmpty())
-            return new GridFinishedFuture<>(ctx.kernalContext(), Collections.<K, V>emptyMap());
+            return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
 
         if (keyCheck)
             validateCacheKeys(keys);
@@ -308,7 +308,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                         break; // While.
                     }
                     catch (IgniteCheckedException e) {
-                        return new GridFinishedFuture<>(ctx.kernalContext(), e);
+                        return new GridFinishedFuture<>(e);
                     }
                     finally {
                         if (entry != null)
@@ -325,7 +325,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             if (success) {
                 sendTtlUpdateRequest(expiryPlc);
 
-                return ctx.wrapCloneMap(new GridFinishedFuture<>(ctx.kernalContext(), locVals));
+                return ctx.wrapCloneMap(new GridFinishedFuture<>(locVals));
             }
         }
 
@@ -625,11 +625,11 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                     filter);
             }
             catch (IgniteCheckedException e) {
-                return new GridFinishedFuture<>(ctx.kernalContext(), e);
+                return new GridFinishedFuture<>(e);
             }
         }
         else {
-            return new GridEmbeddedFuture<>(true, keyFut,
+            return new GridEmbeddedFuture<>(keyFut,
                 new C2<Object, Exception, IgniteInternalFuture<Exception>>() {
                     @Override public IgniteInternalFuture<Exception> apply(Object o, Exception exx) {
                         if (exx != null)
@@ -646,8 +646,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                             accessTtl,
                             filter);
                     }
-                },
-                ctx.kernalContext());
+                }
+            );
         }
     }
 
@@ -731,7 +731,6 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             fut.map();
 
             return new GridDhtEmbeddedFuture<>(
-                ctx.kernalContext(),
                 fut,
                 new C2<Boolean, Exception, Exception>() {
                     @Override public Exception apply(Boolean b, Exception e) {
@@ -742,7 +741,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
 
                         return e;
                     }
-                });
+                }, false);
         }
         else {
             // Handle implicit locks for pessimistic transactions.
@@ -758,7 +757,6 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
                 accessTtl);
 
             return new GridDhtEmbeddedFuture<>(
-                ctx.kernalContext(),
                 txFut,
                 new C2<GridCacheReturn<V>, Exception, Exception>() {
                     @Override public Exception apply(GridCacheReturn<V> ret,
@@ -770,7 +768,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
 
                         return e;
                     }
-                });
+                }, false);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index 21f6364..88c7f63 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -159,7 +159,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
 
         futId = IgniteUuid.randomUuid();
 
-        log = U.logger(ctx, logRef, GridDhtColocatedLockFuture.class);
+        log = U.logger(cctx.kernalContext(), logRef, GridDhtColocatedLockFuture.class);
 
         if (timeout > 0) {
             timeoutObj = new LockTimeoutObject();
@@ -888,7 +888,6 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
 
         // Add new future.
         add(new GridEmbeddedFuture<>(
-            cctx.kernalContext(),
             fut,
             new C2<Exception, Exception, Boolean>() {
                 @Override public Boolean apply(Exception resEx, Exception e) {
@@ -934,7 +933,8 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
 
                     return true;
                 }
-            }
+            },
+            false
         ));
     }
 
@@ -1120,7 +1120,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
          */
         MiniFuture(ClusterNode node, Collection<K> keys,
             Deque<GridNearLockMapping<K, V>> mappings) {
-            super(cctx.kernalContext());
+            super();
 
             this.node = node;
             this.keys = keys;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 170a0c0..7f655df 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -52,6 +52,9 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
     /** Logger reference. */
     private static final AtomicReference<IgniteLogger> logRef = new AtomicReference<>();
 
+    /** Logger. */
+    private static IgniteLogger log;
+
     /** Wait for 1 second for topology to change. */
     private static final long REMAP_PAUSE = 1000;
 
@@ -61,9 +64,6 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
     /** Topology. */
     private GridDhtPartitionTopology<K, V> top;
 
-    /** Logger. */
-    private IgniteLogger log;
-
     /** Keys to request. */
     private Collection<? extends K> keys;
 
@@ -91,8 +91,11 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
      * @param keys Keys.
      * @param preloader Preloader.
      */
-    public GridDhtForceKeysFuture(GridCacheContext<K, V> cctx, long topVer, Collection<? extends K> keys,
-        GridDhtPreloader<K, V> preloader) {
+    public GridDhtForceKeysFuture(
+        GridCacheContext<K, V> cctx,
+        long topVer, Collection<? extends K> keys,
+        GridDhtPreloader<K, V> preloader
+    ) {
         super(cctx.kernalContext());
 
         assert topVer != 0 : topVer;
@@ -105,9 +108,8 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
 
         top = cctx.dht().topology();
 
-        log = U.logger(ctx, logRef, GridDhtForceKeysFuture.class);
-
-        syncNotify(true);
+        if (log == null)
+            log = U.logger(cctx.kernalContext(), logRef, GridDhtForceKeysFuture.class);
     }
 
     /**
@@ -405,7 +407,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
          * @param exc Exclude node list.
          */
         MiniFuture(ClusterNode node, Collection<K> keys, int curTopVer, Collection<ClusterNode> exc) {
-            super(cctx.kernalContext());
+            super();
 
             assert node != null;
             assert curTopVer > 0;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
index 6a1f7a1..941ffb7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
@@ -1051,7 +1051,7 @@ public class GridDhtPartitionDemandPool<K, V> {
          * @param workers List of workers.
          */
         private SyncFuture(Collection<DemandWorker> workers) {
-            super(cctx.kernalContext());
+            super();
 
             assert workers.size() == poolSize();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/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 db9bd08..ef77cce 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
@@ -150,9 +150,13 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
      * @param discoEvt Discovery event.
      * @param exchId Exchange id.
      */
-    public GridDhtPartitionsExchangeFuture(GridCacheSharedContext<K, V> cctx, boolean reassign, DiscoveryEvent discoEvt,
-        GridDhtPartitionExchangeId exchId) {
-        super(cctx.kernalContext());
+    public GridDhtPartitionsExchangeFuture(
+        GridCacheSharedContext<K, V> cctx,
+        boolean reassign,
+        DiscoveryEvent discoEvt,
+        GridDhtPartitionExchangeId exchId
+    ) {
+        super();
         dummy = true;
         forcePreload = false;
 
@@ -161,8 +165,6 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
         this.discoEvt = discoEvt;
         this.cctx = cctx;
 
-        syncNotify(true);
-
         onDone(exchId.topologyVersion());
     }
 
@@ -176,7 +178,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
      */
     public GridDhtPartitionsExchangeFuture(GridCacheSharedContext<K, V> cctx, DiscoveryEvent discoEvt,
         GridDhtPartitionExchangeId exchId) {
-        super(cctx.kernalContext());
+        super();
         dummy = false;
         forcePreload = true;
 
@@ -186,8 +188,6 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
 
         reassign = true;
 
-        syncNotify(true);
-
         onDone(exchId.topologyVersion());
     }
 
@@ -198,9 +198,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
      */
     public GridDhtPartitionsExchangeFuture(GridCacheSharedContext<K, V> cctx, ReadWriteLock busyLock,
         GridDhtPartitionExchangeId exchId) {
-        super(cctx.kernalContext());
-
-        syncNotify(true);
+        super();
 
         assert busyLock != null;
         assert exchId != null;
@@ -220,7 +218,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
 
         assert oldestNode.get() != null;
 
-        initFut = new GridFutureAdapter<>(ctx, true);
+        initFut = new GridFutureAdapter<>();
 
         if (log.isDebugEnabled())
             log.debug("Creating exchange future [localNode=" + cctx.localNodeId() +
@@ -805,7 +803,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
                 log.debug("Received full partition map from unexpected node [oldest=" + curOldest.id() +
                     ", unexpectedNodeId=" + nodeId + ']');
 
-            ClusterNode sender = ctx.discovery().node(nodeId);
+            ClusterNode sender = cctx.discovery().node(nodeId);
 
             if (sender == null) {
                 if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/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 0821431..ae28dba 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
@@ -120,7 +120,7 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
 
         top = cctx.dht().topology();
 
-        startFut = new GridFutureAdapter<>(cctx.kernalContext(), false);
+        startFut = new GridFutureAdapter<>();
     }
 
     /** {@inheritDoc} */
@@ -499,7 +499,12 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
             if (topReadyFut == null)
                 startFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
                     @Override public void apply(IgniteInternalFuture<?> syncFut) {
-                        fut.init();
+                        cctx.kernalContext().closure().runLocalSafe(
+                            new Runnable() {
+                                @Override public void run() {
+                                    fut.init();
+                                }
+                            });
                     }
                 });
             else {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index 22403ef..9b738fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -374,7 +374,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
         ctx.checkSecurity(GridSecurityPermission.CACHE_READ);
 
         if (F.isEmpty(keys))
-            return new GridFinishedFuture<>(ctx.kernalContext(), Collections.<K, V>emptyMap());
+            return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
 
         GridCacheProjectionImpl<K, V> prj = ctx.projectionPerCall();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 94d2e96..6555865 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -277,7 +277,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
         boolean skipVal
     ) {
         if (F.isEmpty(keys))
-            return new GridFinishedFuture<>(ctx.kernalContext(), Collections.<K, V>emptyMap());
+            return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
 
         if (keyCheck)
             validateCacheKeys(keys);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 9628f3b..ca07dec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -161,7 +161,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
 
         ver = tx == null ? cctx.versions().next() : tx.xidVersion();
 
-        log = U.logger(ctx, logRef, GridNearGetFuture.class);
+        log = U.logger(cctx.kernalContext(), logRef, GridNearGetFuture.class);
     }
 
     /**
@@ -347,7 +347,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                             remapKeys.add(key);
                     }
 
-                    long updTopVer = ctx.discovery().topologyVersion();
+                    long updTopVer = cctx.discovery().topologyVersion();
 
                     assert updTopVer > topVer : "Got invalid partitions for local node but topology version did " +
                         "not change [topVer=" + topVer + ", updTopVer=" + updTopVer +
@@ -511,7 +511,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                         key0 = (K)cctx.unwrapPortableIfNeeded(key, !deserializePortable);
                     }
 
-                    add(new GridFinishedFuture<>(cctx.kernalContext(), Collections.singletonMap(key0, v)));
+                    add(new GridFinishedFuture<>(Collections.singletonMap(key0, v)));
                 }
                 else {
                     if (primary == null)
@@ -704,8 +704,6 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
          * @param topVer Topology version.
          */
         MiniFuture(ClusterNode node, LinkedHashMap<K, Boolean> keys, Map<K, GridCacheVersion> savedVers, long topVer) {
-            super(cctx.kernalContext());
-
             this.node = node;
             this.keys = keys;
             this.savedVers = savedVers;
@@ -751,7 +749,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
             if (log.isDebugEnabled())
                 log.debug("Remote node left grid while sending or waiting for reply (will retry): " + this);
 
-            long updTopVer = ctx.discovery().topologyVersion();
+            long updTopVer = cctx.discovery().topologyVersion();
 
             if (updTopVer > topVer) {
                 // Remap.
@@ -760,12 +758,13 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                 onDone(Collections.<K, V>emptyMap());
             }
             else {
-                final RemapTimeoutObject timeout = new RemapTimeoutObject(ctx.config().getNetworkTimeout(), topVer, e);
+                final RemapTimeoutObject timeout = new RemapTimeoutObject(
+                    cctx.kernalContext().config().getNetworkTimeout(), topVer, e);
 
-                ctx.discovery().topologyFuture(topVer + 1).listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                cctx.discovery().topologyFuture(topVer + 1).listenAsync(new CI1<IgniteInternalFuture<Long>>() {
                     @Override public void apply(IgniteInternalFuture<Long> longIgniteFuture) {
                         if (timeout.finish()) {
-                            ctx.timeout().removeTimeoutObject(timeout);
+                            cctx.kernalContext().timeout().removeTimeoutObject(timeout);
 
                             // Remap.
                             map(keys.keySet(), F.t(node, keys), cctx.affinity().affinityTopologyVersion());
@@ -775,7 +774,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                     }
                 });
 
-                ctx.timeout().addTimeoutObject(timeout);
+                cctx.kernalContext().timeout().addTimeoutObject(timeout);
             }
         }
 
@@ -812,7 +811,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                     log.debug("Remapping mini get future [invalidParts=" + invalidParts + ", fut=" + this + ']');
 
                 // Need to wait for next topology version to remap.
-                IgniteInternalFuture<Long> topFut = ctx.discovery().topologyFuture(rmtTopVer);
+                IgniteInternalFuture<Long> topFut = cctx.discovery().topologyFuture(rmtTopVer);
 
                 topFut.listenAsync(new CIX1<IgniteInternalFuture<Long>>() {
                     @Override public void applyx(IgniteInternalFuture<Long> fut) throws IgniteCheckedException {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index 210772d..f7f7e41 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -171,7 +171,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
 
         entries = new ArrayList<>(keys.size());
 
-        log = U.logger(ctx, logRef, GridNearLockFuture.class);
+        log = U.logger(cctx.kernalContext(), logRef, GridNearLockFuture.class);
 
         if (timeout > 0) {
             timeoutObj = new LockTimeoutObject();
@@ -965,7 +965,6 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
 
             // Add new future.
             add(new GridEmbeddedFuture<>(
-                cctx.kernalContext(),
                 fut,
                 new C2<GridNearLockResponse<K, V>, Exception, Boolean>() {
                     @Override public Boolean apply(GridNearLockResponse<K, V> res, Exception e) {
@@ -1092,7 +1091,8 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
 
                         return true;
                     }
-                }
+                },
+                false
             ));
         }
         else {
@@ -1259,7 +1259,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
          */
         MiniFuture(ClusterNode node, Collection<K> keys,
             ConcurrentLinkedDeque8<GridNearLockMapping<K, V>> mappings) {
-            super(cctx.kernalContext());
+            super();
 
             this.node = node;
             this.keys = keys;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
index d6ec9dd..ff595c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -109,7 +109,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
         ctx.checkSecurity(GridSecurityPermission.CACHE_READ);
 
         if (F.isEmpty(keys))
-            return new GridFinishedFuture<>(ctx.kernalContext(), Collections.<K, V>emptyMap());
+            return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
 
         IgniteTxLocalAdapter<K, V> tx = ctx.tm().threadLocalTx();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index f3811c6..08b6a58 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -102,7 +102,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
         futId = IgniteUuid.randomUuid();
 
-        log = U.logger(ctx, logRef, GridNearTxFinishFuture.class);
+        log = U.logger(cctx.kernalContext(), logRef, GridNearTxFinishFuture.class);
     }
 
     /** {@inheritDoc} */
@@ -434,7 +434,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
          * @param m Mapping.
          */
         MiniFuture(GridDistributedTxMapping<K, V> m) {
-            super(cctx.kernalContext());
+            super();
 
             this.m = m;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 06e4767..7cad8cc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -884,12 +884,12 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     ) {
         if (state() != PREPARING) {
             if (timedOut())
-                return new GridFinishedFuture<>(cctx.kernalContext(),
+                return new GridFinishedFuture<>(
                     new IgniteTxTimeoutCheckedException("Transaction timed out: " + this));
 
             setRollbackOnly();
 
-            return new GridFinishedFuture<>(cctx.kernalContext(),
+            return new GridFinishedFuture<>(
                 new IgniteCheckedException("Invalid transaction state for prepare [state=" + state() + ", tx=" + this + ']'));
         }
 
@@ -965,7 +965,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             if (prep != null)
                 return (IgniteInternalFuture<IgniteInternalTx>)(IgniteInternalFuture)prep;
 
-            return new GridFinishedFuture<IgniteInternalTx>(cctx.kernalContext(), this);
+            return new GridFinishedFuture<IgniteInternalTx>(this);
         }
 
         final GridDhtTxFinishFuture<K, V> fut = new GridDhtTxFinishFuture<>(cctx, this, /*commit*/true);
@@ -1076,13 +1076,13 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             checkValid();
         }
         catch (IgniteCheckedException e) {
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
 
         final GridCacheReturn<V> ret = new GridCacheReturn<>(false);
 
         if (F.isEmpty(keys))
-            return new GridFinishedFuture<>(cctx.kernalContext(), ret);
+            return new GridFinishedFuture<>(ret);
 
         init();
 
@@ -1108,8 +1108,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
                     return ret;
                 }
-            },
-            cctx.kernalContext());
+            }
+        );
     }
 
     /** {@inheritDoc} */
@@ -1252,7 +1252,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
          * @param tx Transaction.
          */
         private PessimisticPrepareFuture(GridKernalContext ctx, IgniteInternalTx<K, V> tx) {
-            super(ctx);
+            super();
             this.tx = tx;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
index 18fda47..ad61f13 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
@@ -116,7 +116,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
 
         futId = IgniteUuid.randomUuid();
 
-        log = U.logger(ctx, logRef, GridNearTxPrepareFuture.class);
+        log = U.logger(cctx.kernalContext(), logRef, GridNearTxPrepareFuture.class);
     }
 
     /** {@inheritDoc} */
@@ -362,11 +362,13 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
                     }
                 }
                 else {
-                    topFut.syncNotify(false);
-
                     topFut.listenAsync(new CI1<IgniteInternalFuture<Long>>() {
                         @Override public void apply(IgniteInternalFuture<Long> t) {
-                            prepare();
+                            cctx.kernalContext().closure().runLocalSafe(new Runnable() {
+                                @Override public void run() {
+                                    prepare();
+                                }
+                            });
                         }
                     });
                 }
@@ -859,7 +861,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
          */
         MiniFuture(GridDistributedTxMapping<K, V> m,
             ConcurrentLinkedDeque8<GridDistributedTxMapping<K, V>> mappings) {
-            super(cctx.kernalContext());
+            super();
 
             this.m = m;
             this.mappings = mappings;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
index 53b216c..5833270 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
@@ -127,7 +127,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
     public IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
         @Nullable IgniteTxLocalEx<K, V> tx, IgnitePredicate<Cache.Entry<K, V>>[] filter) {
         if (F.isEmpty(keys))
-            return new GridFinishedFuture<>(ctx.kernalContext(), true);
+            return new GridFinishedFuture<>(true);
 
         GridLocalLockFuture<K, V> fut = new GridLocalLockFuture<>(ctx, keys, tx, this, timeout, filter);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
index d4918c1..2048e5d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalLockFuture.java
@@ -112,7 +112,7 @@ public final class GridLocalLockFuture<K, V> extends GridFutureAdapter<Boolean>
         GridLocalCache<K, V> cache,
         long timeout,
         IgnitePredicate<Cache.Entry<K, V>>[] filter) {
-        super(cctx.kernalContext());
+        super();
 
         assert keys != null;
         assert cache != null;
@@ -131,7 +131,7 @@ public final class GridLocalLockFuture<K, V> extends GridFutureAdapter<Boolean>
 
         entries = new ArrayList<>(keys.size());
 
-        log = U.logger(ctx, logRef, GridLocalLockFuture.class);
+        log = U.logger(cctx.kernalContext(), logRef, GridLocalLockFuture.class);
 
         if (timeout > 0) {
             timeoutObj = new LockTimeoutObject();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
index 6727f7e..da7f73b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
@@ -111,10 +111,10 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
         try {
             prepare();
 
-            return new GridFinishedFuture<IgniteInternalTx<K, V>>(cctx.kernalContext(), this);
+            return new GridFinishedFuture<IgniteInternalTx<K, V>>(this);
         }
         catch (IgniteCheckedException e) {
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
     }
 
@@ -153,7 +153,7 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
         catch (IgniteCheckedException e) {
             state(UNKNOWN);
 
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
 
         GridLocalTxFuture<K, V> fut = this.fut.get();
@@ -185,10 +185,10 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
 
             state(ROLLED_BACK);
 
-            return new GridFinishedFuture<IgniteInternalTx>(cctx.kernalContext(), this);
+            return new GridFinishedFuture<IgniteInternalTx>(this);
         }
         catch (IgniteCheckedException e) {
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java
index 66a5eb2..b9934fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTxFuture.java
@@ -83,7 +83,7 @@ final class GridLocalTxFuture<K, V> extends GridFutureAdapter<IgniteInternalTx<K
     GridLocalTxFuture(
         GridCacheSharedContext<K, V> cctx,
         GridLocalTx<K, V> tx) {
-        super(cctx.kernalContext());
+        super();
 
         assert cctx != null;
         assert tx != null;
@@ -91,7 +91,7 @@ final class GridLocalTxFuture<K, V> extends GridFutureAdapter<IgniteInternalTx<K
         this.cctx = cctx;
         this.tx = tx;
 
-        log = U.logger(ctx, logRef,  GridLocalTxFuture.class);
+        log = U.logger(cctx.kernalContext(), logRef,  GridLocalTxFuture.class);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index c3da493..5988069 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -1554,7 +1554,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         boolean invalidate,
         long accessTtl,
         IgnitePredicate<Cache.Entry<K, V>>[] filter) {
-        return new GridFinishedFutureEx<>(new UnsupportedOperationException("Locks are not supported for " +
+        return new GridFinishedFuture<>(new UnsupportedOperationException("Locks are not supported for " +
             "CacheAtomicityMode.ATOMIC mode (use CacheAtomicityMode.TRANSACTIONAL instead)"));
     }
 
@@ -1563,7 +1563,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     @Override public IgniteInternalFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys,
         long timeout,
         @Nullable IgnitePredicate<Cache.Entry<K, V>>... filter) {
-        return new GridFinishedFutureEx<>(new UnsupportedOperationException("Locks are not supported for " +
+        return new GridFinishedFuture<>(new UnsupportedOperationException("Locks are not supported for " +
             "CacheAtomicityMode.ATOMIC mode (use CacheAtomicityMode.TRANSACTIONAL instead)"));
     }
 
@@ -1613,7 +1613,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                         @Override public IgniteInternalFuture apply(Object t, Exception e) {
                             return ctx.closures().callLocalSafe(op);
                         }
-                    }, ctx.kernalContext());
+                    });
 
                 saveFuture(holder, f);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
index 4f668e6..1093c4e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
@@ -56,7 +56,7 @@ public class GridCacheDistributedFieldsQueryFuture
         GridCacheQueryBean qry, Iterable<ClusterNode> nodes) {
         super((GridCacheContext<Object, Object>)ctx, reqId, qry, nodes);
 
-        metaFut = new GridFutureAdapter<>(ctx.kernalContext());
+        metaFut = new GridFutureAdapter<>();
 
         if (!qry.query().includeMetadata())
             metaFut.onDone();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java
index 93a8fd9..222abc1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java
@@ -48,6 +48,6 @@ public class GridCacheFieldsQueryErrorFuture extends GridCacheQueryErrorFuture<L
      * @return Metadata.
      */
     public IgniteInternalFuture<List<GridQueryFieldMetadata>> metadata() {
-        return new GridFinishedFuture<>(ctx, incMeta ? Collections.<GridQueryFieldMetadata>emptyList() : null);
+        return new GridFinishedFuture<>(incMeta ? Collections.<GridQueryFieldMetadata>emptyList() : null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
index 782d9da..70d97fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
@@ -52,7 +52,7 @@ public class GridCacheLocalFieldsQueryFuture
     public GridCacheLocalFieldsQueryFuture(GridCacheContext<?, ?> ctx, GridCacheQueryBean qry) {
         super((GridCacheContext<Object, Object>)ctx, qry);
 
-        metaFut = new GridFutureAdapter<>(ctx.kernalContext());
+        metaFut = new GridFutureAdapter<>();
 
         if (!qry.query().includeMetadata())
             metaFut.onDone();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
index 3d8efbf..8831b5d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
@@ -63,7 +63,7 @@ public class GridCacheLocalQueryFuture<K, V, R> extends GridCacheQueryFutureAdap
      * Executes query runnable.
      */
     void execute() {
-        fut = ctx.closure().runLocalSafe(run, true);
+        fut = cctx.kernalContext().closure().runLocalSafe(run, true);
     }
 
     /** {@inheritDoc} */
@@ -130,7 +130,7 @@ public class GridCacheLocalQueryFuture<K, V, R> extends GridCacheQueryFutureAdap
                 rdc,
                 qry.query(),
                 GridCacheLocalQueryFuture.this,
-                ctx.localNodeId(),
+                cctx.localNodeId(),
                 cctx.io().nextIoId(),
                 qry.query().includeMetadata(),
                 true,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
index f4fcb37..627d6b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryErrorFuture.java
@@ -36,7 +36,7 @@ public class GridCacheQueryErrorFuture<T> extends GridFinishedFuture<Collection<
      * @param err Error.
      */
     public GridCacheQueryErrorFuture(GridKernalContext ctx, Throwable err) {
-        super(ctx, err);
+        super(err);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
index 0748936..9e4e3d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -101,13 +101,13 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
      * @param loc Local query or not.
      */
     protected GridCacheQueryFutureAdapter(GridCacheContext<K, V> cctx, GridCacheQueryBean qry, boolean loc) {
-        super(cctx.kernalContext());
+        super();
 
         this.cctx = cctx;
         this.qry = qry;
         this.loc = loc;
 
-        log = U.logger(ctx, logRef, GridCacheQueryFutureAdapter.class);
+        log = U.logger(cctx.kernalContext(), logRef, GridCacheQueryFutureAdapter.class);
 
         startTime = U.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 8fa48aa..b4fbef5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -1511,7 +1511,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             fut = futs.get(qryInfo.requestId());
 
             if (fut == null) {
-                futs.put(qryInfo.requestId(), fut = new GridFutureAdapter<>(cctx.kernalContext()));
+                futs.put(qryInfo.requestId(), fut = new GridFutureAdapter<>());
 
                 exec = true;
             }
@@ -1642,7 +1642,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
             if (fut == null) {
                 resMap.put(qryInfo.requestId(), fut =
-                    new GridFutureAdapter<>(cctx.kernalContext()));
+                    new GridFutureAdapter<>());
 
                 exec = true;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index abdb99c..409c381 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -959,7 +959,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
         GridFutureAdapter<IgniteInternalTx> fut = finFut.get();
 
         if (fut == null) {
-            fut = new GridFutureAdapter<IgniteInternalTx>(cctx.kernalContext()) {
+            fut = new GridFutureAdapter<IgniteInternalTx>() {
                 @Override public String toString() {
                     return S.toString(GridFutureAdapter.class, this, "tx", IgniteTxAdapter.this);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/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 b29f721..5115781 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
@@ -173,10 +173,9 @@ public class IgniteTxHandler<K, V> {
         final IgniteInClosure<GridNearTxPrepareResponse<K, V>> completeCb
     ) {
 
-        IgniteInternalFuture<Object> fut = new GridFinishedFutureEx<>(); // TODO force preload keys.
+        IgniteInternalFuture<Object> fut = new GridFinishedFuture<>(); // TODO force preload keys.
 
         return new GridEmbeddedFuture<>(
-            ctx.kernalContext(),
             fut,
             new C2<Object, Exception, IgniteInternalFuture<IgniteInternalTx<K, V>>>() {
                 @Override public IgniteInternalFuture<IgniteInternalTx<K, V>> apply(Object o, Exception ex) {
@@ -240,7 +239,7 @@ public class IgniteTxHandler<K, V> {
                 e.unmarshal(ctx, false, ctx.deploy().globalLoader());
         }
         catch (IgniteCheckedException e) {
-            return new GridFinishedFuture<>(ctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
 
         GridDhtTxLocal<K, V> tx;
@@ -343,7 +342,7 @@ public class IgniteTxHandler<K, V> {
             return fut;
         }
         else
-            return new GridFinishedFuture<>(ctx.kernalContext(), (IgniteInternalTx<K, V>)null);
+            return new GridFinishedFuture<>((IgniteInternalTx<K, V>)null);
     }
 
     /**
@@ -443,7 +442,7 @@ public class IgniteTxHandler<K, V> {
 
         // Transaction on local cache only.
         if (locTx != null && !locTx.nearLocallyMapped() && !locTx.colocatedLocallyMapped())
-            return new GridFinishedFutureEx<IgniteInternalTx>(locTx);
+            return new GridFinishedFuture<IgniteInternalTx>(locTx);
 
         if (log.isDebugEnabled())
             log.debug("Processing near tx finish request [nodeId=" + nodeId + ", req=" + req + "]");
@@ -607,7 +606,7 @@ public class IgniteTxHandler<K, V> {
                 return rollbackFut;
             }
 
-            return new GridFinishedFuture<>(ctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
     }
 
@@ -637,7 +636,7 @@ public class IgniteTxHandler<K, V> {
             if (tx != null)
                 return tx.rollbackAsync();
 
-            return new GridFinishedFuture<>(ctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 8bc5230..67d33fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -356,14 +356,14 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     for (K key : keys)
                         c.apply(key, null);
 
-                    return new GridFinishedFuture<>(cctx.kernalContext(), false);
+                    return new GridFinishedFuture<>(false);
                 }
 
-                return new GridFinishedFuture<>(cctx.kernalContext(),
+                return new GridFinishedFuture<>(
                     cacheCtx.store().loadAllFromStore(this, keys, c));
             }
             catch (IgniteCheckedException e) {
-                return new GridFinishedFuture<>(cctx.kernalContext(), e);
+                return new GridFinishedFuture<>(e);
             }
         }
         else
@@ -1413,7 +1413,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
 
         final Collection<K> loaded = new HashSet<>();
 
-        return new GridEmbeddedFuture<>(cctx.kernalContext(),
+        return new GridEmbeddedFuture<>(
             loadMissing(
                 cacheCtx,
                 true, false, missedMap.keySet(), deserializePortable, skipVals, new CI2<K, V>() {
@@ -1564,7 +1564,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
 
                     return map;
                 }
-            });
+            },
+        false);
     }
 
     /** {@inheritDoc} */
@@ -1575,7 +1576,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
         final boolean deserializePortable,
         final boolean skipVals) {
         if (F.isEmpty(keys))
-            return new GridFinishedFuture<>(cctx.kernalContext(), Collections.<K, V>emptyMap());
+            return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
 
         init();
 
@@ -1605,7 +1606,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                 skipVals);
 
             if (single && missed.isEmpty())
-                return new GridFinishedFuture<>(cctx.kernalContext(), retMap);
+                return new GridFinishedFuture<>(retMap);
 
             // Handle locks.
             if (pessimistic() && !readCommitted() && !groupLock() && !skipVals) {
@@ -1712,7 +1713,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                         if (!missed.isEmpty() && (cacheCtx.isReplicated() || cacheCtx.isLocal()))
                             return checkMissed(cacheCtx, retMap, missed, null, deserializePortable, skipVals);
 
-                        return new GridFinishedFuture<>(cctx.kernalContext(), Collections.<K, V>emptyMap());
+                        return new GridFinishedFuture<>(Collections.<K, V>emptyMap());
                     }
                 };
 
@@ -1729,24 +1730,23 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                         IgniteInternalFuture<Map<K, V>> fut1 = plc2.apply(fut.get(), null);
 
                         return fut1.isDone() ?
-                            new GridFinishedFutureEx<>(finClos.apply(fut1.get(), null)) :
-                            new GridEmbeddedFuture<>(cctx.kernalContext(), fut1, finClos);
+                            new GridFinishedFuture<>(finClos.apply(fut1.get(), null)) :
+                            new GridEmbeddedFuture<>(fut1, finClos, false);
                     }
                     catch (GridClosureException e) {
-                        return new GridFinishedFuture<>(cctx.kernalContext(), e.unwrap());
+                        return new GridFinishedFuture<>(e.unwrap());
                     }
                     catch (IgniteCheckedException e) {
                         try {
                             return plc2.apply(false, e);
                         }
                         catch (Exception e1) {
-                            return new GridFinishedFuture<>(cctx.kernalContext(), e1);
+                            return new GridFinishedFuture<>(e1);
                         }
                     }
                 }
                 else {
                     return new GridEmbeddedFuture<>(
-                        cctx.kernalContext(),
                         fut,
                         plc2,
                         finClos);
@@ -1764,17 +1764,16 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                                 it.remove();
 
                     if (missed.isEmpty())
-                        return new GridFinishedFuture<>(cctx.kernalContext(), retMap);
+                        return new GridFinishedFuture<>(retMap);
 
                     return new GridEmbeddedFuture<>(
-                        cctx.kernalContext(),
                         // First future.
                         checkMissed(cacheCtx, retMap, missed, redos, deserializePortable, skipVals),
                         // Closure that returns another future, based on result from first.
                         new PMC<Map<K, V>>() {
                             @Override public IgniteInternalFuture<Map<K, V>> postMiss(Map<K, V> map) {
                                 if (redos.isEmpty())
-                                    return new GridFinishedFuture<>(cctx.kernalContext(),
+                                    return new GridFinishedFuture<>(
                                         Collections.<K, V>emptyMap());
 
                                 if (log.isDebugEnabled())
@@ -1807,13 +1806,13 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     );
                 }
 
-                return new GridFinishedFuture<>(cctx.kernalContext(), retMap);
+                return new GridFinishedFuture<>(retMap);
             }
         }
         catch (IgniteCheckedException e) {
             setRollbackOnly();
 
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
     }
 
@@ -1934,7 +1933,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
             addActiveCache(cacheCtx);
         }
         catch (IgniteCheckedException e) {
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
 
         Set<K> skipped = null;
@@ -2212,7 +2211,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
             }
         }
         catch (IgniteCheckedException e) {
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
 
         if (missedForLoad != null) {
@@ -2240,7 +2239,6 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                 });
 
             return new GridEmbeddedFuture<>(
-                cctx.kernalContext(),
                 fut,
                 new C2<Boolean, Exception, Set<K>>() {
                     @Override public Set<K> apply(Boolean b, Exception e) {
@@ -2249,11 +2247,12 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
 
                         return Collections.emptySet();
                     }
-                }
+                },
+                false
             );
         }
 
-        return new GridFinishedFuture<>(cctx.kernalContext(), skipped);
+        return new GridFinishedFuture<>(skipped);
     }
 
     /**
@@ -2495,7 +2494,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     }
                 }
                 catch (IgniteException e) {
-                    return new GridFinishedFuture<>(cctx.kernalContext(), e);
+                    return new GridFinishedFuture<>(e);
                 }
             }
             else
@@ -2512,7 +2511,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     }
                 }
                 catch (IgniteException e) {
-                    return new GridFinishedFuture<>(cctx.kernalContext(), e);
+                    return new GridFinishedFuture<>(e);
                 }
             }
             else
@@ -2534,7 +2533,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
             checkValid();
         }
         catch (IgniteCheckedException e) {
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
 
         init();
@@ -2547,10 +2546,10 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     commit();
                 }
                 catch (IgniteCheckedException e) {
-                    return new GridFinishedFuture<>(cctx.kernalContext(), e);
+                    return new GridFinishedFuture<>(e);
                 }
 
-            return new GridFinishedFuture<>(cctx.kernalContext(), ret.success(true));
+            return new GridFinishedFuture<>(ret.success(true));
         }
 
         try {
@@ -2632,22 +2631,22 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                         return plc1.apply(fut.get(), null);
                     }
                     catch (GridClosureException e) {
-                        return new GridFinishedFuture<>(cctx.kernalContext(), e.unwrap());
+                        return new GridFinishedFuture<>(e.unwrap());
                     }
                     catch (IgniteCheckedException e) {
                         try {
                             return plc1.apply(false, e);
                         }
                         catch (Exception e1) {
-                            return new GridFinishedFuture<>(cctx.kernalContext(), e1);
+                            return new GridFinishedFuture<>(e1);
                         }
                     }
                 }
                 else
                     return new GridEmbeddedFuture<>(
                         fut,
-                        plc1,
-                        cctx.kernalContext());
+                        plc1
+                    );
             }
             else {
                 if (implicit()) {
@@ -2659,7 +2658,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                         loadFut.get();
                     }
                     catch (IgniteCheckedException e) {
-                        return new GridFinishedFutureEx<>(new GridCacheReturn<V>(), e);
+                        return new GridFinishedFuture<>(e);
                     }
 
                     return commitAsync().chain(new CX1<IgniteInternalFuture<IgniteInternalTx>, GridCacheReturn<V>>() {
@@ -2683,7 +2682,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
         catch (IgniteCheckedException e) {
             setRollbackOnly();
 
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
     }
 
@@ -2740,7 +2739,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     keys0 = null;
             }
             catch (IgniteException e) {
-                return new GridFinishedFuture<>(cctx.kernalContext(), e);
+                return new GridFinishedFuture<>(e);
             }
         }
         else
@@ -2757,7 +2756,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
             checkValid();
         }
         catch (IgniteCheckedException e) {
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
 
         final GridCacheReturn<V> ret = new GridCacheReturn<>(false);
@@ -2768,11 +2767,11 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     commit();
                 }
                 catch (IgniteCheckedException e) {
-                    return new GridFinishedFuture<>(cctx.kernalContext(), e);
+                    return new GridFinishedFuture<>(e);
                 }
             }
 
-            return new GridFinishedFuture<>(cctx.kernalContext(), ret.success(true));
+            return new GridFinishedFuture<>(ret.success(true));
         }
 
         init();
@@ -2856,22 +2855,22 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                         return plc1.apply(fut.get(), null);
                     }
                     catch (GridClosureException e) {
-                        return new GridFinishedFuture<>(cctx.kernalContext(), e.unwrap());
+                        return new GridFinishedFuture<>(e.unwrap());
                     }
                     catch (IgniteCheckedException e) {
                         try {
                             return plc1.apply(false, e);
                         }
                         catch (Exception e1) {
-                            return new GridFinishedFuture<>(cctx.kernalContext(), e1);
+                            return new GridFinishedFuture<>(e1);
                         }
                     }
                 }
                 else
                     return new GridEmbeddedFuture<>(
                         fut,
-                        plc1,
-                        cctx.kernalContext());
+                        plc1
+                    );
             }
             else {
                 if (implicit()) {
@@ -2900,7 +2899,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
         catch (IgniteCheckedException e) {
             setRollbackOnly();
 
-            return new GridFinishedFuture<>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<>(e);
         }
     }
 
@@ -3005,12 +3004,12 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     isInvalidate(),
                     -1L,
                     CU.<K, V>empty()) :
-                new GridFinishedFuture<>(cctx.kernalContext());
+                new GridFinishedFuture<>();
         }
         catch (IgniteCheckedException e) {
             setRollbackOnly();
 
-            return new GridFinishedFuture<Object>(cctx.kernalContext(), e);
+            return new GridFinishedFuture<Object>(e);
         }
     }
 
@@ -3467,7 +3466,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
 
                 rollback = false;
 
-                return new GridFinishedFuture<>(cctx.kernalContext(), r);
+                return new GridFinishedFuture<>(r);
             }
             catch (final IgniteCheckedException ex) {
                 if (commit && commitAfterLock())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
index 95f54a1..eef7cc6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/TransactionProxyImpl.java
@@ -295,7 +295,7 @@ public class TransactionProxyImpl<K, V> implements TransactionProxy, Externaliza
      * @param res Result to convert to finished future.
      */
     private void save(Object res) {
-        asyncRes = new IgniteFinishedFutureImplEx<>(res);
+        asyncRes = new IgniteFinishedFutureImpl<>(res);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c1b46951/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
index 657539c..28b07e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
@@ -472,10 +472,10 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         try {
             if (job == null)
-                return new GridFinishedFuture<>(ctx);
+                return new GridFinishedFuture<>();
 
             if (F.isEmpty(nodes))
-                return new GridFinishedFuture<>(ctx, U.emptyTopologyException());
+                return new GridFinishedFuture<>(U.emptyTopologyException());
 
             ctx.task().setThreadContext(TC_NO_FAILOVER, true);
             ctx.task().setThreadContext(TC_SUBGRID, nodes);
@@ -504,10 +504,10 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         try {
             if (F.isEmpty(jobs))
-                return new GridFinishedFuture<>(ctx);
+                return new GridFinishedFuture<>();
 
             if (F.isEmpty(nodes))
-                return new GridFinishedFuture<>(ctx, U.emptyTopologyException());
+                return new GridFinishedFuture<>(U.emptyTopologyException());
 
             ctx.task().setThreadContext(TC_NO_FAILOVER, true);
             ctx.task().setThreadContext(TC_SUBGRID, nodes);
@@ -585,7 +585,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         try {
             if (F.isEmpty(nodes))
-                return new GridFinishedFuture<>(ctx, U.emptyTopologyException());
+                return new GridFinishedFuture<>(U.emptyTopologyException());
 
             ctx.task().setThreadContext(TC_SUBGRID, nodes);
 
@@ -608,7 +608,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         try {
             if (F.isEmpty(nodes))
-                return new GridFinishedFuture<>(ctx, U.emptyTopologyException());
+                return new GridFinishedFuture<>(U.emptyTopologyException());
 
             ctx.task().setThreadContext(TC_SUBGRID, nodes);
             ctx.task().setThreadContext(TC_NO_FAILOVER, true);
@@ -731,7 +731,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      */
     private IgniteInternalFuture<?> runLocal(@Nullable final Runnable c, GridClosurePolicy plc) throws IgniteCheckedException {
         if (c == null)
-            return new GridFinishedFuture(ctx);
+            return new GridFinishedFuture();
 
         enterBusy();
 
@@ -834,18 +834,18 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                 try {
                     c.run();
 
-                    return new GridFinishedFuture(ctx);
+                    return new GridFinishedFuture();
                 }
                 catch (Throwable t) {
                     if (t instanceof Error)
                         U.error(log, "Closure execution failed with error.", t);
 
-                    return new GridFinishedFuture(ctx, U.cast(t));
+                    return new GridFinishedFuture(U.cast(t));
                 }
             }
             // If failed for other reasons - return error future.
             else
-                return new GridFinishedFuture(ctx, U.cast(e));
+                return new GridFinishedFuture(U.cast(e));
         }
     }
 
@@ -868,7 +868,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      */
     private <R> IgniteInternalFuture<R> callLocal(@Nullable final Callable<R> c, GridClosurePolicy plc) throws IgniteCheckedException {
         if (c == null)
-            return new GridFinishedFuture<>(ctx);
+            return new GridFinishedFuture<>();
 
         enterBusy();
 
@@ -964,16 +964,16 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     ", closure=" + c + ']');
 
                 try {
-                    return new GridFinishedFuture<>(ctx, c.call());
+                    return new GridFinishedFuture<>(c.call());
                 }
                 // If failed again locally - return error future.
                 catch (Exception e2) {
-                    return new GridFinishedFuture<>(ctx, U.cast(e2));
+                    return new GridFinishedFuture<>(U.cast(e2));
                 }
             }
             // If failed for other reasons - return error future.
             else
-                return new GridFinishedFuture<>(ctx, U.cast(e));
+                return new GridFinishedFuture<>(U.cast(e));
         }
     }