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

[10/16] incubator-ignite git commit: # IGNITE-26 Created InternalFuture.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index d041d0f..5dab909 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -124,7 +124,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(InternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -149,7 +149,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
 
         int type = evt.type();
 
-        for (IgniteFuture<?> f : futures()) {
+        for (InternalFuture<?> f : futures()) {
             if (isMini(f)) {
                 MiniFuture mini = (MiniFuture)f;
 
@@ -173,7 +173,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
      */
     @SuppressWarnings( {"unchecked"})
     public void onResult(UUID nodeId, GridDhtForceKeysResponse<K, V> res) {
-        for (IgniteFuture<Object> f : futures())
+        for (InternalFuture<Object> f : futures())
             if (isMini(f)) {
                 MiniFuture mini = (MiniFuture)f;
 
@@ -542,7 +542,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
 
                 return true;
             }
-            catch (GridInterruptedException e) {
+            catch (InternalInterruptedException e) {
                 // Fail.
                 onDone(e);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
index ac0b7e6..1e30854 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
@@ -161,7 +161,7 @@ public class GridDhtPartitionDemandPool<K, V> {
     /**
      * @return Future for {@link GridCachePreloadMode#SYNC} mode.
      */
-    IgniteFuture<?> syncFuture() {
+    InternalFuture<?> syncFuture() {
         return syncFut;
     }
 
@@ -206,8 +206,8 @@ public class GridDhtPartitionDemandPool<K, V> {
             if (log.isDebugEnabled())
                 log.debug("Forcing preload event for future: " + exchFut);
 
-            exchFut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                @Override public void apply(IgniteFuture<Long> t) {
+            exchFut.listenAsync(new CI1<InternalFuture<Long>>() {
+                @Override public void apply(InternalFuture<Long> t) {
                     cctx.shared().exchange().forcePreloadExchange(exchFut);
                 }
             });
@@ -362,8 +362,8 @@ public class GridDhtPartitionDemandPool<K, V> {
 
             obj = new GridTimeoutObjectAdapter(delay) {
                 @Override public void onTimeout() {
-                    exchFut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                        @Override public void apply(IgniteFuture<Long> f) {
+                    exchFut.listenAsync(new CI1<InternalFuture<Long>>() {
+                        @Override public void apply(InternalFuture<Long> f) {
                             cctx.shared().exchange().forcePreloadExchange(exchFut);
                         }
                     });
@@ -481,10 +481,10 @@ public class GridDhtPartitionDemandPool<K, V> {
          * @param entry Preloaded entry.
          * @param topVer Topology version.
          * @return {@code False} if partition has become invalid during preloading.
-         * @throws GridInterruptedException If interrupted.
+         * @throws InternalInterruptedException If interrupted.
          */
         private boolean preloadEntry(ClusterNode pick, int p, GridCacheEntryInfo<K, V> entry, long topVer)
-            throws IgniteCheckedException, GridInterruptedException {
+            throws IgniteCheckedException, InternalInterruptedException {
             try {
                 GridCacheEntryEx<K, V> cached = null;
 
@@ -542,7 +542,7 @@ public class GridDhtPartitionDemandPool<K, V> {
                     return false;
                 }
             }
-            catch (GridInterruptedException e) {
+            catch (InternalInterruptedException e) {
                 throw e;
             }
             catch (IgniteCheckedException e) {
@@ -807,12 +807,12 @@ public class GridDhtPartitionDemandPool<K, V> {
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, GridInterruptedException {
+        @Override protected void body() throws InterruptedException, InternalInterruptedException {
             try {
                 int preloadOrder = cctx.config().getPreloadOrder();
 
                 if (preloadOrder > 0) {
-                    IgniteFuture<?> fut = cctx.kernalContext().cache().orderedPreloadFuture(preloadOrder);
+                    InternalFuture<?> fut = cctx.kernalContext().cache().orderedPreloadFuture(preloadOrder);
 
                     try {
                         if (fut != null) {
@@ -823,7 +823,7 @@ public class GridDhtPartitionDemandPool<K, V> {
                             fut.get();
                         }
                     }
-                    catch (GridInterruptedException ignored) {
+                    catch (InternalInterruptedException ignored) {
                         if (log.isDebugEnabled())
                             log.debug("Failed to wait for ordered preload future (grid is stopping): " +
                                 "[cacheName=" + cctx.name() + ", preloadOrder=" + preloadOrder + ']');
@@ -903,7 +903,7 @@ public class GridDhtPartitionDemandPool<K, V> {
                                         missed.addAll(set);
                                     }
                                 }
-                                catch (GridInterruptedException e) {
+                                catch (InternalInterruptedException e) {
                                     throw e;
                                 }
                                 catch (ClusterTopologyException e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
index 963ad4c..17077e7 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionSupplyPool.java
@@ -198,7 +198,7 @@ class GridDhtPartitionSupplyPool<K, V> {
         }
 
         /** {@inheritDoc} */
-        @Override protected void body() throws InterruptedException, GridInterruptedException {
+        @Override protected void body() throws InterruptedException, InternalInterruptedException {
             while (!isCancelled()) {
                 DemandMessage<K, V> msg = poll(queue, this);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 0a95411..7cd3e57 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -122,7 +122,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
     /** */
     @SuppressWarnings({"FieldCanBeLocal", "UnusedDeclaration"})
     @GridToStringInclude
-    private volatile IgniteFuture<?> partReleaseFut;
+    private volatile InternalFuture<?> partReleaseFut;
 
     /** */
     private final Object mux = new Object();
@@ -369,7 +369,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
     /**
      * @return Init future.
      */
-    IgniteFuture<?> initFuture() {
+    InternalFuture<?> initFuture() {
         return initFut;
     }
 
@@ -396,9 +396,9 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
     /**
      * Starts activity.
      *
-     * @throws GridInterruptedException If interrupted.
+     * @throws InternalInterruptedException If interrupted.
      */
-    public void init() throws GridInterruptedException {
+    public void init() throws InternalInterruptedException {
         assert oldestNode.get() != null;
 
         if (init.compareAndSet(false, true)) {
@@ -445,7 +445,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
                     cacheCtx.preloader().updateLastExchangeFuture(this);
                 }
 
-                IgniteFuture<?> partReleaseFut = cctx.partitionReleaseFuture(topVer);
+                InternalFuture<?> partReleaseFut = cctx.partitionReleaseFuture(topVer);
 
                 // Assign to class variable so it will be included into toString() method.
                 this.partReleaseFut = partReleaseFut;
@@ -486,7 +486,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
                     top.beforeExchange(exchId);
                 }
             }
-            catch (GridInterruptedException e) {
+            catch (InternalInterruptedException e) {
                 onDone(e);
 
                 throw e;
@@ -719,8 +719,8 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
             }
         }
         else {
-            initFut.listenAsync(new CI1<IgniteFuture<Boolean>>() {
-                @Override public void apply(IgniteFuture<Boolean> t) {
+            initFut.listenAsync(new CI1<InternalFuture<Boolean>>() {
+                @Override public void apply(InternalFuture<Boolean> t) {
                     try {
                         if (!t.get()) // Just to check if there was an error.
                             return;
@@ -817,8 +817,8 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
 
         assert exchId.topologyVersion() == msg.topologyVersion();
 
-        initFut.listenAsync(new CI1<IgniteFuture<Boolean>>() {
-            @Override public void apply(IgniteFuture<Boolean> t) {
+        initFut.listenAsync(new CI1<InternalFuture<Boolean>>() {
+            @Override public void apply(InternalFuture<Boolean> t) {
                 assert msg.lastVersion() != null;
 
                 cctx.versions().onReceived(nodeId, msg.lastVersion());
@@ -877,8 +877,8 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
 
         try {
             // Wait for initialization part of this future to complete.
-            initFut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> f) {
+            initFut.listenAsync(new CI1<InternalFuture<?>>() {
+                @Override public void apply(InternalFuture<?> f) {
                     if (isDone())
                         return;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 9706727..766d10c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -253,12 +253,12 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
     /**
      * @return Start future.
      */
-    @Override public IgniteFuture<Object> startFuture() {
+    @Override public InternalFuture<Object> startFuture() {
         return startFut;
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> syncFuture() {
+    @Override public InternalFuture<?> syncFuture() {
         return demandPool.syncFuture();
     }
 
@@ -307,13 +307,13 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
      * @param msg Force keys message.
      */
     private void processForceKeysRequest(final ClusterNode node, final GridDhtForceKeysRequest<K, V> msg) {
-        IgniteFuture<?> fut = cctx.mvcc().finishKeys(msg.keys(), msg.topologyVersion());
+        InternalFuture<?> fut = cctx.mvcc().finishKeys(msg.keys(), msg.topologyVersion());
 
         if (fut.isDone())
             processForceKeysRequest0(node, msg);
         else
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> t) {
+            fut.listenAsync(new CI1<InternalFuture<?>>() {
+                @Override public void apply(InternalFuture<?> t) {
                     processForceKeysRequest0(node, msg);
                 }
             });
@@ -418,8 +418,8 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
         if (log.isDebugEnabled())
             log.debug("Processing affinity assignment request [node=" + node + ", req=" + req + ']');
 
-        cctx.affinity().affinityReadyFuture(req.topologyVersion()).listenAsync(new CI1<IgniteFuture<Long>>() {
-            @Override public void apply(IgniteFuture<Long> fut) {
+        cctx.affinity().affinityReadyFuture(req.topologyVersion()).listenAsync(new CI1<InternalFuture<Long>>() {
+            @Override public void apply(InternalFuture<Long> fut) {
                 if (log.isDebugEnabled())
                     log.debug("Affinity is ready for topology version, will send response [topVer=" + topVer +
                         ", node=" + node + ']');
@@ -473,27 +473,27 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
     @Override public GridDhtFuture<Object> request(Collection<? extends K> keys, long topVer) {
         final GridDhtForceKeysFuture<K, V> fut = new GridDhtForceKeysFuture<>(cctx, topVer, keys, this);
 
-        IgniteFuture<?> topReadyFut = cctx.affinity().affinityReadyFuturex(topVer);
+        InternalFuture<?> topReadyFut = cctx.affinity().affinityReadyFuturex(topVer);
 
         if (startFut.isDone() && topReadyFut == null)
             fut.init();
         else {
             if (topReadyFut == null)
-                startFut.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> syncFut) {
+                startFut.listenAsync(new CI1<InternalFuture<?>>() {
+                    @Override public void apply(InternalFuture<?> syncFut) {
                         fut.init();
                     }
                 });
             else {
                 GridCompoundFuture<Object, Object> compound = new GridCompoundFuture<>(cctx.kernalContext());
 
-                compound.add((IgniteFuture<Object>)startFut);
-                compound.add((IgniteFuture<Object>)topReadyFut);
+                compound.add((InternalFuture<Object>)startFut);
+                compound.add((InternalFuture<Object>)topReadyFut);
 
                 compound.markInitialized();
 
-                compound.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> syncFut) {
+                compound.listenAsync(new CI1<InternalFuture<?>>() {
+                    @Override public void apply(InternalFuture<?> syncFut) {
                         fut.init();
                     }
                 });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java
index ce5e19c..a424791 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -325,7 +325,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteFuture<Map<K, V>> getAllAsync(
+    @Override protected InternalFuture<Map<K, V>> getAllAsync(
         @Nullable Collection<? extends K> keys,
         boolean forcePrimary,
         boolean skipTx,
@@ -375,7 +375,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> putAsync(K key,
+    @Override public InternalFuture<V> putAsync(K key,
         V val,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl,
@@ -385,7 +385,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> putxAsync(K key,
+    @Override public InternalFuture<Boolean> putxAsync(K key,
         V val,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl,
@@ -399,7 +399,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putIfAbsentAsync(K key, V val) {
+    @Override public InternalFuture<V> putIfAbsentAsync(K key, V val) {
         return dht.putIfAbsentAsync(key, val);
     }
 
@@ -409,7 +409,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxIfAbsentAsync(K key, V val) {
+    @Override public InternalFuture<Boolean> putxIfAbsentAsync(K key, V val) {
         return dht.putxIfAbsentAsync(key, val);
     }
 
@@ -419,7 +419,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(K key, V val) {
+    @Override public InternalFuture<V> replaceAsync(K key, V val) {
         return dht.replaceAsync(key, val);
     }
 
@@ -429,7 +429,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(K key, V val) {
+    @Override public InternalFuture<Boolean> replacexAsync(K key, V val) {
         return dht.replacexAsync(key, val);
     }
 
@@ -439,7 +439,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
+    @Override public InternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
         return dht.replaceAsync(key, oldVal, newVal);
     }
 
@@ -455,13 +455,13 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
+    @Override public InternalFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
         return dht.removexAsync(key, val);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
+    @Override public InternalFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
         return dht.replacexAsync(key, oldVal, newVal);
     }
 
@@ -472,7 +472,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
+    @Override public InternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
         return dht.putAllAsync(m, filter);
     }
@@ -483,7 +483,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap) throws IgniteCheckedException {
+    @Override public InternalFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap) throws IgniteCheckedException {
         return dht.putAllDrAsync(drMap);
     }
 
@@ -499,7 +499,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> transformAsync(K key,
+    @Override public InternalFuture<?> transformAsync(K key,
         IgniteClosure<V, V> transformer,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl) {
@@ -512,7 +512,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> transformAllAsync(@Nullable Map<? extends K, ? extends IgniteClosure<V, V>> m) {
+    @Override public InternalFuture<?> transformAllAsync(@Nullable Map<? extends K, ? extends IgniteClosure<V, V>> m) {
         return dht.transformAllAsync(m);
     }
 
@@ -525,7 +525,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> removeAsync(K key,
+    @Override public InternalFuture<V> removeAsync(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
         return dht.removeAsync(key, entry, filter);
@@ -538,7 +538,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(Collection<? extends K> keys,
+    @Override public InternalFuture<?> removeAllAsync(Collection<? extends K> keys,
         IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
         return dht.removeAllAsync(keys, filter);
     }
@@ -552,7 +552,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> removexAsync(K key,
+    @Override public InternalFuture<Boolean> removexAsync(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
         return dht.removexAsync(key, entry, filter);
@@ -564,7 +564,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(K key, V val) {
+    @Override public InternalFuture<Boolean> removeAsync(K key, V val) {
         return dht.removeAsync(key, val);
     }
 
@@ -574,7 +574,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+    @Override public InternalFuture<?> removeAllAsync(IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
         return dht.removeAllAsync(keySet(filter));
     }
 
@@ -584,12 +584,12 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException {
+    @Override public InternalFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException {
         return dht.removeAllDrAsync(drMap);
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
+    @Override protected InternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
         long timeout,
         @Nullable GridCacheTxLocalEx<K, V> tx,
         boolean isInvalidate,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheAdapter.java
index b785103..267f589 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -161,10 +161,10 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "RedundantCast"})
-    @Override public IgniteFuture<Object> readThroughAllAsync(Collection<? extends K> keys, boolean reload,
+    @Override public InternalFuture<Object> readThroughAllAsync(Collection<? extends K> keys, boolean reload,
         GridCacheTxEx<K, V> tx, IgnitePredicate<GridCacheEntry<K, V>>[] filter, @Nullable UUID subjId, String taskName,
         IgniteBiInClosure<K, V> vis) {
-        return (IgniteFuture)loadAsync(tx, keys, reload, false, filter, subjId, taskName, true);
+        return (InternalFuture)loadAsync(tx, keys, reload, false, filter, subjId, taskName, true);
     }
 
     /** {@inheritDoc} */
@@ -177,7 +177,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys,
+    @Override public InternalFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
         GridCompoundFuture fut = new GridCompoundFuture(ctx.kernalContext());
 
@@ -216,7 +216,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked"})
-    @Override public IgniteFuture<?> reloadAllAsync() {
+    @Override public InternalFuture<?> reloadAllAsync() {
         GridCompoundFuture fut = new GridCompoundFuture(ctx.kernalContext());
 
         fut.add(super.reloadAllAsync());
@@ -229,7 +229,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked"})
-    @Override public IgniteFuture<?> reloadAllAsync(@Nullable IgnitePredicate<GridCacheEntry<K, V>> filter) {
+    @Override public InternalFuture<?> reloadAllAsync(@Nullable IgnitePredicate<GridCacheEntry<K, V>> filter) {
         GridCompoundFuture fut = new GridCompoundFuture(ctx.kernalContext());
 
         fut.add(super.reloadAllAsync());
@@ -248,7 +248,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
      * @param filter Filter.
      * @return Loaded values.
      */
-    public IgniteFuture<Map<K, V>> loadAsync(@Nullable GridCacheTxEx tx, @Nullable Collection<? extends K> keys,
+    public InternalFuture<Map<K, V>> loadAsync(@Nullable GridCacheTxEx tx, @Nullable Collection<? extends K> keys,
         boolean reload, boolean forcePrimary, @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter,
         @Nullable UUID subjId, String taskName, boolean deserializePortable) {
         if (F.isEmpty(keys))
@@ -274,7 +274,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> loadCacheAsync(IgniteBiPredicate<K, V> p, long ttl, Object[] args) {
+    @Override public InternalFuture<?> loadCacheAsync(IgniteBiPredicate<K, V> p, long ttl, Object[] args) {
         return dht().loadCacheAsync(p, ttl, args);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearGetFuture.java
index 6e1f494..71192b3 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -12,7 +12,6 @@ package org.gridgain.grid.kernal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.dht.*;
@@ -181,8 +180,8 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
     /** {@inheritDoc} */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<Map<K, V>>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<Map<K, V>> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<InternalFuture<Map<K, V>>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(InternalFuture<Map<K, V>> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -193,7 +192,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        for (IgniteFuture<Map<K, V>> fut : futures())
+        for (InternalFuture<Map<K, V>> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -212,7 +211,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
      * @param res Result.
      */
     void onResult(UUID nodeId, GridNearGetResponse<K, V> res) {
-        for (IgniteFuture<Map<K, V>> fut : futures())
+        for (InternalFuture<Map<K, V>> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -241,7 +240,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<Map<K, V>> f) {
+    private boolean isMini(InternalFuture<Map<K, V>> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -314,8 +313,8 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                 }
 
                 // Add new future.
-                add(fut.chain(new C1<IgniteFuture<Collection<GridCacheEntryInfo<K, V>>>, Map<K, V>>() {
-                    @Override public Map<K, V> apply(IgniteFuture<Collection<GridCacheEntryInfo<K, V>>> fut) {
+                add(fut.chain(new C1<InternalFuture<Collection<GridCacheEntryInfo<K, V>>>, Map<K, V>>() {
+                    @Override public Map<K, V> apply(InternalFuture<Collection<GridCacheEntryInfo<K, V>>> fut) {
                         try {
                             return loadEntries(n.id(), mappedKeys.keySet(), fut.get(), saved, topVer);
                         }
@@ -703,8 +702,8 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
             else {
                 final RemapTimeoutObject timeout = new RemapTimeoutObject(ctx.config().getNetworkTimeout(), topVer, e);
 
-                ctx.discovery().topologyFuture(topVer + 1).listenAsync(new CI1<IgniteFuture<Long>>() {
-                    @Override public void apply(IgniteFuture<Long> longIgniteFuture) {
+                ctx.discovery().topologyFuture(topVer + 1).listenAsync(new CI1<InternalFuture<Long>>() {
+                    @Override public void apply(InternalFuture<Long> longIgniteFuture) {
                         if (timeout.finish()) {
                             ctx.timeout().removeTimeoutObject(timeout);
 
@@ -753,10 +752,10 @@ 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.
-                IgniteFuture<Long> topFut = ctx.discovery().topologyFuture(rmtTopVer);
+                InternalFuture<Long> topFut = ctx.discovery().topologyFuture(rmtTopVer);
 
-                topFut.listenAsync(new CIX1<IgniteFuture<Long>>() {
-                    @Override public void applyx(IgniteFuture<Long> fut) throws IgniteCheckedException {
+                topFut.listenAsync(new CIX1<InternalFuture<Long>>() {
+                    @Override public void applyx(InternalFuture<Long> fut) throws IgniteCheckedException {
                         long readyTopVer = fut.get();
 
                         // This will append new futures to compound list.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearLockFuture.java
index 15f9a85..a4cd619 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -12,7 +12,6 @@ package org.gridgain.grid.kernal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.managers.discovery.*;
 import org.gridgain.grid.kernal.processors.cache.*;
@@ -169,8 +168,8 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
      */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<InternalFuture<?>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(InternalFuture<?> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -398,7 +397,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
     @Override public boolean onNodeLeft(UUID nodeId) {
         boolean found = false;
 
-        for (IgniteFuture<?> fut : futures()) {
+        for (InternalFuture<?> fut : futures()) {
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -432,7 +431,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
             if (log.isDebugEnabled())
                 log.debug("Received lock response from node [nodeId=" + nodeId + ", res=" + res + ", fut=" + this + ']');
 
-            for (IgniteFuture<Boolean> fut : pending()) {
+            for (InternalFuture<Boolean> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture mini = (MiniFuture)fut;
 
@@ -623,7 +622,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(InternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -683,8 +682,8 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
                     markInitialized();
                 }
                 else {
-                    fut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                        @Override public void apply(IgniteFuture<Long> t) {
+                    fut.listenAsync(new CI1<InternalFuture<Long>>() {
+                        @Override public void apply(InternalFuture<Long> t) {
                             mapOnTopology();
                         }
                     });
@@ -949,7 +948,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
             if (log.isDebugEnabled())
                 log.debug("Before locally locking near request: " + req);
 
-            IgniteFuture<GridNearLockResponse<K, V>> fut = dht().lockAllAsync(cctx, cctx.localNode(), req, filter);
+            InternalFuture<GridNearLockResponse<K, V>> fut = dht().lockAllAsync(cctx, cctx.localNode(), req, filter);
 
             // Add new future.
             add(new GridEmbeddedFuture<>(
@@ -1089,7 +1088,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
 
             add(fut); // Append new future.
 
-            IgniteFuture<?> txSync = null;
+            InternalFuture<?> txSync = null;
 
             if (inTx())
                 txSync = cctx.tm().awaitFinishAckAsync(node.id(), tx.threadId());
@@ -1108,8 +1107,8 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
                 }
             }
             else {
-                txSync.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> t) {
+                txSync.listenAsync(new CI1<InternalFuture<?>>() {
+                    @Override public void apply(InternalFuture<?> t) {
                         try {
                             if (log.isDebugEnabled())
                                 log.debug("Sending near lock request [node=" + node.id() + ", req=" + req + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java
index 9b4d117..72b755c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -82,7 +82,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(
+    @Override public InternalFuture<Map<K, V>> getAllAsync(
         @Nullable final Collection<? extends K> keys,
         boolean forcePrimary,
         boolean skipTx,
@@ -102,7 +102,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
 
         if (tx != null && !tx.implicit() && !skipTx) {
             return asyncOp(tx, new AsyncOp<Map<K, V>>(keys) {
-                @Override public IgniteFuture<Map<K, V>> op(GridCacheTxLocalAdapter<K, V> tx) {
+                @Override public InternalFuture<Map<K, V>> op(GridCacheTxLocalAdapter<K, V> tx) {
                     return ctx.wrapCloneMap(tx.getAllAsync(ctx, keys, entry, deserializePortable, filter));
                 }
             });
@@ -119,7 +119,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
      * @param filter Filter.
      * @return Future.
      */
-    IgniteFuture<Map<K, V>> txLoadAsync(GridNearTxLocal<K, V> tx, @Nullable Collection<? extends K> keys,
+    InternalFuture<Map<K, V>> txLoadAsync(GridNearTxLocal<K, V> tx, @Nullable Collection<? extends K> keys,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, boolean deserializePortable) {
         assert tx != null;
 
@@ -369,7 +369,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
+    @Override protected InternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
         GridCacheTxLocalEx<K, V> tx, boolean isInvalidate, boolean isRead, boolean retval,
         GridCacheTxIsolation isolation, IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
         GridNearLockFuture<K, V> fut = new GridNearLockFuture<>(ctx, keys, (GridNearTxLocal<K, V>)tx, isRead,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 176fdd0..9647c61 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -12,7 +12,6 @@ package org.gridgain.grid.kernal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.*;
@@ -109,8 +108,8 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<InternalFuture<?>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(InternalFuture<?> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -121,7 +120,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        for (IgniteFuture<?> fut : futures())
+        for (InternalFuture<?> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -188,7 +187,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      */
     public void onResult(UUID nodeId, GridNearTxFinishResponse<K, V> res) {
         if (!isDone())
-            for (IgniteFuture<GridCacheTx> fut : futures()) {
+            for (InternalFuture<GridCacheTx> fut : futures()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -244,7 +243,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(InternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -271,7 +270,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             // Finish local mapping only as we need send commit message to backups.
             for (GridDistributedTxMapping<K, V> m : mappings.values()) {
                 if (m.node().isLocal()) {
-                    IgniteFuture<GridCacheTx> fut = cctx.tm().txHandler().finishColocatedLocal(commit, tx);
+                    InternalFuture<GridCacheTx> fut = cctx.tm().txHandler().finishColocatedLocal(commit, tx);
 
                     // Add new future.
                     if (fut != null)
@@ -292,7 +291,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             if (!isSync()) {
                 boolean complete = true;
 
-                for (IgniteFuture<?> f : pending())
+                for (InternalFuture<?> f : pending())
                     // Mini-future in non-sync mode gets done when message gets sent.
                     if (isMini(f) && !f.isDone())
                         complete = false;
@@ -356,7 +355,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
         if (n.isLocal()) {
             req.miniId(IgniteUuid.randomUuid());
 
-            IgniteFuture<GridCacheTx> fut = cctx.tm().txHandler().finish(n.id(), tx, req);
+            InternalFuture<GridCacheTx> fut = cctx.tm().txHandler().finish(n.id(), tx, req);
 
             // Add new future.
             if (fut != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
index 5ff258f..530a384 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -50,7 +50,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
         new ConcurrentHashMap8<>();
 
     /** Future. */
-    private final AtomicReference<IgniteFuture<GridCacheTxEx<K, V>>> prepFut =
+    private final AtomicReference<InternalFuture<GridCacheTxEx<K, V>>> prepFut =
         new AtomicReference<>();
 
     /** */
@@ -159,7 +159,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteFuture<Boolean> addReader(long msgId, GridDhtCacheEntry<K, V> cached,
+    @Override protected InternalFuture<Boolean> addReader(long msgId, GridDhtCacheEntry<K, V> cached,
         GridCacheTxEntry<K, V> entry, long topVer) {
         // We are in near transaction, do not add local node as reader.
         return null;
@@ -253,15 +253,15 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> loadMissing(
+    @Override public InternalFuture<Boolean> loadMissing(
         GridCacheContext<K, V> cacheCtx,
         boolean async, final Collection<? extends K> keys,
         boolean deserializePortable,
         final IgniteBiInClosure<K, V> c
     ) {
         if (cacheCtx.isNear()) {
-            return cacheCtx.nearTx().txLoadAsync(this, keys, CU.<K, V>empty(), deserializePortable).chain(new C1<IgniteFuture<Map<K, V>>, Boolean>() {
-                @Override public Boolean apply(IgniteFuture<Map<K, V>> f) {
+            return cacheCtx.nearTx().txLoadAsync(this, keys, CU.<K, V>empty(), deserializePortable).chain(new C1<InternalFuture<Map<K, V>>, Boolean>() {
+                @Override public Boolean apply(InternalFuture<Map<K, V>> f) {
                     try {
                         Map<K, V> map = f.get();
 
@@ -285,8 +285,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
             return cacheCtx.colocated().loadAsync(keys, /*reload*/false, /*force primary*/false, topologyVersion(),
                 CU.subjectId(this, cctx), resolveTaskName(), deserializePortable, null)
-                .chain(new C1<IgniteFuture<Map<K, V>>, Boolean>() {
-                    @Override public Boolean apply(IgniteFuture<Map<K, V>> f) {
+                .chain(new C1<InternalFuture<Map<K, V>>, Boolean>() {
+                    @Override public Boolean apply(InternalFuture<Map<K, V>> f) {
                         try {
                             Map<K, V> map = f.get();
 
@@ -508,7 +508,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     /**
      * @return Commit fut.
      */
-    @Override public IgniteFuture<GridCacheTxEx<K, V>> future() {
+    @Override public InternalFuture<GridCacheTxEx<K, V>> future() {
         return prepFut.get();
     }
 
@@ -656,8 +656,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheTxEx<K, V>> prepareAsync() {
-        IgniteFuture<GridCacheTxEx<K, V>> fut = prepFut.get();
+    @Override public InternalFuture<GridCacheTxEx<K, V>> prepareAsync() {
+        InternalFuture<GridCacheTxEx<K, V>> fut = prepFut.get();
 
         if (fut == null) {
             // Future must be created before any exception can be thrown.
@@ -724,7 +724,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"ThrowableInstanceNeverThrown"})
-    @Override public IgniteFuture<GridCacheTx> commitAsync() {
+    @Override public InternalFuture<GridCacheTx> commitAsync() {
         if (log.isDebugEnabled())
             log.debug("Committing near local tx: " + this);
 
@@ -737,10 +737,10 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
         cctx.mvcc().addFuture(fut);
 
-        IgniteFuture<GridCacheTxEx<K, V>> prepareFut = prepFut.get();
+        InternalFuture<GridCacheTxEx<K, V>> prepareFut = prepFut.get();
 
-        prepareFut.listenAsync(new CI1<IgniteFuture<GridCacheTxEx<K, V>>>() {
-            @Override public void apply(IgniteFuture<GridCacheTxEx<K, V>> f) {
+        prepareFut.listenAsync(new CI1<InternalFuture<GridCacheTxEx<K, V>>>() {
+            @Override public void apply(InternalFuture<GridCacheTxEx<K, V>> f) {
                 GridNearTxFinishFuture<K, V> fut0 = commitFut.get();
 
                 try {
@@ -770,7 +770,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheTx> rollbackAsync() {
+    @Override public InternalFuture<GridCacheTx> rollbackAsync() {
         if (log.isDebugEnabled())
             log.debug("Rolling back near tx: " + this);
 
@@ -784,7 +784,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
         cctx.mvcc().addFuture(fut);
 
-        IgniteFuture<GridCacheTxEx<K, V>> prepFut = this.prepFut.get();
+        InternalFuture<GridCacheTxEx<K, V>> prepFut = this.prepFut.get();
 
         if (prepFut == null || prepFut.isDone()) {
             try {
@@ -808,8 +808,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             }
         }
         else {
-            prepFut.listenAsync(new CI1<IgniteFuture<GridCacheTxEx<K, V>>>() {
-                @Override public void apply(IgniteFuture<GridCacheTxEx<K, V>> f) {
+            prepFut.listenAsync(new CI1<InternalFuture<GridCacheTxEx<K, V>>>() {
+                @Override public void apply(InternalFuture<GridCacheTxEx<K, V>> f) {
                     try {
                         // Check for errors in prepare future.
                         f.get();
@@ -852,7 +852,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
      * @return Future that will be completed when locks are acquired.
      */
     @SuppressWarnings("TypeMayBeWeakened")
-    public IgniteFuture<GridCacheTxEx<K, V>> prepareAsyncLocal(@Nullable Collection<GridCacheTxEntry<K, V>> reads,
+    public InternalFuture<GridCacheTxEx<K, V>> prepareAsyncLocal(@Nullable Collection<GridCacheTxEntry<K, V>> reads,
         @Nullable Collection<GridCacheTxEntry<K, V>> writes, Map<UUID, Collection<UUID>> txNodes, boolean last,
         Collection<UUID> lastBackups) {
         assert optimistic();
@@ -918,7 +918,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
      *
      * @return Commit future.
      */
-    public IgniteFuture<GridCacheTx> commitAsyncLocal() {
+    public InternalFuture<GridCacheTx> commitAsyncLocal() {
         if (log.isDebugEnabled())
             log.debug("Committing colocated tx locally: " + this);
 
@@ -926,12 +926,12 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
         if (pessimistic())
             prepareAsync();
 
-        IgniteFuture<GridCacheTxEx<K, V>> prep = prepFut.get();
+        InternalFuture<GridCacheTxEx<K, V>> prep = prepFut.get();
 
         // Do not create finish future if there are no remote nodes.
         if (F.isEmpty(dhtMap) && F.isEmpty(nearMap)) {
             if (prep != null)
-                return (IgniteFuture<GridCacheTx>)(IgniteFuture)prep;
+                return (InternalFuture<GridCacheTx>)(InternalFuture)prep;
 
             return new GridFinishedFuture<GridCacheTx>(cctx.kernalContext(), this);
         }
@@ -962,8 +962,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             }
         }
         else
-            prep.listenAsync(new CI1<IgniteFuture<GridCacheTxEx<K, V>>>() {
-                @Override public void apply(IgniteFuture<GridCacheTxEx<K, V>> f) {
+            prep.listenAsync(new CI1<InternalFuture<GridCacheTxEx<K, V>>>() {
+                @Override public void apply(InternalFuture<GridCacheTxEx<K, V>> f) {
                     try {
                         f.get(); // Check for errors of a parent future.
 
@@ -991,7 +991,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
      *
      * @return Commit future.
      */
-    public IgniteFuture<GridCacheTx> rollbackAsyncLocal() {
+    public InternalFuture<GridCacheTx> rollbackAsyncLocal() {
         if (log.isDebugEnabled())
             log.debug("Rolling back colocated tx locally: " + this);
 
@@ -999,7 +999,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
         cctx.mvcc().addFuture(fut);
 
-        IgniteFuture<GridCacheTxEx<K, V>> prep = prepFut.get();
+        InternalFuture<GridCacheTxEx<K, V>> prep = prepFut.get();
 
         if (prep == null || prep.isDone()) {
             try {
@@ -1015,8 +1015,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             fut.finish();
         }
         else
-            prep.listenAsync(new CI1<IgniteFuture<GridCacheTxEx<K, V>>>() {
-                @Override public void apply(IgniteFuture<GridCacheTxEx<K, V>> f) {
+            prep.listenAsync(new CI1<InternalFuture<GridCacheTxEx<K, V>>>() {
+                @Override public void apply(InternalFuture<GridCacheTxEx<K, V>> f) {
                     try {
                         f.get(); // Check for errors of a parent future.
                     }
@@ -1033,7 +1033,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    public IgniteFuture<GridCacheReturn<V>> lockAllAsync(GridCacheContext<K, V> cacheCtx, final Collection<? extends K> keys,
+    public InternalFuture<GridCacheReturn<V>> lockAllAsync(GridCacheContext<K, V> cacheCtx, final Collection<? extends K> keys,
         boolean implicit, boolean read) {
         assert pessimistic();
 
@@ -1054,7 +1054,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
         if (log.isDebugEnabled())
             log.debug("Before acquiring transaction lock on keys: " + keys);
 
-        IgniteFuture<Boolean> fut = cacheCtx.colocated().lockAllAsyncInternal(keys,
+        InternalFuture<Boolean> fut = cacheCtx.colocated().lockAllAsyncInternal(keys,
             lockTimeout(), this, isInvalidate(), read, /*retval*/false, isolation, CU.<K, V>empty());
 
         return new GridEmbeddedFuture<>(

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxPrepareFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
index a9bcd3b..b293861 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
@@ -12,7 +12,6 @@ package org.gridgain.grid.kernal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.managers.discovery.*;
 import org.gridgain.grid.kernal.processors.cache.*;
@@ -131,8 +130,8 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
      */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<InternalFuture<?>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(InternalFuture<?> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -155,7 +154,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
     @Override public boolean onNodeLeft(UUID nodeId) {
         boolean found = false;
 
-        for (IgniteFuture<?> fut : futures())
+        for (InternalFuture<?> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -251,7 +250,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
      */
     public void onResult(UUID nodeId, GridNearTxPrepareResponse<K, V> res) {
         if (!isDone()) {
-            for (IgniteFuture<GridCacheTxEx<K, V>> fut : pending()) {
+            for (InternalFuture<GridCacheTxEx<K, V>> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -290,7 +289,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(InternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -363,8 +362,8 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
             else {
                 topFut.syncNotify(false);
 
-                topFut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                    @Override public void apply(IgniteFuture<Long> t) {
+                topFut.listenAsync(new CI1<InternalFuture<Long>>() {
+                    @Override public void apply(InternalFuture<Long> t) {
                         prepare();
                     }
                 });
@@ -562,7 +561,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
             // At this point, if any new node joined, then it is
             // waiting for this transaction to complete, so
             // partition reassignments are not possible here.
-            IgniteFuture<GridCacheTxEx<K, V>> fut = cctx.tm().txHandler().prepareTx(n.id(), tx, req);
+            InternalFuture<GridCacheTxEx<K, V>> fut = cctx.tm().txHandler().prepareTx(n.id(), tx, req);
 
             // Add new future.
             add(new GridEmbeddedFuture<>(

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalCache.java
index 8edcf9c..58c74b8 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalCache.java
@@ -84,7 +84,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> txLockAsync(Collection<? extends K> keys, long timeout,
+    @Override public InternalFuture<Boolean> txLockAsync(Collection<? extends K> keys, long timeout,
         GridCacheTxLocalEx<K, V> tx, boolean isRead,
         boolean retval, GridCacheTxIsolation isolation, boolean invalidate,
         IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
@@ -92,7 +92,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
+    @Override public InternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
         IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
         GridCacheTxLocalEx<K, V> tx = ctx.tm().localTx();
 
@@ -106,7 +106,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
      * @param filter Filter.
      * @return Future.
      */
-    public IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
+    public InternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
         @Nullable GridCacheTxLocalEx<K, V> tx, IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
         if (F.isEmpty(keys))
             return new GridFinishedFuture<>(ctx.kernalContext(), true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java
index e769893..2fc3e0e 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java
@@ -65,7 +65,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheTxEx<K, V>> future() {
+    @Override public InternalFuture<GridCacheTxEx<K, V>> future() {
         return fut.get();
     }
 
@@ -103,7 +103,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheTxEx<K, V>> prepareAsync() {
+    @Override public InternalFuture<GridCacheTxEx<K, V>> prepareAsync() {
         try {
             prepare();
 
@@ -142,7 +142,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings( {"unchecked", "RedundantCast"})
-    @Override public IgniteFuture<GridCacheTx> commitAsync() {
+    @Override public InternalFuture<GridCacheTx> commitAsync() {
         try {
             prepare();
         }
@@ -160,11 +160,11 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
 
                 fut.checkLocks();
 
-                return (IgniteFuture)fut;
+                return (InternalFuture)fut;
             }
         }
 
-        return (IgniteFuture)this.fut.get();
+        return (InternalFuture)this.fut.get();
     }
 
     /** {@inheritDoc} */
@@ -172,7 +172,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
         rollbackAsync().get();
     }
 
-    @Override public IgniteFuture<GridCacheTx> rollbackAsync() {
+    @Override public InternalFuture<GridCacheTx> rollbackAsync() {
         try {
             state(ROLLING_BACK);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
index e4d6b00..0262b0f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -151,7 +151,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> putAsync(K key,
+    @Override public InternalFuture<V> putAsync(K key,
         V val,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl,
@@ -165,7 +165,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> putxAsync(K key,
+    @Override public InternalFuture<Boolean> putxAsync(K key,
         V val,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl,
@@ -184,7 +184,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putIfAbsentAsync(K key, V val) {
+    @Override public InternalFuture<V> putIfAbsentAsync(K key, V val) {
         return putAsync(key, val, ctx.noPeekArray());
     }
 
@@ -194,7 +194,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxIfAbsentAsync(K key, V val) {
+    @Override public InternalFuture<Boolean> putxIfAbsentAsync(K key, V val) {
         return putxAsync(key, val, ctx.noPeekArray());
     }
 
@@ -205,7 +205,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(K key, V val) {
+    @Override public InternalFuture<V> replaceAsync(K key, V val) {
         return putAsync(key, val, ctx.hasPeekArray());
     }
 
@@ -215,7 +215,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(K key, V val) {
+    @Override public InternalFuture<Boolean> replacexAsync(K key, V val) {
         return putxAsync(key, val, ctx.hasPeekArray());
     }
 
@@ -225,7 +225,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
+    @Override public InternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
         return putxAsync(key, newVal, ctx.equalsPeekArray(oldVal));
     }
 
@@ -265,7 +265,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
+    @Override public InternalFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
         A.notNull(key, "key");
 
         ctx.denyOnLocalRead();
@@ -275,7 +275,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
+    @Override public InternalFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
         A.notNull(key, "key");
 
         ctx.denyOnLocalRead();
@@ -300,7 +300,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
+    @Override public InternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
         ctx.denyOnLocalRead();
 
@@ -336,7 +336,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> transformAsync(K key,
+    @Override public InternalFuture<?> transformAsync(K key,
         IgniteClosure<V, V> transformer,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl) {
@@ -364,7 +364,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> transformAllAsync(@Nullable Map<? extends K, ? extends IgniteClosure<V, V>> m) {
+    @Override public InternalFuture<?> transformAllAsync(@Nullable Map<? extends K, ? extends IgniteClosure<V, V>> m) {
         ctx.denyOnLocalRead();
 
         if (F.isEmpty(m))
@@ -392,7 +392,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> removeAsync(K key,
+    @Override public InternalFuture<V> removeAsync(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
         ctx.denyOnLocalRead();
@@ -417,7 +417,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(Collection<? extends K> keys,
+    @Override public InternalFuture<?> removeAllAsync(Collection<? extends K> keys,
         IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
         ctx.denyOnLocalRead();
 
@@ -445,7 +445,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> removexAsync(K key,
+    @Override public InternalFuture<Boolean> removexAsync(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
         A.notNull(key, "key");
@@ -472,7 +472,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(K key, V val) {
+    @Override public InternalFuture<Boolean> removeAsync(K key, V val) {
         return removexAsync(key, ctx.equalsPeekArray(val));
     }
 
@@ -483,7 +483,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
+    @Override public InternalFuture<?> removeAllAsync(IgnitePredicate<GridCacheEntry<K, V>>[] filter) {
         return removeAllAsync(keySet(filter), filter);
     }
 
@@ -528,7 +528,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(
+    @Override public InternalFuture<Map<K, V>> getAllAsync(
         @Nullable final Collection<? extends K> keys,
         final boolean forcePrimary,
         boolean skipTx,
@@ -665,7 +665,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      * @param filter Cache entry filter for atomic updates.
      * @return Completion future.
      */
-    private IgniteFuture updateAllAsync0(
+    private InternalFuture updateAllAsync0(
         @Nullable final Map<? extends K, ? extends V> map,
         @Nullable final Map<? extends K, ? extends IgniteClosure<V, V>> transformMap,
         final boolean retval,
@@ -702,7 +702,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      * @param filter Cache entry filter.
      * @return Completion future.
      */
-    private IgniteFuture removeAllAsync0(
+    private InternalFuture removeAllAsync0(
         @Nullable final Collection<? extends K> keys,
         final boolean retval,
         final boolean rawRetval,
@@ -1245,7 +1245,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> txLockAsync(Collection<? extends K> keys,
+    @Override public InternalFuture<Boolean> txLockAsync(Collection<? extends K> keys,
         long timeout,
         GridCacheTxLocalEx<K, V> tx,
         boolean isRead,
@@ -1259,7 +1259,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys,
+    @Override public InternalFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys,
         long timeout,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) {
         return new GridFinishedFutureEx<>(new UnsupportedOperationException("Locks are not supported for " +
@@ -1279,8 +1279,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    protected IgniteFuture asyncOp(final Callable<?> op) {
-        IgniteFuture fail = asyncOpAcquire();
+    protected InternalFuture asyncOp(final Callable<?> op) {
+        InternalFuture fail = asyncOpAcquire();
 
         if (fail != null)
             return fail;
@@ -1290,12 +1290,12 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         holder.lock();
 
         try {
-            IgniteFuture fut = holder.future();
+            InternalFuture fut = holder.future();
 
             if (fut != null && !fut.isDone()) {
-                IgniteFuture f = new GridEmbeddedFuture(fut,
-                    new C2<Object, Exception, IgniteFuture>() {
-                        @Override public IgniteFuture apply(Object t, Exception e) {
+                InternalFuture f = new GridEmbeddedFuture(fut,
+                    new C2<Object, Exception, InternalFuture>() {
+                        @Override public InternalFuture apply(Object t, Exception e) {
                             return ctx.closures().callLocalSafe(op);
                         }
                     }, ctx.kernalContext());
@@ -1305,7 +1305,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                 return f;
             }
 
-            IgniteFuture f = ctx.closures().callLocalSafe(op);
+            InternalFuture f = ctx.closures().callLocalSafe(op);
 
             saveFuture(holder, f);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
index 6bf1f45..c842839 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
@@ -86,7 +86,7 @@ public class GridCacheDistributedFieldsQueryFuture
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<List<GridQueryFieldMetadata>> metadata() {
+    @Override public InternalFuture<List<GridQueryFieldMetadata>> metadata() {
         return metaFut;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
index b3d658b..4c12c7e 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
@@ -185,7 +185,7 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
 
     /** {@inheritDoc} */
     @SuppressWarnings("NonPrivateFieldAccessedInSynchronizedContext")
-    @Override protected void loadAllPages() throws GridInterruptedException {
+    @Override protected void loadAllPages() throws InternalInterruptedException {
         assert !Thread.holdsLock(mux);
 
         U.await(firstPageLatch);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
index b9d2474..71c3c25 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -293,7 +293,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
                         try {
                             U.sleep(RESEND_FREQ);
                         }
-                        catch (GridInterruptedException e1) {
+                        catch (InternalInterruptedException e1) {
                             U.error(log,
                                 "Waiting for queries response resending was interrupted (response will not be sent) " +
                                 "[nodeId=" + nodeId + ", response=" + res + "]", e1);
@@ -537,8 +537,8 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             cctx.io().addOrderedHandler(topic, resHnd);
 
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> fut) {
+            fut.listenAsync(new CI1<InternalFuture<?>>() {
+                @Override public void apply(InternalFuture<?> fut) {
                     cctx.io().removeOrderedHandler(topic);
                 }
             });
@@ -646,8 +646,8 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             cctx.io().addOrderedHandler(topic, resHnd);
 
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> fut) {
+            fut.listenAsync(new CI1<InternalFuture<?>>() {
+                @Override public void apply(InternalFuture<?> fut) {
                     cctx.io().removeOrderedHandler(topic);
                 }
             });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java
index 3da96d7..ebd6467 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheFieldsQueryErrorFuture.java
@@ -40,7 +40,7 @@ public class GridCacheFieldsQueryErrorFuture extends GridCacheQueryErrorFuture<L
     /**
      * @return Metadata.
      */
-    public IgniteFuture<List<GridQueryFieldMetadata>> metadata() {
+    public InternalFuture<List<GridQueryFieldMetadata>> metadata() {
         return new GridFinishedFuture<>(ctx, incMeta ? Collections.<GridQueryFieldMetadata>emptyList() : null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
index 5114f3a..97f6254 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
@@ -66,7 +66,7 @@ public class GridCacheLocalFieldsQueryFuture
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<List<GridQueryFieldMetadata>> metadata() {
+    @Override public InternalFuture<List<GridQueryFieldMetadata>> metadata() {
         return metaFut;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
index df55b83..aa0eed7 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
@@ -31,7 +31,7 @@ public class GridCacheLocalQueryFuture<K, V, R> extends GridCacheQueryFutureAdap
     private Runnable run;
 
     /** */
-    private IgniteFuture<?> fut;
+    private InternalFuture<?> fut;
 
     /**
      * Required by {@link Externalizable}.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java
index 3ba1ceb..0e4efed 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java
@@ -163,21 +163,21 @@ public class GridCacheQueriesImpl<K, V> implements GridCacheQueriesEx<K, V>, Ext
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildIndexes(Class<?> cls) {
+    @Override public InternalFuture<?> rebuildIndexes(Class<?> cls) {
         A.notNull(cls, "cls");
 
         return ctx.queries().rebuildIndexes(cls);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildIndexes(String typeName) {
+    @Override public InternalFuture<?> rebuildIndexes(String typeName) {
         A.notNull("typeName", typeName);
 
         return ctx.queries().rebuildIndexes(typeName);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildAllIndexes() {
+    @Override public InternalFuture<?> rebuildAllIndexes() {
         return ctx.queries().rebuildAllIndexes();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java
index 9edcf6a..9a8b588 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java
@@ -166,7 +166,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildIndexes(Class<?> cls) {
+    @Override public InternalFuture<?> rebuildIndexes(Class<?> cls) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -178,7 +178,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildIndexes(String typeName) {
+    @Override public InternalFuture<?> rebuildIndexes(String typeName) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -190,7 +190,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildAllIndexes() {
+    @Override public InternalFuture<?> rebuildAllIndexes() {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {