http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractIteratorsSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractIteratorsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractIteratorsSelfTest.java index 6982236..c75d7dd 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractIteratorsSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractIteratorsSelfTest.java @@ -86,7 +86,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 InternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() { @Override public void applyx() throws IgniteCheckedException { for (int i = 0; i < entryCount(); i++) cache().put(KEY_PREFIX + i, i); @@ -165,7 +165,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 InternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() { @Override public void applyx() throws IgniteCheckedException { for (int i = 0; i < entryCount(); i++) cache().put(KEY_PREFIX + i, i); @@ -254,7 +254,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 InternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new CAX() { @Override public void applyx() throws IgniteCheckedException { for (int i = 0; i < entryCount(); i++) cache().put(KEY_PREFIX + i, i); @@ -322,7 +322,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 InternalFuture<?> 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/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java index 6686911..169d32a 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractRemoveFailureTest.java @@ -11,7 +11,6 @@ package org.gridgain.grid.kernal.processors.cache; import org.apache.ignite.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.util.typedef.*; import org.gridgain.grid.util.typedef.internal.*; @@ -121,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>() { + InternalFuture<?> updateFut = GridTestUtils.runAsync(new Callable<Void>() { @Override public Void call() throws Exception { ThreadLocalRandom rnd = ThreadLocalRandom.current(); @@ -170,7 +169,7 @@ public abstract class GridCacheAbstractRemoveFailureTest extends GridCacheAbstra } }); - IgniteFuture<?> killFut = GridTestUtils.runAsync(new Callable<Void>() { + InternalFuture<?> 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/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java index 2cee21a..bda7a4e 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAsyncOperationsLimitSelfTest.java @@ -49,10 +49,10 @@ public class GridCacheAsyncOperationsLimitSelfTest extends GridCacheAbstractSelf cnt.incrementAndGet(); - IgniteFuture<Boolean> fut = cache().putxAsync("key" + i, i); + InternalFuture<Boolean> fut = cache().putxAsync("key" + i, i); - fut.listenAsync(new CI1<IgniteFuture<Boolean>>() { - @Override public void apply(IgniteFuture<Boolean> t) { + fut.listenAsync(new CI1<InternalFuture<Boolean>>() { + @Override public void apply(InternalFuture<Boolean> t) { cnt.decrementAndGet(); max.setIfGreater(cnt.get()); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentMapTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentMapTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentMapTest.java index 7c38ee8..578b027 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentMapTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentMapTest.java @@ -84,7 +84,7 @@ public class GridCacheConcurrentMapTest extends GridCommonAbstractTest { final AtomicBoolean done = new AtomicBoolean(); - IgniteFuture<?> fut1 = multithreadedAsync( + InternalFuture<?> fut1 = multithreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { while (!done.get()) { @@ -104,7 +104,7 @@ public class GridCacheConcurrentMapTest extends GridCommonAbstractTest { 3 ); - IgniteFuture<?> fut2 = multithreadedAsync( + InternalFuture<?> 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/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java index dcbf820..d462da6 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java @@ -15,7 +15,6 @@ import org.apache.ignite.compute.*; import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; import org.apache.ignite.resources.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.cache.affinity.*; import org.gridgain.grid.cache.datastructures.*; @@ -305,7 +304,7 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest { comp.execute(RequestTask.class, new Message(terminalId, nodeId)); - ComputeTaskFuture<Void> f1 = comp.future(); + InternalComputeTaskFuture<Void> f1 = comp.future(); submitTime.setIfGreater(System.currentTimeMillis() - submitTime1); @@ -315,7 +314,7 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest { comp.execute(ResponseTask.class, new Message(terminalId, nodeId)); - ComputeTaskFuture<Void> f2 = comp.future(); + InternalComputeTaskFuture<Void> f2 = comp.future(); submitTime.setIfGreater(System.currentTimeMillis() - submitTime1); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java index d5e744c..9b5f8b3 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFinishPartitionsSelfTest.java @@ -123,10 +123,10 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest cache.get(key); - IgniteFuture<?> fut = grid.context().cache().context().partitionReleaseFuture(GRID_CNT + 1); + InternalFuture<?> fut = grid.context().cache().context().partitionReleaseFuture(GRID_CNT + 1); - fut.listenAsync(new CI1<IgniteFuture<?>>() { - @Override public void apply(IgniteFuture<?> e) { + fut.listenAsync(new CI1<InternalFuture<?>>() { + @Override public void apply(InternalFuture<?> e) { latch.countDown(); } }); @@ -186,9 +186,9 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest GridCacheAdapter<String, Integer> internal = grid.internalCache(); - IgniteFuture<?> nearFut = internal.context().mvcc().finishKeys(Collections.singletonList(key), 2); + InternalFuture<?> nearFut = internal.context().mvcc().finishKeys(Collections.singletonList(key), 2); - IgniteFuture<?> dhtFut = internal.context().near().dht().context().mvcc().finishKeys( + InternalFuture<?> dhtFut = internal.context().near().dht().context().mvcc().finishKeys( Collections.singletonList(key), 2); assert !nearFut.isDone(); @@ -220,12 +220,12 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest info("Start time: " + start); - IgniteFuture<?> fut = ctx.partitionReleaseFuture(GRID_CNT + 1); + InternalFuture<?> fut = ctx.partitionReleaseFuture(GRID_CNT + 1); assert fut != null; - fut.listenAsync(new CI1<IgniteFuture<?>>() { - @Override public void apply(IgniteFuture<?> e) { + fut.listenAsync(new CI1<InternalFuture<?>>() { + @Override public void apply(InternalFuture<?> e) { end.set(System.currentTimeMillis()); latch.countDown(); @@ -274,12 +274,12 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest info("Start time: " + start); - IgniteFuture<?> fut = ctx.partitionReleaseFuture(GRID_CNT + 1); + InternalFuture<?> fut = ctx.partitionReleaseFuture(GRID_CNT + 1); assert fut != null; - fut.listenAsync(new CI1<IgniteFuture<?>>() { - @Override public void apply(IgniteFuture<?> e) { + fut.listenAsync(new CI1<InternalFuture<?>>() { + @Override public void apply(InternalFuture<?> e) { end.set(System.currentTimeMillis()); latch.countDown(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java index 28e6073..852a7d3 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheFullTextQueryMultithreadedSelfTest.java @@ -67,7 +67,7 @@ public class GridCacheFullTextQueryMultithreadedSelfTest extends GridCacheAbstra final GridCache<Integer, H2TextValue> c = grid(0).cache(null); - IgniteFuture<?> fut1 = multithreadedAsync(new Callable() { + InternalFuture<?> fut1 = multithreadedAsync(new Callable() { @Override public Object call() throws Exception { for (int i = 0; i < keyCnt; i++) { c.putx(i, new H2TextValue(txt)); @@ -90,7 +90,7 @@ public class GridCacheFullTextQueryMultithreadedSelfTest extends GridCacheAbstra final AtomicBoolean stop = new AtomicBoolean(); - IgniteFuture<?> fut2 = multithreadedAsync(new Callable() { + InternalFuture<?> fut2 = multithreadedAsync(new Callable() { @Override public Object call() throws Exception { int cnt = 0; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java index 8a929bf..c397d69 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGetAndTransformStoreAbstractTest.java @@ -110,7 +110,7 @@ public abstract class GridCacheGetAndTransformStoreAbstractTest extends GridComm final IgniteClosure<String, String> trans = new TransformClosure(); - IgniteFuture<?> fut = multithreadedAsync( + InternalFuture<?> fut = multithreadedAsync( new Callable<Object>() { @Override public Object call() throws Exception { GridCache<Integer, String> c = cache(ThreadLocalRandom.current().nextInt(3)); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java index f9fe5e9..571cc1f 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockAbstractSelfTest.java @@ -13,7 +13,6 @@ import org.apache.ignite.*; import org.apache.ignite.configuration.*; import org.apache.ignite.events.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.cache.affinity.*; import org.gridgain.grid.cache.store.*; @@ -359,7 +358,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr final CountDownLatch unlockLatch = new CountDownLatch(1); final CountDownLatch lockLatch = new CountDownLatch(1); - IgniteFuture<?> fut = multithreadedAsync(new Runnable() { + InternalFuture<?> fut = multithreadedAsync(new Runnable() { @Override public void run() { try { assertTrue(cache.lock(key1, 0)); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java index c812755..f7a5891 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java @@ -15,7 +15,6 @@ import org.apache.ignite.cluster.*; import org.apache.ignite.compute.*; import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; @@ -76,7 +75,7 @@ public class GridCacheGroupLockFailoverSelfTest extends GridCommonAbstractTest { * Result future queue (restrict the queue size * to 50 in order to prevent in-memory data grid from over loading). */ - private final BlockingQueue<ComputeTaskFuture<?>> resQueue = new LinkedBlockingQueue<>(10); + private final BlockingQueue<InternalComputeTaskFuture<?>> resQueue = new LinkedBlockingQueue<>(10); /** * @return {@code True} if test should use optimistic transactions. @@ -317,11 +316,11 @@ public class GridCacheGroupLockFailoverSelfTest extends GridCommonAbstractTest { comp.execute(new GridCacheGroupLockPutTask(preferredNodeId, CACHE_NAME, optimisticTx()), dataChunk); - ComputeTaskFuture<Void> fut = comp.future(); + InternalComputeTaskFuture<Void> fut = comp.future(); - fut.listenAsync(new CI1<IgniteFuture<Void>>() { - @Override public void apply(IgniteFuture<Void> f) { - ComputeTaskFuture taskFut = (ComputeTaskFuture)f; + fut.listenAsync(new CI1<InternalFuture<Void>>() { + @Override public void apply(InternalFuture<Void> f) { + InternalComputeTaskFuture taskFut = (InternalComputeTaskFuture)f; boolean fail = false; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java index 7a9ac04..11b0139 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java @@ -14,7 +14,6 @@ import org.apache.ignite.cluster.*; import org.apache.ignite.compute.*; import org.apache.ignite.lang.*; import org.apache.ignite.resources.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.util.typedef.*; import org.jetbrains.annotations.*; @@ -61,7 +60,7 @@ class GridCacheGroupLockPutTask extends ComputeTaskAdapter<Collection<Integer>, * @return Map of grid jobs assigned to subgrid node. Unless {@link org.apache.ignite.compute.ComputeTaskContinuousMapper} is injected into task, if * {@code null} or empty map is returned, exception will be thrown. * @throws IgniteCheckedException If mapping could not complete successfully. This exception will be thrown out of {@link - * org.apache.ignite.compute.ComputeTaskFuture#get()} method. + * InternalComputeTaskFuture#get()} method. */ @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable final Collection<Integer> data) throws IgniteCheckedException { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheIncrementTransformTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheIncrementTransformTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheIncrementTransformTest.java index 44b8618..9cf1c36 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheIncrementTransformTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheIncrementTransformTest.java @@ -105,7 +105,7 @@ public class GridCacheIncrementTransformTest extends GridCommonAbstractTest { final AtomicBoolean stop = new AtomicBoolean(); final AtomicReference<Throwable> error = new AtomicReference<>(); - IgniteFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() { + InternalFuture<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/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheLuceneQueryIndexTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheLuceneQueryIndexTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheLuceneQueryIndexTest.java index ce2ef84..54bd6ec 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheLuceneQueryIndexTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheLuceneQueryIndexTest.java @@ -101,7 +101,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest { final int keyCnt = 10000; - final IgniteFuture<?> fut = multithreadedAsync( + final InternalFuture<?> fut = multithreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { int threadIdx = threadIdxGen.getAndIncrement() % 2; @@ -121,7 +121,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest { }, 10); - IgniteFuture<?> fut1 = multithreadedAsync( + InternalFuture<?> fut1 = multithreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { while (!fut.isDone()) { @@ -158,7 +158,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest { final int keyCnt = 10000; - final IgniteFuture<?> fut = multithreadedAsync( + final InternalFuture<?> fut = multithreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { int threadIdx = threadIdxGen.getAndIncrement() % 2; @@ -192,7 +192,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest { }, 10); - IgniteFuture<?> fut1 = multithreadedAsync( + InternalFuture<?> fut1 = multithreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { while (!fut.isDone()) { @@ -231,7 +231,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest { final ObjectValue val = new ObjectValue("String value"); - final IgniteFuture<?> fut = multithreadedAsync( + final InternalFuture<?> fut = multithreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { int threadIdx = threadIdxGen.getAndIncrement() % 2; @@ -265,7 +265,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest { }, 10); - IgniteFuture<?> fut1 = multithreadedAsync( + InternalFuture<?> fut1 = multithreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { while (!fut.isDone()) { @@ -307,7 +307,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 InternalFuture<?> fut = multithreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { int threadIdx = threadIdxGen.getAndIncrement() % 2; @@ -341,7 +341,7 @@ public class GridCacheLuceneQueryIndexTest extends GridCommonAbstractTest { }, 1); - IgniteFuture<?> fut1 = multithreadedAsync( + InternalFuture<?> 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/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMissingCommitVersionSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMissingCommitVersionSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMissingCommitVersionSelfTest.java index a22ae6f..b06456c 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMissingCommitVersionSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMissingCommitVersionSelfTest.java @@ -107,12 +107,12 @@ public class GridCacheMissingCommitVersionSelfTest extends GridCommonAbstractTes log.info("Trying to update " + failedKey); - IgniteFuture<?> fut = cache.putAsync(failedKey, 2); + InternalFuture<?> fut = cache.putAsync(failedKey, 2); try { fut.get(5000); } - catch (IgniteFutureTimeoutException ignore) { + catch (InternalFutureTimeoutException ignore) { fail("Put failed to finish in 5s."); } } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java index 1db99d9..3f5210f 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMixedPartitionExchangeSelfTest.java @@ -71,7 +71,7 @@ public class GridCacheMixedPartitionExchangeSelfTest extends GridCommonAbstractT final AtomicBoolean finished = new AtomicBoolean(); - IgniteFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new IgniteCallable<Object>() { + InternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new IgniteCallable<Object>() { @Override public Object call() throws Exception { Random rnd = new Random(); @@ -130,7 +130,7 @@ public class GridCacheMixedPartitionExchangeSelfTest extends GridCommonAbstractT GridCacheContext<Object, Object> cctx = grid.internalCache(null).context(); - IgniteFuture<Long> verFut = cctx.affinity().affinityReadyFuture(topVer); + InternalFuture<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/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMultiUpdateLockSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMultiUpdateLockSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMultiUpdateLockSelfTest.java index 60ebe1a..f07f43f 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMultiUpdateLockSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheMultiUpdateLockSelfTest.java @@ -107,7 +107,7 @@ public class GridCacheMultiUpdateLockSelfTest extends GridCommonAbstractTest { long topVer = cache.beginMultiUpdate(); - IgniteFuture<?> startFut; + InternalFuture<?> startFut; try { assertEquals(3, topVer); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java index a57799b..b9cd1db 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest.java @@ -261,7 +261,7 @@ public abstract class GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest extend final int THREADS = 5; final int ITERATIONS_PER_THREAD = iterations(); - IgniteFuture<Long> putFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() { + InternalFuture<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) @@ -276,7 +276,7 @@ public abstract class GridCacheOffHeapMultiThreadedUpdateAbstractSelfTest extend final AtomicBoolean stop = new AtomicBoolean(); - IgniteFuture<Long> getFut; + InternalFuture<Long> getFut; try { getFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePreloadingEvictionsSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePreloadingEvictionsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePreloadingEvictionsSelfTest.java index 91f45c1..4d5bab2 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePreloadingEvictionsSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePreloadingEvictionsSelfTest.java @@ -106,7 +106,7 @@ public class GridCachePreloadingEvictionsSelfTest extends GridCommonAbstractTest int oldSize = cache1.size(); - IgniteFuture fut = multithreadedAsync( + InternalFuture fut = multithreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { startLatch.await(); @@ -177,10 +177,10 @@ public class GridCachePreloadingEvictionsSelfTest extends GridCommonAbstractTest * @param ignite1 Grid 1. * @param ignite2 Grid 2. * @param oldSize Old size, stable size should be . - * @throws GridInterruptedException If interrupted. + * @throws InternalInterruptedException If interrupted. */ private void sleepUntilCashesEqualize(final Ignite ignite1, final Ignite ignite2, final int oldSize) - throws GridInterruptedException { + throws InternalInterruptedException { info("Sleeping..."); assertTrue(GridTestUtils.waitForCondition(new PA() { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java index 7998316..0fa4709 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java @@ -8,7 +8,6 @@ import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; import org.apache.ignite.resources.*; import org.apache.ignite.spi.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; @@ -72,7 +71,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest { * Result future queue (restrict the queue size * to 50 in order to prevent in-memory data grid from over loading). */ - private final BlockingQueue<ComputeTaskFuture<?>> resQueue = new LinkedBlockingQueue<>(50); + private final BlockingQueue<InternalComputeTaskFuture<?>> resQueue = new LinkedBlockingQueue<>(50); /** Test failover SPI. */ private MasterFailoverSpi failoverSpi = new MasterFailoverSpi((IgnitePredicate)workerNodesFilter); @@ -238,13 +237,13 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest { CACHE_NAME), dataChunk); - ComputeTaskFuture<Void> fut = comp.future(); + InternalComputeTaskFuture<Void> fut = comp.future(); resQueue.put(fut); // Blocks if queue is full. - fut.listenAsync(new CI1<IgniteFuture<Void>>() { - @Override public void apply(IgniteFuture<Void> f) { - ComputeTaskFuture<?> taskFut = (ComputeTaskFuture<?>)f; + fut.listenAsync(new CI1<InternalFuture<Void>>() { + @Override public void apply(InternalFuture<Void> f) { + InternalComputeTaskFuture<?> taskFut = (InternalComputeTaskFuture<?>)f; try { taskFut.get(); //if something went wrong - we'll get exception here @@ -410,13 +409,13 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest { comp.execute(new GridCachePutAllTask(nodeId, CACHE_NAME), data); - ComputeTaskFuture<Void> fut = comp.future(); + InternalComputeTaskFuture<Void> fut = comp.future(); resQueue.put(fut); // Blocks if queue is full. - fut.listenAsync(new CI1<IgniteFuture<Void>>() { - @Override public void apply(IgniteFuture<Void> f) { - ComputeTaskFuture<?> taskFut = (ComputeTaskFuture<?>)f; + fut.listenAsync(new CI1<InternalFuture<Void>>() { + @Override public void apply(InternalFuture<Void> f) { + InternalComputeTaskFuture<?> taskFut = (InternalComputeTaskFuture<?>)f; try { taskFut.get(); //if something went wrong - we'll get exception here @@ -461,13 +460,13 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest { for (Map.Entry<UUID, Collection<Integer>> entry : dataChunks.entrySet()) { comp.execute(new GridCachePutAllTask(entry.getKey(), CACHE_NAME), entry.getValue()); - ComputeTaskFuture<Void> fut = comp.future(); + InternalComputeTaskFuture<Void> fut = comp.future(); resQueue.put(fut); // Blocks if queue is full. - fut.listenAsync(new CI1<IgniteFuture<Void>>() { - @Override public void apply(IgniteFuture<Void> f) { - ComputeTaskFuture<?> taskFut = (ComputeTaskFuture<?>)f; + fut.listenAsync(new CI1<InternalFuture<Void>>() { + @Override public void apply(InternalFuture<Void> f) { + InternalComputeTaskFuture<?> taskFut = (InternalComputeTaskFuture<?>)f; try { taskFut.get(); //if something went wrong - we'll get exception here http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java index 80c4747..41bf196 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReferenceCleanupSelfTest.java @@ -382,7 +382,7 @@ public class GridCacheReferenceCleanupSelfTest extends GridCommonAbstractTest { refs.add(new WeakReference<Object>(cacheContext(cache))); - Collection<IgniteFuture<?>> futs = new ArrayList<>(1000); + Collection<InternalFuture<?>> futs = new ArrayList<>(1000); for (int i = 0; i < 1000; i++) { TestValue val = new TestValue(i); @@ -392,7 +392,7 @@ public class GridCacheReferenceCleanupSelfTest extends GridCommonAbstractTest { futs.add(cache.putxAsync(i, val)); } - for (IgniteFuture<?> fut : futs) + for (InternalFuture<?> fut : futs) fut.get(); } finally { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java index ed6f067..d7d5aaa 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheReplicatedSynchronousCommitTest.java @@ -129,7 +129,7 @@ public class GridCacheReplicatedSynchronousCommitTest extends GridCommonAbstract GridCache<Integer, String> cache1 = ignite1.cache(null); GridCache<Integer, String> cache3 = ignite3.cache(null); - IgniteFuture<?> fut = multithreadedAsync( + InternalFuture<?> 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/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java index 3402297..c6c8d97 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java @@ -129,7 +129,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<InternalFuture<?>> putFuts = new ArrayList<>(); for (int j = 0; j < PUT_THREADS; j++) { final int key = j; @@ -166,7 +166,7 @@ public class GridCacheStopSelfTest extends GridCommonAbstractTest { stopGrid(0); - for (IgniteFuture<?> fut : putFuts) { + for (InternalFuture<?> fut : putFuts) { try { fut.get(); } http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapPreloadSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapPreloadSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapPreloadSelfTest.java index af82c3e..3c43a02 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapPreloadSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheSwapPreloadSelfTest.java @@ -141,7 +141,7 @@ public class GridCacheSwapPreloadSelfTest extends GridCommonAbstractTest { /** @throws Exception If failed. */ private void checkSwapMultithreaded() throws Exception { final AtomicBoolean done = new AtomicBoolean(); - IgniteFuture<?> fut = null; + InternalFuture<?> fut = null; try { startGrid(0); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTtlManagerLoadTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTtlManagerLoadTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTtlManagerLoadTest.java index 371ad91..00c786a 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTtlManagerLoadTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTtlManagerLoadTest.java @@ -34,7 +34,7 @@ public class GridCacheTtlManagerLoadTest extends GridCacheTtlManagerSelfTest { try { final AtomicBoolean stop = new AtomicBoolean(); - IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() { + InternalFuture<?> fut = multithreadedAsync(new Callable<Object>() { @Override public Object call() throws Exception { GridCache<Object,Object> cache = g.cache(null); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiThreadedAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiThreadedAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiThreadedAbstractTest.java index 1e2811f..f179055 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiThreadedAbstractTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiThreadedAbstractTest.java @@ -10,7 +10,6 @@ package org.gridgain.grid.kernal.processors.cache; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.testframework.*; import org.jetbrains.annotations.*; @@ -210,7 +209,7 @@ public abstract class GridCacheTxMultiThreadedAbstractTest extends GridCacheTxAb cache.put(key, 0L); - List<IgniteFuture<Collection<Long>>> futs = new ArrayList<>(THREADS); + List<InternalFuture<Collection<Long>>> futs = new ArrayList<>(THREADS); for (int i = 0; i < THREADS; i++) { futs.add(GridTestUtils.runAsync(new Callable<Collection<Long>>() { @@ -243,7 +242,7 @@ public abstract class GridCacheTxMultiThreadedAbstractTest extends GridCacheTxAb List<Collection<Long>> cols = new ArrayList<>(THREADS); - for (IgniteFuture<Collection<Long>> fut : futs) { + for (InternalFuture<Collection<Long>> fut : futs) { Collection<Long> col = fut.get(); assertEquals(ITERATIONS, col.size()); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheVariableTopologySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheVariableTopologySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheVariableTopologySelfTest.java index 33a145f..0a41157 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheVariableTopologySelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheVariableTopologySelfTest.java @@ -107,7 +107,7 @@ public class GridCacheVariableTopologySelfTest extends GridCommonAbstractTest { final AtomicBoolean done = new AtomicBoolean(); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CAX() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CAX() { /** */ private int cnt; http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java index 3cc93bd..2303cc3 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractSelfTest.java @@ -91,7 +91,7 @@ public abstract class GridCacheWriteBehindStoreAbstractSelfTest extends GridComm final AtomicInteger operations = new AtomicInteger(); - IgniteFuture<?> fut = multithreadedAsync(new Runnable() { + InternalFuture<?> 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/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java index 0c91822..a5cb28a 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreAbstractTest.java @@ -256,7 +256,7 @@ public abstract class GridCacheWriteBehindStoreAbstractTest extends GridCommonAb final GridCache<Integer, String> cache = cache(); - IgniteFuture<?> fut = multithreadedAsync(new Runnable() { + InternalFuture<?> 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/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStorePartitionedMultiNodeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStorePartitionedMultiNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStorePartitionedMultiNodeSelfTest.java index 274a8ff..366a467 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStorePartitionedMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStorePartitionedMultiNodeSelfTest.java @@ -163,9 +163,9 @@ public class GridCacheWriteBehindStorePartitionedMultiNodeSelfTest extends GridC } /** - * @throws GridInterruptedException If sleep was interrupted. + * @throws InternalInterruptedException If sleep was interrupted. */ - private void checkWrites() throws GridInterruptedException { + private void checkWrites() throws InternalInterruptedException { U.sleep(WRITE_BEHIND_FLUSH_FREQ * 2); Collection<Integer> allKeys = new ArrayList<>(100); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreSelfTest.java index 0e84d80..564300c 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheWriteBehindStoreSelfTest.java @@ -138,7 +138,7 @@ public class GridCacheWriteBehindStoreSelfTest extends GridCacheWriteBehindStore final AtomicInteger actualPutCnt = new AtomicInteger(); - IgniteFuture<?> fut = multithreadedAsync(new Runnable() { + InternalFuture<?> fut = multithreadedAsync(new Runnable() { @SuppressWarnings({"NullableProblems"}) @Override public void run() { try { @@ -198,7 +198,7 @@ public class GridCacheWriteBehindStoreSelfTest extends GridCacheWriteBehindStore try { final AtomicBoolean running = new AtomicBoolean(true); - IgniteFuture<?> fut = multithreadedAsync(new Runnable() { + InternalFuture<?> fut = multithreadedAsync(new Runnable() { @SuppressWarnings({"NullableProblems"}) @Override public void run() { try { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java index 5c82786..50755c6 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java @@ -114,7 +114,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr try { GridCacheAtomicLong s = cache().dataStructures().atomicLong(STRUCTURE_NAME, 1, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -164,7 +164,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr try { GridCacheAtomicLong s = cache().dataStructures().atomicLong(STRUCTURE_NAME, 1, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -243,7 +243,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr try { GridCacheAtomicReference<Integer> s = cache().dataStructures().atomicReference(STRUCTURE_NAME, 1, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -292,7 +292,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr try { GridCacheAtomicReference<Integer> s = cache().dataStructures().atomicReference(STRUCTURE_NAME, 1, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -378,7 +378,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr try { GridCacheAtomicStamped<Integer, Integer> s = cache().dataStructures().atomicStamped(STRUCTURE_NAME, 1, 1, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -441,7 +441,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr try { GridCacheAtomicStamped<Integer, Integer> s = cache().dataStructures().atomicStamped(STRUCTURE_NAME, 1, 1, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -536,7 +536,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr GridCacheCountDownLatch s = cache().dataStructures().countDownLatch(STRUCTURE_NAME, Integer.MAX_VALUE, false, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -591,7 +591,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr GridCacheCountDownLatch s = cache().dataStructures() .countDownLatch(STRUCTURE_NAME, Integer.MAX_VALUE, false, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -676,7 +676,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr s.put(1); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -726,7 +726,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr s.put(1); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { @@ -801,7 +801,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr try { GridCacheAtomicSequence s = cache().dataStructures().atomicSequence(STRUCTURE_NAME, 1, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { String name = UUID.randomUUID().toString(); @@ -852,7 +852,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr final AtomicInteger idx = new AtomicInteger(gridCount()); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { int id = idx.getAndIncrement(); @@ -903,7 +903,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr try { GridCacheAtomicSequence s = cache().dataStructures().atomicSequence(STRUCTURE_NAME, 1, true); - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() { @Override public void apply() { try { for (int i = 0; i < TOP_CHANGE_CNT; i++) { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java index 7145c51..60df229 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java @@ -159,7 +159,7 @@ public abstract class GridCacheAbstractQueueFailoverDataConsistencySelfTest exte final AtomicBoolean stop = new AtomicBoolean(); - IgniteFuture<?> fut = startNodeKiller(stop, new AtomicInteger(), killIdxs); + InternalFuture<?> fut = startNodeKiller(stop, new AtomicInteger(), killIdxs); final int ITEMS = (atomicityMode() == ATOMIC) ? 10_000 : 3000; @@ -265,7 +265,7 @@ public abstract class GridCacheAbstractQueueFailoverDataConsistencySelfTest exte final AtomicInteger stopCnt = new AtomicInteger(); - IgniteFuture<?> fut = startNodeKiller(stop, stopCnt, killIdxs); + InternalFuture<?> fut = startNodeKiller(stop, stopCnt, killIdxs); int err = 0; @@ -318,7 +318,7 @@ public abstract class GridCacheAbstractQueueFailoverDataConsistencySelfTest exte * @param killIdxs Indexes of nodes to kill. * @return Future completing when thread finishes. */ - private IgniteFuture<?> startNodeKiller(final AtomicBoolean stop, final AtomicInteger killCnt, + private InternalFuture<?> startNodeKiller(final AtomicBoolean stop, final AtomicInteger killCnt, final List<Integer> killIdxs) { return GridTestUtils.runAsync(new Callable<Void>() { @Override public Void call() throws Exception { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java index 166d107..f95c6b3 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java @@ -144,7 +144,7 @@ public class GridCacheCountDownLatchSelfTest extends GridCommonAbstractTest impl assert latch.count() == 2; - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync( + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { GridCacheCountDownLatch latch = grid(0).cache("local").dataStructures() @@ -211,7 +211,7 @@ public class GridCacheCountDownLatchSelfTest extends GridCommonAbstractTest impl @Nullable @Override public Object call() throws Exception { // Test latch in multiple threads on each node. - IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync( + InternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync( new Callable<Object>() { @Nullable @Override public Object call() throws Exception { GridCacheCountDownLatch latch = ignite.cache(cacheName).dataStructures() @@ -238,7 +238,7 @@ public class GridCacheCountDownLatchSelfTest extends GridCommonAbstractTest impl } }); - IgniteFuture<Object> fut = comp.future(); + InternalFuture<Object> fut = comp.future(); Thread.sleep(3000); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java index e66ac39..5812b8e 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java @@ -200,7 +200,7 @@ public class GridCacheQueueCleanupSelfTest extends GridCacheAbstractSelfTest { * @param queueName Queue name. * @return Future completing when thread finishes. */ - private IgniteFuture<?> startAddPollThread(final Ignite ignite, final AtomicBoolean stop, final String queueName) { + private InternalFuture<?> startAddPollThread(final Ignite ignite, final AtomicBoolean stop, final String queueName) { return GridTestUtils.runAsync(new Callable<Void>() { @Override public Void call() throws Exception { GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures().queue(queueName, 0, false, true); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java index 04b4f98..6907318 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java @@ -82,9 +82,9 @@ public abstract class GridCacheQueueJoinedNodeSelfAbstractTest extends GridCommo comp.run(putJob); - IgniteFuture<?> fut = comp.future(); + InternalFuture<?> fut = comp.future(); - Collection<IgniteFuture<?>> futs = new ArrayList<>(GRID_CNT - 1); + Collection<InternalFuture<?>> futs = new ArrayList<>(GRID_CNT - 1); Collection<TakeJob> jobs = new ArrayList<>(GRID_CNT - 1); @@ -127,7 +127,7 @@ public abstract class GridCacheQueueJoinedNodeSelfAbstractTest extends GridCommo info(">>> Joined node polled " + polled); - for (IgniteFuture<?> f : futs) + for (InternalFuture<?> f : futs) f.cancel(); putJob.stop(true); @@ -237,18 +237,18 @@ public abstract class GridCacheQueueJoinedNodeSelfAbstractTest extends GridCommo /** * Awaits for a given count of items to be taken. * - * @throws GridInterruptedException If interrupted. + * @throws InternalInterruptedException If interrupted. */ - private void awaitItems() throws GridInterruptedException { + private void awaitItems() throws InternalInterruptedException { U.await(takeLatch); } /** * Awaits for a given count of items to be taken. * - * @throws GridInterruptedException If interrupted. + * @throws InternalInterruptedException If interrupted. */ - private void awaitDone() throws GridInterruptedException { + private void awaitDone() throws InternalInterruptedException { U.await(doneLatch); } @@ -273,7 +273,7 @@ public abstract class GridCacheQueueJoinedNodeSelfAbstractTest extends GridCommo } } catch (IgniteException e) { - if (e.getCause() instanceof GridInterruptedException || e.getCause() instanceof InterruptedException) + if (e.getCause() instanceof InternalInterruptedException || e.getCause() instanceof InterruptedException) log.info("Cancelling job due to interruption: " + e.getMessage()); else fail("Unexpected error: " + e); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java index acca39c..66f24d9 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java @@ -14,7 +14,6 @@ import org.apache.ignite.cluster.*; import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; import org.apache.ignite.resources.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.cache.datastructures.*; import org.apache.ignite.spi.discovery.tcp.*; @@ -170,7 +169,7 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon final Ignite g = startGrid(GRID_CNT + 1); - IgniteFuture<Object> fut1 = GridTestUtils.runAsync(new Callable<Object>() { + InternalFuture<Object> fut1 = GridTestUtils.runAsync(new Callable<Object>() { @Override public Object call() throws Exception { info(">>> Executing put callable [node=" + g.cluster().localNode().id() + ", thread=" + Thread.currentThread().getName() + ", aff=" + @@ -201,7 +200,7 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon final Ignite g1 = startGrid(GRID_CNT + 2); - IgniteFuture<Object> fut2 = GridTestUtils.runAsync(new Callable<Object>() { + InternalFuture<Object> fut2 = GridTestUtils.runAsync(new Callable<Object>() { @SuppressWarnings("BusyWait") @Override public Object call() throws Exception { try { @@ -355,7 +354,7 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon info("Queue name: " + queueName + ", collocated: " + collocated); try { - Collection<IgniteFuture> futs = new ArrayList<>(); + Collection<InternalFuture> futs = new ArrayList<>(); final int THREADS_PER_NODE = 3; final int ITEMS_PER_THREAD = 1000; @@ -377,7 +376,7 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon }, THREADS_PER_NODE, "testPutMultiNode")); } - for (IgniteFuture fut : futs) + for (InternalFuture fut : futs) fut.get(); GridCache cache = grid(0).cache(null); @@ -420,8 +419,8 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon info("Queue name: " + queueName + ", collocated: " + collocated); try { - Collection<IgniteFuture> putFuts = new ArrayList<>(); - Collection<IgniteFuture> pollFuts = new ArrayList<>(); + Collection<InternalFuture> putFuts = new ArrayList<>(); + Collection<InternalFuture> pollFuts = new ArrayList<>(); final int PUT_THREADS_PER_NODE = 3; final int POLL_THREADS_PER_NODE = 2; @@ -472,12 +471,12 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon } } - for (IgniteFuture fut : putFuts) + for (InternalFuture fut : putFuts) fut.get(); stopPoll.set(true); - for (IgniteFuture fut : pollFuts) + for (InternalFuture fut : pollFuts) fut.get(); GridCache cache = grid(0).cache(null); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java index 9dcd07c..2c8329e 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java @@ -660,7 +660,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf assertNotNull(set0); - Collection<IgniteFuture> futs = new ArrayList<>(); + Collection<InternalFuture> futs = new ArrayList<>(); final int THREADS_PER_NODE = 5; final int KEY_RANGE = 10_000; @@ -715,7 +715,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf }, THREADS_PER_NODE, "testSetMultithreaded")); } - for (IgniteFuture fut : futs) + for (InternalFuture fut : futs) fut.get(); } @@ -767,7 +767,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf final AtomicInteger val = new AtomicInteger(10_000); - IgniteFuture<?> fut; + InternalFuture<?> fut; try { fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java index 6052220..db1cb5f 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java @@ -99,7 +99,7 @@ public class GridCacheSetFailoverAbstractSelfTest extends GridCacheAbstractSelfT AtomicBoolean stop = new AtomicBoolean(); - IgniteFuture<?> killFut = startNodeKiller(stop); + InternalFuture<?> killFut = startNodeKiller(stop); long stopTime = System.currentTimeMillis() + TEST_DURATION; @@ -205,7 +205,7 @@ public class GridCacheSetFailoverAbstractSelfTest extends GridCacheAbstractSelfT * @param stop Stop flag. * @return Future completing when thread finishes. */ - private IgniteFuture<?> startNodeKiller(final AtomicBoolean stop) { + private InternalFuture<?> startNodeKiller(final AtomicBoolean stop) { return GridTestUtils.runAsync(new Callable<Void>() { @Override public Void call() throws Exception { ThreadLocalRandom rnd = ThreadLocalRandom.current(); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java index ab255fd..e6fab4b 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java @@ -78,7 +78,7 @@ public class GridCachePartitionedQueueCreateMultiNodeSelfTest extends GridCommon public void testQueueCreation() throws Exception { final AtomicInteger idx = new AtomicInteger(); - IgniteFuture<?> fut = multithreadedAsync( + InternalFuture<?> fut = multithreadedAsync( new Callable<Object>() { @Override public Object call() throws Exception { Ignite ignite = startGrid(idx.getAndIncrement()); @@ -130,7 +130,7 @@ public class GridCachePartitionedQueueCreateMultiNodeSelfTest extends GridCommon final CountDownLatch latch = new CountDownLatch(threadCnt); - IgniteFuture<?> fut = multithreadedAsync( + InternalFuture<?> fut = multithreadedAsync( new Callable<Object>() { @Override public Object call() throws Exception { Ignite ignite = startGrid(idx.getAndIncrement()); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java index 05f65df..abfae02 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java @@ -13,7 +13,6 @@ import org.apache.ignite.*; import org.apache.ignite.cluster.*; import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.cache.affinity.*; import org.gridgain.grid.cache.datastructures.*; @@ -93,7 +92,7 @@ public class GridCachePartitionedQueueEntryMoveSelfTest extends GridCommonAbstra final CountDownLatch latch1 = new CountDownLatch(1); //final CountDownLatch latch2 = new CountDownLatch(1); - IgniteFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() { + InternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() { @Override public Void call() throws IgniteCheckedException { Ignite ignite = grid(0); @@ -133,7 +132,7 @@ public class GridCachePartitionedQueueEntryMoveSelfTest extends GridCommonAbstra //latch2.countDown(); - IgniteFuture<?> fut2 = GridTestUtils.runAsync(new Callable<Void>() { + InternalFuture<?> fut2 = GridTestUtils.runAsync(new Callable<Void>() { @Override public Void call() throws IgniteCheckedException { Ignite ignite = grid(GRID_CNT); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java index cabe4db..e5a83ba 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java @@ -14,7 +14,6 @@ import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; import org.apache.ignite.marshaller.optimized.*; import org.apache.ignite.resources.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; @@ -113,7 +112,7 @@ public abstract class GridCacheAbstractJobExecutionTest extends GridCommonAbstra Ignite ignite = grid(0); - Collection<IgniteFuture<?>> futs = new LinkedList<>(); + Collection<InternalFuture<?>> futs = new LinkedList<>(); IgniteCompute comp = ignite.compute().enableAsync(); @@ -150,7 +149,7 @@ public abstract class GridCacheAbstractJobExecutionTest extends GridCommonAbstra futs.add(comp.future()); } - for (IgniteFuture<?> fut : futs) + for (InternalFuture<?> fut : futs) fut.get(); // Wait for completion. for (int i = 0; i < GRID_CNT; i++) { http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java index 74525c1..bf6c2cb 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java @@ -14,7 +14,6 @@ import org.apache.ignite.cluster.*; import org.apache.ignite.configuration.*; import org.apache.ignite.lang.*; import org.apache.ignite.spi.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.gridgain.grid.cache.affinity.*; import org.gridgain.grid.kernal.*; @@ -114,7 +113,7 @@ public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest { commSpi.skipNearRequest = true; - IgniteFuture<Object> fut = cache.putAsync(key, 1); + InternalFuture<Object> fut = cache.putAsync(key, 1); Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients"); @@ -148,7 +147,7 @@ public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest { commSpi.skipNearResponse = true; - IgniteFuture<Object> fut = cache.putAsync(key, 1); + InternalFuture<Object> fut = cache.putAsync(key, 1); Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients"); @@ -182,7 +181,7 @@ public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest { commSpi.skipDhtRequest = true; - IgniteFuture<Object> fut = cache.putAsync(key, 1); + InternalFuture<Object> fut = cache.putAsync(key, 1); Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients"); @@ -217,7 +216,7 @@ public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest { commSpi.skipDhtResponse = true; - IgniteFuture<Object> fut = cache.putAsync(key, 1); + InternalFuture<Object> fut = cache.putAsync(key, 1); Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients"); http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/90948e67/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java ---------------------------------------------------------------------- diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java index 53c6ff0..d326c1b 100644 --- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java +++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java @@ -13,7 +13,6 @@ import org.apache.ignite.*; import org.apache.ignite.configuration.*; import org.apache.ignite.events.*; import org.apache.ignite.lang.*; -import org.gridgain.grid.*; import org.gridgain.grid.cache.*; import org.apache.ignite.spi.discovery.tcp.*; import org.apache.ignite.spi.discovery.tcp.ipfinder.*; @@ -174,7 +173,7 @@ public abstract class GridCacheBasicOpAbstractTest extends GridCommonAbstractTes ignite2.events().localListen(lsnr, EVT_CACHE_OBJECT_PUT, EVT_CACHE_OBJECT_REMOVED); ignite3.events().localListen(lsnr, EVT_CACHE_OBJECT_PUT, EVT_CACHE_OBJECT_REMOVED); - IgniteFuture<String> f1 = cache1.getAsync("async1"); + InternalFuture<String> f1 = cache1.getAsync("async1"); assert f1.get() == null; @@ -191,8 +190,8 @@ public abstract class GridCacheBasicOpAbstractTest extends GridCommonAbstractTes assert latch.await(5, SECONDS); - IgniteFuture<String> f2 = cache2.getAsync("async1"); - IgniteFuture<String> f3 = cache3.getAsync("async1"); + InternalFuture<String> f2 = cache2.getAsync("async1"); + InternalFuture<String> f3 = cache3.getAsync("async1"); String v2 = f2.get(); String v3 = f3.get();