You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by vo...@apache.org on 2015/02/02 11:29:01 UTC

[25/50] [abbrv] incubator-ignite git commit: # sprint-1 moved existing IgniteFuture to internal package

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFutureImplStopGridSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFutureImplStopGridSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFutureImplStopGridSelfTest.java
index bf6eafb..99d490f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFutureImplStopGridSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskFutureImplStopGridSelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.compute.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.junits.common.*;
@@ -67,9 +66,9 @@ public class GridTaskFutureImplStopGridSelfTest extends GridCommonAbstractTest {
         try {
             final ComputeTaskFuture<?> fut = executeAsync(ignite.compute(), GridStopTestTask.class.getName(), null);
 
-            fut.listenAsync(new CI1<IgniteFuture>() {
+            fut.listenAsync(new CI1<IgniteInternalFuture>() {
                 @SuppressWarnings({"NakedNotify"})
-                @Override public void apply(IgniteFuture gridFut) {
+                @Override public void apply(IgniteInternalFuture gridFut) {
                     synchronized (mux) {
                         mux.notifyAll();
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/GridTaskListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskListenerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskListenerSelfTest.java
index abdbcc0..be88ba9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridTaskListenerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTaskListenerSelfTest.java
@@ -47,8 +47,8 @@ public class GridTaskListenerSelfTest extends GridCommonAbstractTest {
     public void testGridTaskListener() throws Exception {
         final AtomicInteger cnt = new AtomicInteger(0);
 
-        IgniteInClosure<IgniteFuture<?>> lsnr = new CI1<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> fut) {
+        IgniteInClosure<IgniteInternalFuture<?>> lsnr = new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> fut) {
                 assert fut != null;
 
                 cnt.incrementAndGet();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java
index 067e5e4..6b3ffc8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java
@@ -250,7 +250,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
 
             Ignite ignite = startGrid(gridName);
 
-            IgniteFuture fut = executeAsync(ignite.compute(), new GridMultiNodeGlobalConsumerTask(), null);
+            IgniteInternalFuture fut = executeAsync(ignite.compute(), new GridMultiNodeGlobalConsumerTask(), null);
 
             executeAsync(ignite.compute(), GridMultiNodeTestCheckPointTask.class, null).get(2 * 60 * 1000);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/managers/events/GridEventStorageManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/events/GridEventStorageManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/events/GridEventStorageManagerSelfTest.java
index 1399e1f..8252cc5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/events/GridEventStorageManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/events/GridEventStorageManagerSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.managers.events;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -61,7 +62,7 @@ public class GridEventStorageManagerSelfTest extends GridCommonAbstractTest {
 
         final int usrType = Integer.MAX_VALUE - 1;
 
-        IgniteFuture<IgniteEvent> fut = waitForLocalEvent(ignite.events(), new IgnitePredicate<IgniteEvent>() {
+        IgniteInternalFuture<IgniteEvent> fut = waitForLocalEvent(ignite.events(), new IgnitePredicate<IgniteEvent>() {
             @Override public boolean apply(IgniteEvent e) {
                 return e.type() == usrType;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
index f2dcb6d..b50f061 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFailoverSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.transactions.*;
@@ -255,7 +256,7 @@ public abstract class GridCacheAbstractFailoverSelfTest extends GridCacheAbstrac
 
         final int half = ENTRY_CNT / 2;
 
-        IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
             @Override public void apply() {
                 info("Run topology change.");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFlagsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFlagsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFlagsTest.java
index 2ec9e7c..f136ca4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFlagsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFlagsTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.store.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 
 import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
@@ -74,7 +74,7 @@ public abstract class GridCacheAbstractFlagsTest extends GridCacheAbstractSelfTe
 
             final AtomicInteger cntr = new AtomicInteger();
 
-            IgniteFuture<?> f = multithreadedAsync(new Callable() {
+            IgniteInternalFuture<?> f = multithreadedAsync(new Callable() {
                 @Override public Object call() throws Exception {
                     int idx = cntr.getAndIncrement() % gridCount();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
index 1efc144..15a4a92 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiMultithreadedSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -72,7 +73,7 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
      * @throws Exception In case of error.
      */
     private void runTest(final IgniteInClosure<GridCache<String, Integer>> c) throws Exception {
-        final IgniteFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(new CAX() {
+        final IgniteInternalFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 while (true) {
                     int i = cnt.getAndIncrement();
@@ -90,7 +91,7 @@ public abstract class GridCacheAbstractFullApiMultithreadedSelfTest extends Grid
             }
         }, WRITE_THREAD_CNT, WRITE_THREAD_NAME);
 
-        IgniteFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(new CA() {
+        IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(new CA() {
             @Override public void apply() {
                 GridCache<String, Integer> cache = cache();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index 84a9b31..5c4e4d5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -432,9 +432,9 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         cache().put("key1", 1);
         cache().put("key2", 2);
 
-        IgniteFuture<Integer> fut1 = cache().getAsync("key1");
-        IgniteFuture<Integer> fut2 = cache().getAsync("key2");
-        IgniteFuture<Integer> fut3 = cache().getAsync("wrongKey");
+        IgniteInternalFuture<Integer> fut1 = cache().getAsync("key1");
+        IgniteInternalFuture<Integer> fut2 = cache().getAsync("key2");
+        IgniteInternalFuture<Integer> fut3 = cache().getAsync("wrongKey");
 
         assert fut1.get() == 1;
         assert fut2.get() == 2;
@@ -448,8 +448,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         cache().put("key1", 1);
         cache().put("key2", 100);
 
-        IgniteFuture<Integer> fut1 = cache().projection(gte100).getAsync("key1");
-        IgniteFuture<Integer> fut2 = cache().projection(gte100).getAsync("key2");
+        IgniteInternalFuture<Integer> fut1 = cache().projection(gte100).getAsync("key1");
+        IgniteInternalFuture<Integer> fut2 = cache().projection(gte100).getAsync("key2");
 
         assert fut1.get() == null;
         assert fut2.get() == 100;
@@ -674,8 +674,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
             }
         }, NullPointerException.class, null);
 
-        IgniteFuture<Map<String, Integer>> fut2 = cache().getAllAsync(Collections.<String>emptyList());
-        IgniteFuture<Map<String, Integer>> fut3 = cache().getAllAsync(F.asList("key1", "key2"));
+        IgniteInternalFuture<Map<String, Integer>> fut2 = cache().getAllAsync(Collections.<String>emptyList());
+        IgniteInternalFuture<Map<String, Integer>> fut3 = cache().getAllAsync(F.asList("key1", "key2"));
 
         assert fut2.get().isEmpty();
         assert fut3.get().size() == 2 : "Invalid map: " + fut3.get();
@@ -694,8 +694,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         List<String> keys = F.asList("key1", "key2", "key3", "key4");
 
-        IgniteFuture<Map<String, Integer>> fut1 = cache().projection(gte100).getAllAsync(keys);
-        IgniteFuture<Map<String, Integer>> fut2 = cache().projection(gte200).getAllAsync(keys);
+        IgniteInternalFuture<Map<String, Integer>> fut1 = cache().projection(gte100).getAllAsync(keys);
+        IgniteInternalFuture<Map<String, Integer>> fut2 = cache().projection(gte200).getAllAsync(keys);
 
         assert fut1.get().size() == 4 : "Invalid map: " + fut1.get();
         assert fut1.get().get("key1") == 100;
@@ -1234,8 +1234,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         cache().put("key1", 1);
         cache().put("key2", 2);
 
-        IgniteFuture<Integer> fut1 = cache().putAsync("key1", 10);
-        IgniteFuture<Integer> fut2 = cache().putAsync("key2", 11);
+        IgniteInternalFuture<Integer> fut1 = cache().putAsync("key1", 10);
+        IgniteInternalFuture<Integer> fut2 = cache().putAsync("key2", 11);
 
         assertEquals((Integer)1, fut1.get(5000));
         assertEquals((Integer)2, fut2.get(5000));
@@ -1248,8 +1248,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception In case of error.
      */
     public void testPutAsync0() throws Exception {
-        IgniteFuture<Integer> fut1 = cache().putAsync("key1", 0);
-        IgniteFuture<Integer> fut2 = cache().putAsync("key2", 1);
+        IgniteInternalFuture<Integer> fut1 = cache().putAsync("key1", 0);
+        IgniteInternalFuture<Integer> fut2 = cache().putAsync("key2", 1);
 
         assert fut1.get(5000) == null;
         assert fut2.get(5000) == null;
@@ -1268,15 +1268,15 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         assertNull(asyncCache.invoke("key1", INCR_PROCESSOR));
 
-        IgniteFuture<?> fut0 = asyncCache.future();
+        IgniteInternalFuture<?> fut0 = asyncCache.future();
 
         assertNull(asyncCache.invoke("key2", INCR_PROCESSOR));
 
-        IgniteFuture<?> fut1 = asyncCache.future();
+        IgniteInternalFuture<?> fut1 = asyncCache.future();
 
         assertNull(asyncCache.invoke("key3", RMV_PROCESSOR));
 
-        IgniteFuture<?> fut2 = asyncCache.future();
+        IgniteInternalFuture<?> fut2 = asyncCache.future();
 
         fut0.get();
         fut1.get();
@@ -1632,10 +1632,10 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
         cache().put("key2", 1);
 
-        IgniteFuture<Boolean> fut1 = cache().putxAsync("key1", 10);
-        IgniteFuture<Boolean> fut2 = cache().putxAsync("key2", 11);
+        IgniteInternalFuture<Boolean> fut1 = cache().putxAsync("key1", 10);
+        IgniteInternalFuture<Boolean> fut2 = cache().putxAsync("key2", 11);
 
-        IgniteFuture<IgniteTx> f = null;
+        IgniteInternalFuture<IgniteTx> f = null;
 
         if (tx != null) {
             tx = (IgniteTx)tx.withAsync();
@@ -1660,12 +1660,12 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception In case of error.
      */
     public void testPutxAsyncFiltered() throws Exception {
-        IgniteFuture<Boolean> f1 = cache().putxAsync("key1", 1);
-        IgniteFuture<Boolean> f2 = cache().putxAsync("key1", 101, F.<String, Integer>cacheHasPeekValue());
-        IgniteFuture<Boolean> f3 = cache().putxAsync("key2", 2);
-        IgniteFuture<Boolean> f4 = cache().putxAsync("key2", 202, F.<String, Integer>cacheHasPeekValue());
-        IgniteFuture<Boolean> f5 = cache().putxAsync("key1", 1, F.<String, Integer>cacheNoPeekValue());
-        IgniteFuture<Boolean> f6 = cache().putxAsync("key2", 2, F.<String, Integer>cacheNoPeekValue());
+        IgniteInternalFuture<Boolean> f1 = cache().putxAsync("key1", 1);
+        IgniteInternalFuture<Boolean> f2 = cache().putxAsync("key1", 101, F.<String, Integer>cacheHasPeekValue());
+        IgniteInternalFuture<Boolean> f3 = cache().putxAsync("key2", 2);
+        IgniteInternalFuture<Boolean> f4 = cache().putxAsync("key2", 202, F.<String, Integer>cacheHasPeekValue());
+        IgniteInternalFuture<Boolean> f5 = cache().putxAsync("key1", 1, F.<String, Integer>cacheNoPeekValue());
+        IgniteInternalFuture<Boolean> f6 = cache().putxAsync("key2", 2, F.<String, Integer>cacheNoPeekValue());
 
         assert f1.get() : "Invalid future1: " + f1;
         assert f2.get() : "Invalid future2: " + f2;
@@ -1957,12 +1957,12 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     public void testPutAllAsync() throws Exception {
         Map<String, Integer> map = F.asMap("key1", 1, "key2", 2);
 
-        IgniteFuture<?> f1 = cache().putAllAsync(map);
+        IgniteInternalFuture<?> f1 = cache().putAllAsync(map);
 
         map.put("key1", 10);
         map.put("key2", 20);
 
-        IgniteFuture<?> f2 = cache().putAllAsync(map);
+        IgniteInternalFuture<?> f2 = cache().putAllAsync(map);
 
         f2.get();
         f1.get();
@@ -1979,11 +1979,11 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     public void testPutAllAsyncFiltered() throws Exception {
         Map<String, Integer> map1 = F.asMap("key1", 1, "key2", 2);
 
-        IgniteFuture<?> f1 = cache().putAllAsync(map1, F.<String, Integer>cacheNoPeekValue());
+        IgniteInternalFuture<?> f1 = cache().putAllAsync(map1, F.<String, Integer>cacheNoPeekValue());
 
         Map<String, Integer> map2 = F.asMap("key1", 10, "key2", 20, "key3", 3);
 
-        IgniteFuture<?> f2 = cache().putAllAsync(map2, F.<String, Integer>cacheNoPeekValue());
+        IgniteInternalFuture<?> f2 = cache().putAllAsync(map2, F.<String, Integer>cacheNoPeekValue());
 
         f2.get();
         f1.get();
@@ -2077,12 +2077,12 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         IgniteTx tx = txEnabled() ? cache().txStart() : null;
 
         try {
-            IgniteFuture<Integer> fut1 = cache().putIfAbsentAsync("key", 1);
+            IgniteInternalFuture<Integer> fut1 = cache().putIfAbsentAsync("key", 1);
 
             assert fut1.get() == null;
             assert cache().get("key") != null && cache().get("key") == 1;
 
-            IgniteFuture<Integer> fut2 = cache().putIfAbsentAsync("key", 2);
+            IgniteInternalFuture<Integer> fut2 = cache().putIfAbsentAsync("key", 2);
 
             assert fut2.get() != null && fut2.get() == 1;
             assert cache().get("key") != null && cache().get("key") == 1;
@@ -2193,12 +2193,12 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     private void checkPutxIfAbsentAsync(boolean inTx) throws Exception {
-        IgniteFuture<Boolean> fut1 = cache().putxIfAbsentAsync("key", 1);
+        IgniteInternalFuture<Boolean> fut1 = cache().putxIfAbsentAsync("key", 1);
 
         assert fut1.get();
         assert cache().get("key") != null && cache().get("key") == 1;
 
-        IgniteFuture<Boolean> fut2 = cache().putxIfAbsentAsync("key", 2);
+        IgniteInternalFuture<Boolean> fut2 = cache().putxIfAbsentAsync("key", 2);
 
         assert !fut2.get();
         assert cache().get("key") != null && cache().get("key") == 1;
@@ -2241,8 +2241,8 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception In case of error.
      */
     public void testPutxIfAbsentAsyncConcurrent() throws Exception {
-        IgniteFuture<Boolean> fut1 = cache().putxIfAbsentAsync("key1", 1);
-        IgniteFuture<Boolean> fut2 = cache().putxIfAbsentAsync("key2", 2);
+        IgniteInternalFuture<Boolean> fut1 = cache().putxIfAbsentAsync("key1", 1);
+        IgniteInternalFuture<Boolean> fut2 = cache().putxIfAbsentAsync("key2", 2);
 
         assert fut1.get();
         assert fut2.get();
@@ -3736,7 +3736,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                 }
             });
 
-            IgniteFuture<Boolean> f = comp.future();
+            IgniteInternalFuture<Boolean> f = comp.future();
 
                 // Let another thread start.
             latch.await();
@@ -3793,7 +3793,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                 @Override public Boolean call() throws Exception {
                     syncLatch.countDown();
 
-                    IgniteFuture<Boolean> f = e.lockAsync(1000);
+                    IgniteInternalFuture<Boolean> f = e.lockAsync(1000);
 
                     try {
                         f.get(100);
@@ -3813,7 +3813,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                 }
             });
 
-            IgniteFuture<Boolean> f = comp.future();
+            IgniteInternalFuture<Boolean> f = comp.future();
 
             syncLatch.await();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java
index 4c24208..021a736 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractIteratorsSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
 
@@ -94,7 +94,7 @@ public abstract class GridCacheAbstractIteratorsSelfTest extends GridCacheAbstra
         for (int i = 0; i < gridCount(); i++)
             cache(i).removeAll();
 
-        final IgniteFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
+        final IgniteInternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 for (int i = 0; i < entryCount(); i++)
                     cache().put(KEY_PREFIX + i, i);
@@ -173,7 +173,7 @@ public abstract class GridCacheAbstractIteratorsSelfTest extends GridCacheAbstra
         for (int i = 0; i < gridCount(); i++)
             cache(i).removeAll();
 
-        final IgniteFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
+        final IgniteInternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 for (int i = 0; i < entryCount(); i++)
                     cache().put(KEY_PREFIX + i, i);
@@ -262,7 +262,7 @@ public abstract class GridCacheAbstractIteratorsSelfTest extends GridCacheAbstra
         for (int i = 0; i < gridCount(); i++)
             cache(i).removeAll();
 
-        final IgniteFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
+        final IgniteInternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 for (int i = 0; i < entryCount(); i++)
                     cache().put(KEY_PREFIX + i, i);
@@ -330,7 +330,7 @@ public abstract class GridCacheAbstractIteratorsSelfTest extends GridCacheAbstra
         for (int i = 0; i < gridCount(); i++)
             cache(i).removeAll();
 
-        final IgniteFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
+        final IgniteInternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 for (int i = 0; i < entryCount(); i++)
                     cache().put(KEY_PREFIX + i, i);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
index 466e178..fa3aeb3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractMetricsSelfTest.java
@@ -19,9 +19,9 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.GridUtils;
 import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.transactions.*;
 
@@ -116,7 +116,7 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
 
         assertEquals(cache.metrics().getAverageRemoveTime(), 0.0, 0.0);
 
-        IgniteFuture<Object> fut = cache.removeAsync(1);
+        IgniteInternalFuture<Object> fut = cache.removeAsync(1);
 
         assertEquals(1, (int)fut.get());
 
@@ -149,7 +149,7 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
 
         cache.put(key, key);
 
-        IgniteFuture<Boolean> fut = cache.removeAsync(key, key);
+        IgniteInternalFuture<Boolean> fut = cache.removeAsync(key, key);
 
         assertTrue(fut.get());
 
@@ -225,7 +225,7 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
 
         assertEquals(cache.metrics().getAverageRemoveTime(), 0.0, 0.0);
 
-        IgniteFuture<?> fut = cache.removeAllAsync(keys);
+        IgniteInternalFuture<?> fut = cache.removeAllAsync(keys);
 
         fut.get();
 
@@ -299,7 +299,7 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
         keys.add(2);
         keys.add(3);
 
-        IgniteFuture<Map<Object, Object>> fut = cache.getAllAsync(keys);
+        IgniteInternalFuture<Map<Object, Object>> fut = cache.getAllAsync(keys);
 
         fut.get();
 
@@ -340,7 +340,7 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
         assertEquals(0.0, cache.metrics().getAveragePutTime(), 0.0);
         assertEquals(0, cache.metrics().getCachePuts());
 
-        IgniteFuture<Boolean> fut = cache.putxAsync(1, 1);
+        IgniteInternalFuture<Boolean> fut = cache.putxAsync(1, 1);
 
         fut.get();
 
@@ -368,7 +368,7 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
         assertEquals(0.0, cache.metrics().getAveragePutTime(), 0.0);
         assertEquals(0.0, cache.metrics().getAverageGetTime(), 0.0);
 
-        IgniteFuture<?> fut = cache.putAsync(key, key);
+        IgniteInternalFuture<?> fut = cache.putAsync(key, key);
 
         fut.get();
 
@@ -396,7 +396,7 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
 
         assertEquals(0.0f, cache.metrics().getAveragePutTime());
 
-        IgniteFuture<Boolean> fut = cache.putxIfAbsentAsync(key, key);
+        IgniteInternalFuture<Boolean> fut = cache.putxIfAbsentAsync(key, key);
 
         fut.get();
 
@@ -423,7 +423,7 @@ public abstract class GridCacheAbstractMetricsSelfTest extends GridCacheAbstract
 
         assertEquals(0.0f, cache.metrics().getAveragePutTime());
 
-        IgniteFuture<?> fut = cache.putIfAbsentAsync(key, key);
+        IgniteInternalFuture<?> fut = cache.putIfAbsentAsync(key, key);
 
         fut.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
index 8992d33..9eceb87 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractRemoveFailureTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -120,7 +120,7 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
         // Expected values in cache.
         final Map<Integer, GridTuple<Integer>> expVals = new ConcurrentHashMap8<>();
 
-        IgniteFuture<?> updateFut = GridTestUtils.runAsync(new Callable<Void>() {
+        IgniteInternalFuture<?> updateFut = GridTestUtils.runAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 ThreadLocalRandom rnd = ThreadLocalRandom.current();
 
@@ -169,7 +169,7 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra
             }
         });
 
-        IgniteFuture<?> killFut = GridTestUtils.runAsync(new Callable<Void>() {
+        IgniteInternalFuture<?> killFut = GridTestUtils.runAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 while (!stop.get()) {
                     U.sleep(random(KILL_DELAY.get1(), KILL_DELAY.get2()));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java
index 645f245..b890acc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 
 import java.util.concurrent.atomic.*;
@@ -57,10 +57,10 @@ public class GridCacheAsyncOperationsLimitSelfTest extends GridCacheAbstractSelf
 
             cnt.incrementAndGet();
 
-            IgniteFuture<Boolean> fut = cache().putxAsync("key" + i, i);
+            IgniteInternalFuture<Boolean> fut = cache().putxAsync("key" + i, i);
 
-            fut.listenAsync(new CI1<IgniteFuture<Boolean>>() {
-                @Override public void apply(IgniteFuture<Boolean> t) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<Boolean>>() {
+                @Override public void apply(IgniteInternalFuture<Boolean> t) {
                     cnt.decrementAndGet();
 
                     max.setIfGreater(cnt.get());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapTest.java
index 15fb311..9cc6f1a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentMapTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 
@@ -92,7 +92,7 @@ public class GridCacheConcurrentMapTest extends GridCommonAbstractTest {
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> fut1 = multithreadedAsync(
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     while (!done.get()) {
@@ -112,7 +112,7 @@ public class GridCacheConcurrentMapTest extends GridCommonAbstractTest {
             3
         );
 
-        IgniteFuture<?> fut2 = multithreadedAsync(
+        IgniteInternalFuture<?> fut2 = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     while (!done.get()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
index 5ce38f9..90487ba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFinishPartitionsSelfTest.java
@@ -21,7 +21,6 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
@@ -134,10 +133,10 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
 
                 cache.get(key);
 
-                IgniteFuture<?> fut = grid.context().cache().context().partitionReleaseFuture(GRID_CNT + 1);
+                IgniteInternalFuture<?> fut = grid.context().cache().context().partitionReleaseFuture(GRID_CNT + 1);
 
-                fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> e) {
+                fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> e) {
                         latch.countDown();
                     }
                 });
@@ -197,9 +196,9 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
 
             GridCacheAdapter<String, Integer> internal = grid.internalCache();
 
-            IgniteFuture<?> nearFut = internal.context().mvcc().finishKeys(Collections.singletonList(key), 2);
+            IgniteInternalFuture<?> nearFut = internal.context().mvcc().finishKeys(Collections.singletonList(key), 2);
 
-            IgniteFuture<?> dhtFut = internal.context().near().dht().context().mvcc().finishKeys(
+            IgniteInternalFuture<?> dhtFut = internal.context().near().dht().context().mvcc().finishKeys(
                 Collections.singletonList(key), 2);
 
             assert !nearFut.isDone();
@@ -233,12 +232,12 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
 
         info("Start time: " + start);
 
-        IgniteFuture<?> fut = ctx.partitionReleaseFuture(GRID_CNT + 1);
+        IgniteInternalFuture<?> fut = ctx.partitionReleaseFuture(GRID_CNT + 1);
 
         assert fut != null;
 
-        fut.listenAsync(new CI1<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> e) {
+        fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> e) {
                 end.set(System.currentTimeMillis());
 
                 latch.countDown();
@@ -293,12 +292,12 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
 
             info("Start time: " + start);
 
-            IgniteFuture<?> fut = ctx.partitionReleaseFuture(GRID_CNT + 1);
+            IgniteInternalFuture<?> fut = ctx.partitionReleaseFuture(GRID_CNT + 1);
 
             assert fut != null;
 
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> e) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> e) {
                     end.set(System.currentTimeMillis());
 
                     latch.countDown();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java
index 65cb986..09d92d8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
@@ -75,7 +75,7 @@ public class GridCacheFullTextQueryMultithreadedSelfTest extends GridCacheAbstra
 
         final GridCache<Integer, H2TextValue> c = grid(0).cache(null);
 
-        IgniteFuture<?> fut1 = multithreadedAsync(new Callable() {
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(new Callable() {
                 @Override public Object call() throws Exception {
                     for (int i = 0; i < keyCnt; i++) {
                         c.putx(i, new H2TextValue(txt));
@@ -98,7 +98,7 @@ public class GridCacheFullTextQueryMultithreadedSelfTest extends GridCacheAbstra
 
         final AtomicBoolean stop = new AtomicBoolean();
 
-        IgniteFuture<?> fut2 = multithreadedAsync(new Callable() {
+        IgniteInternalFuture<?> fut2 = multithreadedAsync(new Callable() {
                 @Override public Object call() throws Exception {
                     int cnt = 0;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
index c605840..64cc3a6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -126,7 +126,7 @@ public abstract class GridCacheGetAndTransformStoreAbstractTest extends GridComm
 
             final Processor entryProcessor = new Processor();
 
-            IgniteFuture<?> fut = multithreadedAsync(
+            IgniteInternalFuture<?> fut = multithreadedAsync(
                 new Callable<Object>() {
                     @Override public Object call() throws Exception {
                         IgniteCache<Integer, String> c = jcache(ThreadLocalRandom.current().nextInt(3));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
index f1637df..40aa2bf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockAbstractSelfTest.java
@@ -393,7 +393,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
         final CountDownLatch unlockLatch = new CountDownLatch(1);
         final CountDownLatch lockLatch = new CountDownLatch(1);
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     Lock lock = cache.lock(key1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
index 96879cf..852e84d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -326,8 +327,8 @@ public class GridCacheGroupLockFailoverSelfTest extends GridCommonAbstractTest {
 
         ComputeTaskFuture<Void> fut = comp.future();
 
-        fut.listenAsync(new CI1<IgniteFuture<Void>>() {
-            @Override public void apply(IgniteFuture<Void> f) {
+        fut.listenAsync(new CI1<IgniteInternalFuture<Void>>() {
+            @Override public void apply(IgniteInternalFuture<Void> f) {
                 ComputeTaskFuture taskFut = (ComputeTaskFuture)f;
 
                 boolean fail = false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java
index 50689c6..3a7ac40 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheIncrementTransformTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -113,7 +113,7 @@ public class GridCacheIncrementTransformTest extends GridCommonAbstractTest {
         final AtomicBoolean stop = new AtomicBoolean();
         final AtomicReference<Throwable> error = new AtomicReference<>();
 
-        IgniteFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     Random rnd = new Random();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLuceneQueryIndexTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLuceneQueryIndexTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLuceneQueryIndexTest.java
index 91b20fe..0530419 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLuceneQueryIndexTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLuceneQueryIndexTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -109,7 +109,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest {
 
         final int keyCnt = 10000;
 
-        final IgniteFuture<?> fut = multithreadedAsync(
+        final IgniteInternalFuture<?> fut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     int threadIdx = threadIdxGen.getAndIncrement() % 2;
@@ -129,7 +129,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest {
             },
             10);
 
-        IgniteFuture<?> fut1 = multithreadedAsync(
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     while (!fut.isDone()) {
@@ -166,7 +166,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest {
 
         final int keyCnt = 10000;
 
-        final IgniteFuture<?> fut = multithreadedAsync(
+        final IgniteInternalFuture<?> fut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     int threadIdx = threadIdxGen.getAndIncrement() % 2;
@@ -200,7 +200,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest {
             },
             10);
 
-        IgniteFuture<?> fut1 = multithreadedAsync(
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     while (!fut.isDone()) {
@@ -239,7 +239,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest {
 
         final ObjectValue val = new ObjectValue("String value");
 
-        final IgniteFuture<?> fut = multithreadedAsync(
+        final IgniteInternalFuture<?> fut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     int threadIdx = threadIdxGen.getAndIncrement() % 2;
@@ -273,7 +273,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest {
             },
             10);
 
-        IgniteFuture<?> fut1 = multithreadedAsync(
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     while (!fut.isDone()) {
@@ -315,7 +315,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest {
         for (int i = 0; i < vals.length; i++)
             vals[i] = new ObjectValue("Object value " + i);
 
-        final IgniteFuture<?> fut = multithreadedAsync(
+        final IgniteInternalFuture<?> fut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     int threadIdx = threadIdxGen.getAndIncrement() % 2;
@@ -349,7 +349,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest {
             },
             1);
 
-        IgniteFuture<?> fut1 = multithreadedAsync(
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     while (!fut.isDone()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
index 68f0638..0229318 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMissingCommitVersionSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -115,7 +116,7 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes
 
         log.info("Trying to update " + failedKey);
 
-        IgniteFuture<?> fut = cache.putAsync(failedKey, 2);
+        IgniteInternalFuture<?> fut = cache.putAsync(failedKey, 2);
 
         try {
             fut.get(5000);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
index 71497a9..b7f8239 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java
@@ -80,7 +80,7 @@ public class GridCacheMixedPartitionExchangeSelfTest extends GridCommonAbstractT
 
             final AtomicBoolean finished = new AtomicBoolean();
 
-            IgniteFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new IgniteCallable<Object>() {
+            IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new IgniteCallable<Object>() {
                 @Override public Object call() throws Exception {
                     Random rnd = new Random();
 
@@ -139,7 +139,7 @@ public class GridCacheMixedPartitionExchangeSelfTest extends GridCommonAbstractT
 
                 GridCacheContext<Object, Object> cctx = grid.internalCache(null).context();
 
-                IgniteFuture<Long> verFut = cctx.affinity().affinityReadyFuture(topVer);
+                IgniteInternalFuture<Long> verFut = cctx.affinity().affinityReadyFuture(topVer);
 
                 assertEquals((Long)topVer, verFut.get());
                 assertEquals((Long)topVer, cctx.topologyVersionFuture().get());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMultiUpdateLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMultiUpdateLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMultiUpdateLockSelfTest.java
index 41ef1c0..7eebab8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMultiUpdateLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMultiUpdateLockSelfTest.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.spi.checkpoint.noop.*;
@@ -116,7 +115,7 @@ public class GridCacheMultiUpdateLockSelfTest extends GridCommonAbstractTest {
 
             long topVer = cache.beginMultiUpdate();
 
-            IgniteFuture<?> startFut;
+            IgniteInternalFuture<?> startFut;
 
             try {
                 assertEquals(3, topVer);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
index d24265c..096f448 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.*;
 
@@ -272,7 +273,7 @@ public abstract class GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest extend
         final int THREADS = 5;
         final int ITERATIONS_PER_THREAD = iterations();
 
-        IgniteFuture<Long> putFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+        IgniteInternalFuture<Long> putFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 for (int i = 0; i < ITERATIONS_PER_THREAD; i++) {
                     if (i % 1000 == 0)
@@ -287,7 +288,7 @@ public abstract class GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest extend
 
         final AtomicBoolean stop = new AtomicBoolean();
 
-        IgniteFuture<Long> getFut;
+        IgniteInternalFuture<Long> getFut;
 
         try {
             getFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePreloadingEvictionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePreloadingEvictionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePreloadingEvictionsSelfTest.java
index cf164d4..6902ce2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePreloadingEvictionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePreloadingEvictionsSelfTest.java
@@ -113,7 +113,7 @@ public class GridCachePreloadingEvictionsSelfTest extends GridCommonAbstractTest
 
             int oldSize = cache1.size();
 
-            IgniteFuture fut = multithreadedAsync(
+            IgniteInternalFuture fut = multithreadedAsync(
                 new Callable<Object>() {
                     @Nullable @Override public Object call() throws Exception {
                         startLatch.await();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
index a21eb18..80f62d3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePutAllFailoverSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
@@ -258,8 +259,8 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
 
                     resQueue.put(fut); // Blocks if queue is full.
 
-                    fut.listenAsync(new CI1<IgniteFuture<Void>>() {
-                        @Override public void apply(IgniteFuture<Void> f) {
+                    fut.listenAsync(new CI1<IgniteInternalFuture<Void>>() {
+                        @Override public void apply(IgniteInternalFuture<Void> f) {
                             ComputeTaskFuture<?> taskFut = (ComputeTaskFuture<?>)f;
 
                             try {
@@ -430,8 +431,8 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
 
                     resQueue.put(fut); // Blocks if queue is full.
 
-                    fut.listenAsync(new CI1<IgniteFuture<Void>>() {
-                        @Override public void apply(IgniteFuture<Void> f) {
+                    fut.listenAsync(new CI1<IgniteInternalFuture<Void>>() {
+                        @Override public void apply(IgniteInternalFuture<Void> f) {
                             ComputeTaskFuture<?> taskFut = (ComputeTaskFuture<?>)f;
 
                             try {
@@ -481,8 +482,8 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
 
                 resQueue.put(fut); // Blocks if queue is full.
 
-                fut.listenAsync(new CI1<IgniteFuture<Void>>() {
-                    @Override public void apply(IgniteFuture<Void> f) {
+                fut.listenAsync(new CI1<IgniteInternalFuture<Void>>() {
+                    @Override public void apply(IgniteInternalFuture<Void> f) {
                         ComputeTaskFuture<?> taskFut = (ComputeTaskFuture<?>)f;
 
                         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java
index 97b2ba3..61af5c5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReferenceCleanupSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -390,7 +390,7 @@ public class GridCacheReferenceCleanupSelfTest extends GridCommonAbstractTest {
 
                     refs.add(new WeakReference<Object>(cacheContext(cache)));
 
-                    Collection<IgniteFuture<?>> futs = new ArrayList<>(1000);
+                    Collection<IgniteInternalFuture<?>> futs = new ArrayList<>(1000);
 
                     for (int i = 0; i < 1000; i++) {
                         TestValue val = new TestValue(i);
@@ -400,7 +400,7 @@ public class GridCacheReferenceCleanupSelfTest extends GridCommonAbstractTest {
                         futs.add(cache.putxAsync(i, val));
                     }
 
-                    for (IgniteFuture<?> fut : futs)
+                    for (IgniteInternalFuture<?> fut : futs)
                         fut.get();
                 }
                 finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java
index 77ab9a1..d8e95c2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java
@@ -21,8 +21,8 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.spi.communication.tcp.*;
@@ -137,7 +137,7 @@ public class GridCacheReplicatedSynchronousCommitTest extends GridCommonAbstract
             GridCache<Integer, String> cache1 = ignite1.cache(null);
             GridCache<Integer, String> cache3 = ignite3.cache(null);
 
-            IgniteFuture<?> fut = multithreadedAsync(
+            IgniteInternalFuture<?> fut = multithreadedAsync(
                 new Callable<Object>() {
                     @Nullable @Override public Object call() throws Exception {
                         Thread.sleep(1000);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStopSelfTest.java
index b051b80..f01b9fe 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStopSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.transactions.*;
@@ -138,7 +138,7 @@ public class GridCacheStopSelfTest extends GridCommonAbstractTest {
             assertEquals(atomic ? ATOMIC : TRANSACTIONAL, cache.configuration().getAtomicityMode());
             assertEquals(replicated ? REPLICATED : PARTITIONED, cache.configuration().getCacheMode());
 
-            Collection<IgniteFuture<?>> putFuts = new ArrayList<>();
+            Collection<IgniteInternalFuture<?>> putFuts = new ArrayList<>();
 
             for (int j = 0; j < PUT_THREADS; j++) {
                 final int key = j;
@@ -175,7 +175,7 @@ public class GridCacheStopSelfTest extends GridCommonAbstractTest {
 
             stopGrid(0);
 
-            for (IgniteFuture<?> fut : putFuts) {
+            for (IgniteInternalFuture<?> fut : putFuts) {
                 try {
                     fut.get();
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapPreloadSelfTest.java
index d6a77c7..ea04a12 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapPreloadSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -149,7 +149,7 @@ public class GridCacheSwapPreloadSelfTest extends GridCommonAbstractTest {
     /** @throws Exception If failed. */
     private void checkSwapMultithreaded() throws Exception {
         final AtomicBoolean done = new AtomicBoolean();
-        IgniteFuture<?> fut = null;
+        IgniteInternalFuture<?> fut = null;
 
         try {
             startGrid(0);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerLoadTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerLoadTest.java
index 2140b97..b0b1e50 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerLoadTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
 import javax.cache.expiry.*;
@@ -44,7 +43,7 @@ public class GridCacheTtlManagerLoadTest extends GridCacheTtlManagerSelfTest {
         try {
             final AtomicBoolean stop = new AtomicBoolean();
 
-            IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     IgniteCache<Object,Object> cache = g.jcache(null).
                         withExpiryPolicy(new TouchedExpiryPolicy(new Duration(MILLISECONDS, 1000)));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
index 457c7b4..1d523ac 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheVariableTopologySelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -116,7 +116,7 @@ public class GridCacheVariableTopologySelfTest extends GridCommonAbstractTest {
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CAX() {
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CAX() {
             /** */
             private int cnt;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java
index ff954de..e751b8f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -99,7 +99,7 @@ public abstract class GridCacheWriteBehindStoreAbstractSelfTest extends GridComm
 
         final AtomicInteger operations = new AtomicInteger();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @SuppressWarnings({"NullableProblems"})
             @Override public void run() {
                 // Initialize key set for this thread.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java
index 432c34d..7bf1490 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.transactions.*;
@@ -270,7 +270,7 @@ public abstract class GridCacheWriteBehindStoreAbstractTest extends GridCommonAb
 
         final GridCache<Integer, String> cache = cache();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @SuppressWarnings({"NullableProblems"})
             @Override public void run() {
                 // Initialize key set for this thread.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreSelfTest.java
index 65eaa81..bc43414 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStoreSelfTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.cache;
 
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jdk8.backport.*;
@@ -139,7 +139,7 @@ public class GridCacheWriteBehindStoreSelfTest extends GridCacheWriteBehindStore
 
             final AtomicInteger actualPutCnt = new AtomicInteger();
 
-            IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
                 @SuppressWarnings({"NullableProblems"})
                 @Override public void run() {
                     try {
@@ -199,7 +199,7 @@ public class GridCacheWriteBehindStoreSelfTest extends GridCacheWriteBehindStore
         try {
             final AtomicBoolean running = new AtomicBoolean(true);
 
-            IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
                 @SuppressWarnings({"NullableProblems"})
                 @Override public void run() {
                     try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
index 29c7781..6f0b1d6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheEntryListenerAbstractTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.continuous.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -246,7 +245,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
             final AtomicBoolean done = new AtomicBoolean();
 
-            IgniteFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
                 @Override public Void call() throws Exception {
                     U.sleep(500);
 
@@ -289,7 +288,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
+        IgniteInternalFuture<?> fut = GridTestUtils.runAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 assertFalse(done.get());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java
index 63c42c9..784bd38 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteCacheInvokeAbstractTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
@@ -153,7 +153,7 @@ public abstract class IgniteCacheInvokeAbstractTest extends IgniteCacheAbstractT
 
             assertNull(asyncCache.invoke(key, incProcessor));
 
-            IgniteFuture<Integer> fut = asyncCache.future();
+            IgniteInternalFuture<Integer> fut = asyncCache.future();
 
             assertNotNull(fut);
 
@@ -404,7 +404,7 @@ public abstract class IgniteCacheInvokeAbstractTest extends IgniteCacheAbstractT
 
         assertNull(asyncCache.invokeAll(keys, new IncrementProcessor()));
 
-        IgniteFuture<Map<Integer, EntryProcessorResult<Integer>>> fut = asyncCache.future();
+        IgniteInternalFuture<Map<Integer, EntryProcessorResult<Integer>>> fut = asyncCache.future();
 
         resMap = fut.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
index af541c7..371c281 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiThreadedAbstractTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cache.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.testframework.*;
 import org.jetbrains.annotations.*;
@@ -218,7 +218,7 @@ public abstract class IgniteTxMultiThreadedAbstractTest extends IgniteTxAbstract
 
         cache.put(key, 0L);
 
-        List<IgniteFuture<Collection<Long>>> futs = new ArrayList<>(THREADS);
+        List<IgniteInternalFuture<Collection<Long>>> futs = new ArrayList<>(THREADS);
 
         for (int i = 0; i < THREADS; i++) {
             futs.add(GridTestUtils.runAsync(new Callable<Collection<Long>>() {
@@ -251,7 +251,7 @@ public abstract class IgniteTxMultiThreadedAbstractTest extends IgniteTxAbstract
 
         List<Collection<Long>> cols = new ArrayList<>(THREADS);
 
-        for (IgniteFuture<Collection<Long>> fut : futs) {
+        for (IgniteInternalFuture<Collection<Long>> fut : futs) {
             Collection<Long> col = fut.get();
 
             assertEquals(ITERATIONS, col.size());