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:28:37 UTC

[01/50] [abbrv] incubator-ignite git commit: Applied fix.

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-16 b7e57279e -> d0444b79c


Applied fix.


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

Branch: refs/heads/ignite-16
Commit: 8e8a0c3ffa15442018b9dc696f12210ee36b9898
Parents: 9996140
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Jan 28 17:38:15 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Jan 28 17:38:15 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheContext.java      |  43 +++++
 .../processors/cache/GridCacheEntryEx.java      |   8 +
 .../processors/cache/GridCacheMapEntry.java     | 116 +++++++++---
 .../cache/GridCacheUpdateAtomicResult.java      |   6 +-
 .../processors/cache/GridDrResolveResult.java   |  63 -------
 .../GridDistributedTxRemoteAdapter.java         |  47 +++--
 .../dht/atomic/GridDhtAtomicCache.java          |  10 +-
 .../processors/cache/dr/GridCacheDrManager.java |  57 ++----
 .../cache/dr/os/GridOsCacheDrManager.java       |  35 +---
 .../cache/transactions/IgniteTxAdapter.java     |  45 +++++
 .../transactions/IgniteTxLocalAdapter.java      |  52 +++--
 ...ridCacheVersionAbstractConflictResolver.java |  56 ++++++
 .../GridCacheVersionConflictContext.java        |  73 +++++++
 .../GridCacheVersionConflictContextImpl.java    | 188 +++++++++++++++++++
 .../GridCacheVersionConflictResolver.java       |  59 ++++++
 .../version/GridCacheVersionedEntryEx.java      |   2 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   5 +
 17 files changed, 666 insertions(+), 199 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 571a7a4..3fb5329 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -190,6 +190,9 @@ public class GridCacheContext<K, V> implements Externalizable {
     /** Cache weak query iterator holder. */
     private CacheWeakQueryIteratorsHolder<Map.Entry<K, V>> itHolder;
 
+    /** Conflict resolver. */
+    private GridCacheVersionAbstractConflictResolver conflictRslvr;
+
     /**
      * Empty constructor required for {@link Externalizable}.
      */
@@ -306,6 +309,14 @@ public class GridCacheContext<K, V> implements Externalizable {
             expiryPlc = null;
 
         itHolder = new CacheWeakQueryIteratorsHolder(log);
+
+        // Conflict resolver is determined in two stages:
+        // 1. If DR receiver hub is enabled, then pick it from DR manager.
+        // 2. Otherwise instantiate default resolver in case local store is configured.
+        conflictRslvr = drMgr.conflictResolver();
+
+        if (conflictRslvr == null && storeMgr.isLocalStore())
+            conflictRslvr = new GridCacheVersionConflictResolver();
     }
 
     /**
@@ -1546,6 +1557,38 @@ public class GridCacheContext<K, V> implements Externalizable {
     }
 
     /**
+     * Check whether conflict resolution is required.
+     *
+     * @param oldVer Old version.
+     * @param newVer New version.
+     * @return {@code True} in case DR is required.
+     */
+    public boolean conflictNeedResolve(GridCacheVersion oldVer, GridCacheVersion newVer) {
+        return conflictRslvr != null;
+    }
+
+    /**
+     * Resolve DR conflict.
+     *
+     * @param oldEntry Old entry.
+     * @param newEntry New entry.
+     * @param atomicVerComparator Whether to use atomic version comparator.
+     * @return Conflict resolution result.
+     * @throws IgniteCheckedException In case of exception.
+     */
+    public GridCacheVersionConflictContextImpl<K, V> conflictResolve(GridCacheVersionedEntryEx<K, V> oldEntry,
+        GridCacheVersionedEntryEx<K, V> newEntry, boolean atomicVerComparator) throws IgniteCheckedException {
+        assert conflictRslvr != null : "Should not reach this place.";
+
+        GridCacheVersionConflictContextImpl<K, V> ctx = conflictRslvr.resolve(oldEntry, newEntry, atomicVerComparator);
+
+        if (ctx.isManualResolve())
+            drMgr.onReceiveCacheConflictResolved(ctx.isUseNew(), ctx.isUseOld(), ctx.isMerge());
+
+        return ctx;
+    }
+
+    /**
      * @return Data center ID.
      */
     public byte dataCenterId() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
index 6748d6e..8eeacc5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryEx.java
@@ -671,6 +671,14 @@ public interface GridCacheEntryEx<K, V> {
         throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
+     * Create versioned entry for this cache entry.
+     *
+     * @return Versioned entry.
+     * @throws IgniteCheckedException In case of error.
+     */
+    public GridCacheVersionedEntryEx<K, V> versionedEntry() throws IgniteCheckedException;
+
+    /**
      * Sets new value if passed in version matches the current version
      * (used for read-through only).
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
index 0b34457..3c4e9d6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMapEntry.java
@@ -68,7 +68,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
     private static final byte IS_UNSWAPPED_MASK = 0x02;
 
     /** */
-    private static final Comparator<GridCacheVersion> ATOMIC_VER_COMPARATOR = new GridCacheAtomicVersionComparator();
+    public static final Comparator<GridCacheVersion> ATOMIC_VER_COMPARATOR = new GridCacheAtomicVersionComparator();
 
     /**
      * NOTE
@@ -1658,7 +1658,7 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
 
         GridCacheVersion enqueueVer = null;
 
-        GridDrResolveResult<V> drRes = null;
+        GridCacheVersionConflictContextImpl<K, V> drRes = null;
 
         EntryProcessorResult<?> invokeRes = null;
 
@@ -1675,49 +1675,113 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
             if (isNew())
                 unswap(true, retval);
 
+            boolean newTtlResolved = false;
+
+            boolean drNeedResolve = false;
+
             Object transformClo = null;
 
             if (drResolve) {
-                drRes = cctx.dr().resolveAtomic(this,
-                    op,
-                    writeObj,
-                    valBytes,
-                    expiryPlc,
-                    drTtl,
-                    drExpireTime,
-                    drVer);
+                GridCacheVersion oldDrVer = version().drVersion();
+
+                drNeedResolve = cctx.conflictNeedResolve(oldDrVer, drVer);
+
+                if (drNeedResolve) {
+                    // Get old value.
+                    V oldVal = rawGetOrUnmarshalUnlocked(true);
+
+                    if (writeObj == null && valBytes != null)
+                        writeObj = cctx.marshaller().unmarshal(valBytes, cctx.deploy().globalLoader());
+
+                    if (op == GridCacheOperation.TRANSFORM) {
+                        transformClo = writeObj;
+
+                        writeObj = ((IgniteClosure<V, V>)writeObj).apply(oldVal);
+                    }
+
+                    K k = key();
+
+                    if (drTtl >= 0L) {
+                        // DR TTL is set explicitly
+                        assert drExpireTime >= 0L;
+
+                        newTtl = drTtl;
+                        newExpireTime = drExpireTime;
+                    }
+                    else {
+                        long ttl = expiryPlc != null ? (isNew() ? expiryPlc.forCreate() : expiryPlc.forUpdate()) : -1L;
+
+                        newTtl = ttl < 0 ? ttlExtras() : ttl;
+                        newExpireTime = CU.toExpireTime(newTtl);
+                    }
+
+                    newTtlResolved = true;
+
+                    GridCacheVersionedEntryEx<K, V> oldEntry = versionedEntry();
+                    GridCacheVersionedEntryEx<K, V> newEntry =
+                        new GridCachePlainVersionedEntry<>(k, (V)writeObj, newTtl, newExpireTime, drVer);
+
+                    drRes = cctx.conflictResolve(oldEntry, newEntry, verCheck);
+
+                    assert drRes != null;
 
-                if (drRes != null) {
                     if (drRes.isUseOld()) {
+                        // Handle special case with atomic comparator.
+                        if (!isNew() &&                                            // Not initial value,
+                            verCheck &&                                            // and atomic version check,
+                            oldDrVer.dataCenterId() == drVer.dataCenterId() &&     // and data centers are equal,
+                            ATOMIC_VER_COMPARATOR.compare(oldDrVer, drVer) == 0 && // and both versions are equal,
+                            cctx.writeThrough() &&                                 // and store is enabled,
+                            primary)                                               // and we are primary.
+                        {
+                            V val = rawGetOrUnmarshalUnlocked(false);
+
+                            if (val == null) {
+                                assert deletedUnlocked();
+
+                                cctx.store().removeFromStore(null, key());
+                            }
+                            else
+                                cctx.store().putToStore(null, key(), val, ver);
+                        }
+
                         old = retval ? rawGetOrUnmarshalUnlocked(false) : val;
 
                         return new GridCacheUpdateAtomicResult<>(false,
                             old,
                             null,
                             invokeRes,
-                            -1L,
+                            0L,
                             -1L,
                             null,
                             null,
                             false);
                     }
+                    else if (drRes.isUseNew())
+                        op = writeObj != null ? GridCacheOperation.UPDATE : GridCacheOperation.DELETE;
+                    else {
+                        assert drRes.isMerge();
 
-                    newTtl = drRes.newTtl();
-
-                    newExpireTime = drRes.newExpireTime();
-
-                    newDrExpireTime = drRes.newDrExpireTime();
+                        writeObj = drRes.mergeValue();
+                        valBytes = null;
 
-                    op = drRes.operation();
+                        drVer = null; // Update will be considered as local.
 
-                    writeObj = drRes.value();
+                        op = writeObj != null ? GridCacheOperation.UPDATE : GridCacheOperation.DELETE;
+                    }
 
-                    valBytes = drRes.valueBytes();
+                    newTtl = drRes.ttl();
+                    newExpireTime = drRes.expireTime();
 
-                    if (drRes.isMerge())
-                        drVer = null; // Update will be considered as local.
+                    // Explicit DR expire time will be passed to remote node only in that case.
+                    if (!drRes.explicitTtl() && !drRes.isMerge()) {
+                        if (drRes.isUseNew() && newEntry.dataCenterId() != cctx.dataCenterId() ||
+                            drRes.isUseOld() && oldEntry.dataCenterId() != cctx.dataCenterId())
+                            newDrExpireTime = drRes.expireTime();
+                    }
                 }
                 else
+                    // Nullify DR version on this update, so that we will use regular version during next updates.
                     drVer = null;
             }
 
@@ -3095,6 +3159,14 @@ public abstract class GridCacheMapEntry<K, V> implements GridCacheEntryEx<K, V>
     }
 
     /** {@inheritDoc} */
+    @Override public synchronized GridCacheVersionedEntryEx<K, V> versionedEntry() throws IgniteCheckedException {
+        boolean isNew = isStartVersion();
+
+        return new GridCachePlainVersionedEntry<>(key, isNew ? unswap(true, true) : rawGetOrUnmarshalUnlocked(false),
+            ttlExtras(), expireTimeExtras(), ver.drVersion(), isNew);
+    }
+
+    /** {@inheritDoc} */
     @Override public synchronized boolean versionedValue(V val, GridCacheVersion curVer, GridCacheVersion newVer)
         throws IgniteCheckedException, GridCacheEntryRemovedException {
         checkObsolete();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java
index 0b7d776..34be479 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUpdateAtomicResult.java
@@ -51,7 +51,7 @@ public class GridCacheUpdateAtomicResult<K, V> {
 
     /** DR resolution result. */
     @GridToStringInclude
-    private final GridDrResolveResult<V> drRes;
+    private final GridCacheVersionConflictContextImpl<K, V> drRes;
 
     /** Whether update should be propagated to DHT node. */
     private final boolean sndToDht;
@@ -79,7 +79,7 @@ public class GridCacheUpdateAtomicResult<K, V> {
         long newTtl,
         long drExpireTime,
         @Nullable GridCacheVersion rmvVer,
-        @Nullable GridDrResolveResult<V> drRes,
+        @Nullable GridCacheVersionConflictContextImpl<K, V> drRes,
         boolean sndToDht) {
         this.success = success;
         this.oldVal = oldVal;
@@ -144,7 +144,7 @@ public class GridCacheUpdateAtomicResult<K, V> {
     /**
      * @return DR conflict resolution context.
      */
-    @Nullable public GridDrResolveResult<V> drResolveResult() {
+    @Nullable public GridCacheVersionConflictContextImpl<K, V> drResolveResult() {
         return drRes;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDrResolveResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDrResolveResult.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDrResolveResult.java
deleted file mode 100644
index faf71ca..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridDrResolveResult.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.cache;
-
-/**
- *
- */
-public interface GridDrResolveResult<V> {
-    /**
-     * @return TTL.
-     */
-    public long newTtl();
-
-    /**
-     * @return Expire time.
-     */
-    public long newExpireTime();
-
-    /**
-     * @return DR expire time.
-     */
-    public long newDrExpireTime();
-
-    /**
-     * @return {@code True} in case merge is to be performed.
-     */
-    public boolean isMerge();
-
-    /**
-     * @return {@code True} in case old value should be used.
-     */
-    public boolean isUseOld();
-
-    /**
-     * @return Cache operation.
-     */
-    public GridCacheOperation operation();
-
-    /**
-     * @return Value.
-     */
-    public V value();
-
-    /**
-     * @return Value bytes.
-     */
-    public byte[] valueBytes();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index cea3a0d..f7376cf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -509,24 +509,37 @@ public class GridDistributedTxRemoteAdapter<K, V> extends IgniteTxAdapter<K, V>
                                         if (explicitVer == null)
                                             explicitVer = writeVersion(); // Force write version to be used.
 
-                                        GridDrResolveResult<V> drRes = cacheCtx.dr().resolveTx(cached,
-                                            txEntry,
-                                            explicitVer,
-                                            op,
-                                            val,
-                                            valBytes,
-                                            txEntry.ttl(),
-                                            txEntry.drExpireTime());
-
-                                        if (drRes != null) {
-                                            op = drRes.operation();
-                                            val = drRes.value();
-                                            valBytes = drRes.valueBytes();
-
-                                            if (drRes.isMerge())
+                                        boolean drNeedResolve =
+                                            cacheCtx.conflictNeedResolve(cached.version(), explicitVer);
+
+                                        if (drNeedResolve) {
+                                            IgniteBiTuple<GridCacheOperation, GridCacheVersionConflictContextImpl<K, V>>
+                                                drRes = conflictResolve(op, txEntry.key(), val, valBytes,
+                                                txEntry.ttl(), txEntry.drExpireTime(), explicitVer, cached);
+
+                                            assert drRes != null;
+
+                                            GridCacheVersionConflictContextImpl<K, V> drCtx = drRes.get2();
+
+                                            if (drCtx.isUseOld())
+                                                op = NOOP;
+                                            else if (drCtx.isUseNew()) {
+                                                txEntry.ttl(drCtx.ttl());
+
+                                                if (drCtx.newEntry().dataCenterId() != cacheCtx.dataCenterId())
+                                                    txEntry.drExpireTime(drCtx.expireTime());
+                                                else
+                                                    txEntry.drExpireTime(-1L);
+                                            }
+                                            else if (drCtx.isMerge()) {
+                                                op = drRes.get1();
+                                                val = drCtx.mergeValue();
+                                                valBytes = null;
                                                 explicitVer = writeVersion();
-                                            else if (op == NOOP)
-                                                txEntry.ttl(-1L);
+
+                                                txEntry.ttl(drCtx.ttl());
+                                                txEntry.drExpireTime(-1L);
+                                            }
                                         }
                                         else
                                             // Nullify explicit version so that innerSet/innerRemove will work as usual.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index c993397..c35743f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -1083,7 +1083,11 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                         if (plc != null)
                             expiry = new UpdateExpiryPolicy(plc);
 
-                        if (writeThrough() && keys.size() > 1 && !ctx.dr().receiveEnabled()) {
+                        if (keys.size() > 1 &&                             // Several keys ...
+                            writeThrough() &&                              // and store is enabled ...
+                            !ctx.store().isLocalStore() &&                 // and this is not local store ...
+                            !ctx.dr().receiveEnabled()  // and no DR.
+                        ) {
                             // This method can only be used when there are no replicated entries in the batch.
                             UpdateBatchResult<K, V> updRes = updateWithBatch(node,
                                 hasNear,
@@ -1681,7 +1685,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
                 if (dhtFut != null) {
                     if (updRes.sendToDht()) { // Send to backups even in case of remove-remove scenarios.
-                        GridDrResolveResult<V> ctx = updRes.drResolveResult();
+                        GridCacheVersionConflictContextImpl<K, V> ctx = updRes.drResolveResult();
 
                         long ttl = updRes.newTtl();
                         long expireTime = updRes.drExpireTime();
@@ -1727,7 +1731,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 if (hasNear) {
                     if (primary && updRes.sendToDht()) {
                         if (!ctx.affinity().belongs(node, entry.partition(), topVer)) {
-                            GridDrResolveResult<V> ctx = updRes.drResolveResult();
+                            GridCacheVersionConflictContextImpl<K, V> ctx = updRes.drResolveResult();
 
                             long ttl = updRes.newTtl();
                             long expireTime = updRes.drExpireTime();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
index ff83198..d0a0c26 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/GridCacheDrManager.java
@@ -34,53 +34,9 @@ public interface GridCacheDrManager<K, V> extends GridCacheManager<K, V> {
     public byte dataCenterId();
 
     /**
-     * Handles DR for atomic cache.
-     *
-     * @param e Cache entry.
-     * @param op Operation.
-     * @param writeObj New value.
-     * @param valBytes New value byte.
-     * @param ttl TTL.
-     * @param drTtl DR TTL.
-     * @param drExpireTime DR expire time
-     * @param drVer DR version.
-     * @return DR result.
-     * @throws IgniteCheckedException If update failed.
-     * @throws GridCacheEntryRemovedException If entry is obsolete.
-     */
-    public GridDrResolveResult<V> resolveAtomic(GridCacheEntryEx<K, V> e,
-         GridCacheOperation op,
-         @Nullable Object writeObj,
-         @Nullable byte[] valBytes,
-         @Nullable IgniteCacheExpiryPolicy expiryPlc,
-         long drTtl,
-         long drExpireTime,
-         @Nullable GridCacheVersion drVer) throws IgniteCheckedException, GridCacheEntryRemovedException;
-
-    /**
-     * Handles DR for transactional cache.
-     *
-     * @param e Cache entry.
-     * @param txEntry Transaction entry.
-     * @param newVer Version.
-     * @param op Operation.
-     * @param newVal New value.
-     * @param newValBytes New value bytes.
-     * @param newTtl TTL.
-     * @param newDrExpireTime DR expire time
-     * @return DR result.
-     * @throws IgniteCheckedException If update failed.
-     * @throws GridCacheEntryRemovedException If entry is obsolete.
+     * @return Cache version conflict resolver.
      */
-    public GridDrResolveResult<V> resolveTx(
-        GridCacheEntryEx<K, V> e,
-        IgniteTxEntry<K, V> txEntry,
-        GridCacheVersion newVer,
-        GridCacheOperation op,
-        V newVal,
-        byte[] newValBytes,
-        long newTtl,
-        long newDrExpireTime) throws IgniteCheckedException, GridCacheEntryRemovedException;
+    public GridCacheVersionAbstractConflictResolver conflictResolver();
 
     /**
      * Performs replication.
@@ -138,6 +94,15 @@ public interface GridCacheDrManager<K, V> extends GridCacheManager<K, V> {
     public void onReceiveCacheEntriesReceived(int entriesCnt);
 
     /**
+     * Callback for manual conflict resolution.
+     *
+     * @param useNew Use new.
+     * @param useOld Use old.
+     * @param merge Merge.
+     */
+    public void onReceiveCacheConflictResolved(boolean useNew, boolean useOld, boolean merge);
+
+    /**
      * Resets metrics for current cache.
      */
     public void resetMetrics();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/os/GridOsCacheDrManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/os/GridOsCacheDrManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/os/GridOsCacheDrManager.java
index 20b8804..49f617b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/os/GridOsCacheDrManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/dr/os/GridOsCacheDrManager.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache.dr.os;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.dr.*;
-import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.processors.dr.*;
 import org.jetbrains.annotations.*;
@@ -65,6 +64,11 @@ public class GridOsCacheDrManager<K, V> implements GridCacheDrManager<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public GridCacheVersionAbstractConflictResolver conflictResolver() {
+        return null;
+    }
+
+    /** {@inheritDoc} */
     @Override public void replicate(K key,
         @Nullable byte[] keyBytes,
         @Nullable V val,
@@ -77,30 +81,6 @@ public class GridOsCacheDrManager<K, V> implements GridCacheDrManager<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public GridDrResolveResult<V> resolveAtomic(GridCacheEntryEx<K, V> e,
-        GridCacheOperation op,
-        @Nullable Object writeObj,
-        @Nullable byte[] valBytes,
-        @Nullable IgniteCacheExpiryPolicy expiryPlc,
-        long drTtl,
-        long drExpireTime,
-        @Nullable GridCacheVersion drVer) throws IgniteCheckedException, GridCacheEntryRemovedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridDrResolveResult<V> resolveTx(GridCacheEntryEx<K, V> e,
-        IgniteTxEntry<K, V> txEntry,
-        GridCacheVersion newVer,
-        GridCacheOperation op,
-        V newVal,
-        byte[] newValBytes,
-        long newTtl,
-        long newDrExpireTime) throws IgniteCheckedException, GridCacheEntryRemovedException {
-        return null;
-    }
-
-    /** {@inheritDoc} */
     @Override public void beforeExchange(long topVer, boolean left) throws IgniteCheckedException {
         // No-op.
     }
@@ -116,6 +96,11 @@ public class GridOsCacheDrManager<K, V> implements GridCacheDrManager<K, V> {
     }
 
     /** {@inheritDoc} */
+    @Override public void onReceiveCacheConflictResolved(boolean useNew, boolean useOld, boolean merge) {
+        // No-op.
+    }
+
+    /** {@inheritDoc} */
     @Override public void resetMetrics() {
         // No-op.
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index 47cd12b..e079a5c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -1252,6 +1252,51 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
     }
 
     /**
+     * Resolve DR conflict.
+     *
+     * @param op Initially proposed operation.
+     * @param key Key.
+     * @param newVal New value.
+     * @param newValBytes New value bytes.
+     * @param newTtl New TTL.
+     * @param newDrExpireTime New explicit DR expire time.
+     * @param newVer New version.
+     * @param old Old entry.
+     * @return Tuple with adjusted operation type and conflict context.
+     * @throws org.apache.ignite.IgniteCheckedException In case of eny exception.
+     * @throws GridCacheEntryRemovedException If entry got removed.
+     */
+    protected IgniteBiTuple<GridCacheOperation, GridCacheVersionConflictContextImpl<K, V>> conflictResolve(
+        GridCacheOperation op, K key, V newVal, byte[] newValBytes, long newTtl, long newDrExpireTime,
+        GridCacheVersion newVer, GridCacheEntryEx<K, V> old)
+        throws IgniteCheckedException, GridCacheEntryRemovedException {
+        // Construct old entry info.
+        GridCacheVersionedEntryEx<K, V> oldEntry = old.versionedEntry();
+
+        // Construct new entry info.
+        if (newVal == null && newValBytes != null)
+            newVal = cctx.marshaller().unmarshal(newValBytes, cctx.deploy().globalLoader());
+
+        long newExpireTime = newDrExpireTime >= 0L ? newDrExpireTime : CU.toExpireTime(newTtl);
+
+        GridCacheVersionedEntryEx<K, V> newEntry =
+            new GridCachePlainVersionedEntry<K, V>(key, newVal, newTtl, newExpireTime, newVer);
+
+        GridCacheVersionConflictContextImpl<K, V> ctx = old.context().conflictResolve(oldEntry, newEntry, false);
+
+        if (ctx.isMerge()) {
+            V resVal = ctx.mergeValue();
+
+            if ((op == CREATE || op == UPDATE) && resVal == null)
+                op = DELETE;
+            else if (op == DELETE && resVal != null)
+                op = old.isNewLocked() ? CREATE : UPDATE;
+        }
+
+        return F.t(op, ctx);
+    }
+
+    /**
      * @param e Transaction entry.
      * @param primaryOnly Flag to include backups into check or not.
      * @return {@code True} if entry is locally mapped as a primary or back up node.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 46ab74f..d9c49d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -699,31 +699,45 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                                         }
                                     }
 
-                                    GridDrResolveResult<V> drRes = cacheCtx.dr().resolveTx(cached,
-                                        txEntry,
-                                        explicitVer,
-                                        op,
-                                        val,
-                                        valBytes,
-                                        txEntry.ttl(),
-                                        txEntry.drExpireTime());
-
-                                    if (drRes != null) {
-                                        op = drRes.operation();
-                                        val = drRes.value();
-                                        valBytes = drRes.valueBytes();
-
-                                        if (drRes.isMerge())
+                                    boolean drNeedResolve = cacheCtx.conflictNeedResolve(cached.version(), explicitVer);
+
+                                    if (drNeedResolve) {
+                                        IgniteBiTuple<GridCacheOperation, GridCacheVersionConflictContextImpl<K, V>>
+                                            drRes = conflictResolve(op, txEntry.key(), val, valBytes, txEntry.ttl(),
+                                                txEntry.drExpireTime(), explicitVer, cached);
+
+                                        assert drRes != null;
+
+                                        GridCacheVersionConflictContextImpl<K, V> conflictCtx = drRes.get2();
+
+                                        if (conflictCtx.isUseOld())
+                                            op = NOOP;
+                                        else if (conflictCtx.isUseNew()) {
+                                            txEntry.ttl(conflictCtx.ttl());
+
+                                            if (conflictCtx.newEntry().dataCenterId() != cctx.dataCenterId())
+                                                txEntry.drExpireTime(conflictCtx.expireTime());
+                                            else
+                                                txEntry.drExpireTime(-1L);
+                                        }
+                                        else {
+                                            assert conflictCtx.isMerge();
+
+                                            op = drRes.get1();
+                                            val = conflictCtx.mergeValue();
+                                            valBytes = null;
                                             explicitVer = writeVersion();
-                                        else if (op == NOOP)
-                                            txEntry.ttl(-1L);
+
+                                            txEntry.ttl(conflictCtx.ttl());
+                                            txEntry.drExpireTime(-1L);
+                                        }
                                     }
                                     else
                                         // Nullify explicit version so that innerSet/innerRemove will work as usual.
                                         explicitVer = null;
 
-                                    if (sndTransformedVals || (drRes != null)) {
-                                        assert sndTransformedVals && cacheCtx.isReplicated() || (drRes != null);
+                                    if (sndTransformedVals || drNeedResolve) {
+                                        assert sndTransformedVals && cacheCtx.isReplicated() || drNeedResolve;
 
                                         txEntry.value(val, true, false);
                                         txEntry.valueBytes(valBytes);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionAbstractConflictResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionAbstractConflictResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionAbstractConflictResolver.java
new file mode 100644
index 0000000..a91bd4d
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionAbstractConflictResolver.java
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.*;
+
+/**
+ * Cache version conflict resolver.
+ */
+public abstract class GridCacheVersionAbstractConflictResolver {
+    /**
+     * Resolve the conflict.
+     *
+     * @param oldEntry Old entry.
+     * @param newEntry New entry.
+     * @param atomicVerComparator Whether to use atomic version comparator.
+     * @return Conflict resolution context.
+     * @throws IgniteCheckedException If failed.
+     */
+    public <K, V> GridCacheVersionConflictContextImpl<K, V> resolve(GridCacheVersionedEntryEx<K, V> oldEntry,
+        GridCacheVersionedEntryEx<K, V> newEntry, boolean atomicVerComparator) throws IgniteCheckedException {
+        GridCacheVersionConflictContextImpl<K, V> ctx = new GridCacheVersionConflictContextImpl<>(oldEntry, newEntry);
+
+        resolve0(ctx, oldEntry, newEntry, atomicVerComparator);
+
+        return ctx;
+    }
+
+    /**
+     * Internal conflict resolution routine.
+     *
+     * @param ctx Context.
+     * @param oldEntry Old entry.
+     * @param newEntry New entry.
+     * @param atomicVerComparator Whether to use atomic version comparator.
+     * @throws IgniteCheckedException If failed.
+     */
+    protected abstract <K, V> void resolve0(GridCacheVersionConflictContextImpl<K, V> ctx,
+        GridCacheVersionedEntryEx<K, V> oldEntry, GridCacheVersionedEntryEx<K, V> newEntry,
+        boolean atomicVerComparator) throws IgniteCheckedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContext.java
new file mode 100644
index 0000000..72c323b
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContext.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Cache version conflict context.
+ */
+public interface GridCacheVersionConflictContext<K, V> {
+    /**
+     * Gets old (existing) cache entry.
+     *
+     * @return Old (existing) cache entry.
+     */
+    public GridCacheVersionedEntry<K, V> oldEntry();
+
+    /**
+     * Gets new cache entry.
+     *
+     * @return New cache entry.
+     */
+    public GridCacheVersionedEntry<K, V> newEntry();
+
+    /**
+     * Force cache to ignore new entry and leave old (existing) entry unchanged.
+     */
+    public void useOld();
+
+    /**
+     * Force cache to apply new entry overwriting old (existing) entry.
+     * <p>
+     * Note that updates from remote data centers always have explicit TTL , while local data center
+     * updates will only have explicit TTL in case {@link CacheEntry#timeToLive(long)} was called
+     * before update. In the latter case new entry will pick TTL of the old (existing) entry, even
+     * if it was set through update from remote data center. it means that depending on concurrent
+     * update timings new update might pick unexpected TTL. For example, consider that three updates
+     * of the same key are performed: local update with explicit TTL (1) followed by another local
+     * update without explicit TTL (2) and one remote update (3). In this case you might expect that
+     * update (2) will pick TTL set during update (1). However, in case update (3) occurrs between (1)
+     * and (2) and it overwrites (1) during conflict resolution, then update (2) will pick TTL of
+     * update (3). To have predictable TTL in such cases you should either always set it explicitly
+     * through {@code GridCacheEntry.timeToLive(long)} or use {@link #merge(Object, long)}.
+     */
+    public void useNew();
+
+    /**
+     * Force cache to use neither old, nor new, but some other value passed as argument. In this case old
+     * value will be replaced with merge value and update will be considered as local.
+     * <p>
+     * Also in case of merge you have to specify new TTL explicitly. For unlimited TTL use {@code 0}.
+     *
+     * @param mergeVal Merge value or {@code null} to force remove.
+     * @param ttl Time to live in milliseconds.
+     */
+    public void merge(@Nullable V mergeVal, long ttl);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContextImpl.java
new file mode 100644
index 0000000..228a224
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictContextImpl.java
@@ -0,0 +1,188 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+/**
+ * Conflict context implementation.
+ */
+public class GridCacheVersionConflictContextImpl<K, V> implements GridCacheVersionConflictContext<K, V> {
+    /** Old entry. */
+    @GridToStringInclude
+    private final GridCacheVersionedEntry<K, V> oldEntry;
+
+    /** New entry. */
+    @GridToStringInclude
+    private final GridCacheVersionedEntry<K, V> newEntry;
+
+    /** Current state. */
+    private State state;
+
+    /** Current merge value. */
+    @GridToStringExclude
+    private V mergeVal;
+
+    /** TTL. */
+    private long ttl;
+
+    /** Explicit TTL flag. */
+    private boolean explicitTtl;
+
+    /** Manual resolve flag. */
+    private boolean manualResolve;
+
+    /**
+     * Constructor.
+     *
+     * @param oldEntry Old entry.
+     * @param newEntry New entry.
+     */
+    public GridCacheVersionConflictContextImpl(GridCacheVersionedEntry<K, V> oldEntry,
+        GridCacheVersionedEntry<K, V> newEntry) {
+        assert oldEntry != null && newEntry != null;
+        assert oldEntry.ttl() >= 0 && newEntry.ttl() >= 0;
+
+        this.oldEntry = oldEntry;
+        this.newEntry = newEntry;
+
+        // Set initial state.
+        useNew();
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCacheVersionedEntry<K, V> oldEntry() {
+        return oldEntry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public GridCacheVersionedEntry<K, V> newEntry() {
+        return newEntry;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void useOld() {
+        state = State.USE_OLD;
+    }
+
+    /** {@inheritDoc} */
+    @Override public void useNew() {
+        state = State.USE_NEW;
+
+        if (!explicitTtl)
+            ttl = newEntry.ttl();
+    }
+
+    /** {@inheritDoc} */
+    @Override public void merge(@Nullable V mergeVal, long ttl) {
+        state = State.MERGE;
+
+        this.mergeVal = mergeVal;
+        this.ttl = ttl;
+
+        explicitTtl = true;
+    }
+
+    /**
+     * @return {@code True} in case old value should be used.
+     */
+    public boolean isUseOld() {
+        return state == State.USE_OLD;
+    }
+
+    /**
+     * @return {@code True} in case new value should be used.
+     */
+    public boolean isUseNew() {
+        return state == State.USE_NEW;
+    }
+
+    /**
+     * @return {@code True} in case merge is to be performed.
+     */
+    public boolean isMerge() {
+        return state == State.MERGE;
+    }
+
+    /**
+     * Set manual resolve class.
+     */
+    public void manualResolve() {
+        this.manualResolve = true;
+    }
+
+    /**
+     * @return Manual resolve flag.
+     */
+    public boolean isManualResolve() {
+        return manualResolve;
+    }
+
+    /**
+     * @return Value to merge (if any).
+     */
+    @Nullable public V mergeValue() {
+        return mergeVal;
+    }
+
+    /**
+     * @return TTL.
+     */
+    public long ttl() {
+        return ttl;
+    }
+
+    /**
+     * @return Expire time.
+     */
+    public long expireTime() {
+        return explicitTtl ? CU.toExpireTime(ttl) : isUseNew() ? newEntry.expireTime() :
+            isUseOld() ? oldEntry.expireTime() : 0L;
+    }
+
+    /**
+     * @return Explicit TTL flag.
+     */
+    public boolean explicitTtl() {
+        return explicitTtl;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String toString() {
+        return state == State.MERGE ?
+            S.toString(GridCacheVersionConflictContextImpl.class, this, "mergeValue", mergeVal) :
+            S.toString(GridCacheVersionConflictContextImpl.class, this);
+    }
+
+    /**
+     * State.
+     */
+    private enum State {
+        /** Use old. */
+        USE_OLD,
+
+        /** Use new. */
+        USE_NEW,
+
+        /** Merge. */
+        MERGE
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictResolver.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictResolver.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictResolver.java
new file mode 100644
index 0000000..e327fb9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionConflictResolver.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal.processors.cache.version;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.processors.cache.*;
+
+/**
+ * Default conflict resolver.
+ */
+public class GridCacheVersionConflictResolver extends GridCacheVersionAbstractConflictResolver {
+    /** {@inheritDoc} */
+    @Override protected <K, V> void resolve0(GridCacheVersionConflictContextImpl<K, V> ctx,
+        GridCacheVersionedEntryEx<K, V> oldEntry, GridCacheVersionedEntryEx<K, V> newEntry,
+        boolean atomicVerComparator) throws IgniteCheckedException {
+        if (newEntry.dataCenterId() != oldEntry.dataCenterId())
+            ctx.useNew();
+        else {
+            if (oldEntry.isStartVersion())
+                ctx.useNew();
+            else {
+                if (atomicVerComparator) {
+                    // Handle special case when version check using ATOMIC cache comparator is required.
+                    if (GridCacheMapEntry.ATOMIC_VER_COMPARATOR.compare(oldEntry.version(), newEntry.version()) >= 0)
+                        ctx.useOld();
+                    else
+                        ctx.useNew();
+                }
+                else {
+                    long topVerDiff = newEntry.topologyVersion() - oldEntry.topologyVersion();
+
+                    if (topVerDiff > 0)
+                        ctx.useNew();
+                    else if (topVerDiff < 0)
+                        ctx.useOld();
+                    else if (newEntry.order() > oldEntry.order())
+                        ctx.useNew();
+                    else
+                        ctx.useOld();
+                }
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionedEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionedEntryEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionedEntryEx.java
index b8f7c19..5351966 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionedEntryEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/version/GridCacheVersionedEntryEx.java
@@ -25,7 +25,7 @@ import org.apache.ignite.cache.*;
 public interface GridCacheVersionedEntryEx<K, V> extends GridCacheVersionedEntry<K, V>, GridCacheVersionable {
     /**
      *
-     * @return
+     * @return {@code True} if entry is new.
      */
     public boolean isStartVersion();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8e8a0c3f/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
index 60df4e4..d1b6ce6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTestEntryEx.java
@@ -631,6 +631,11 @@ public class GridCacheTestEntryEx<K, V> extends GridMetadataAwareAdapter impleme
     }
 
     /** @inheritDoc */
+    @Override public GridCacheVersionedEntryEx<K, V> versionedEntry() throws IgniteCheckedException {
+        return null;
+    }
+
+    /** @inheritDoc */
     @Override public boolean versionedValue(V val, GridCacheVersion curVer, GridCacheVersion newVer) {
         assert false; return false;
     }


[13/50] [abbrv] incubator-ignite git commit: # ignite-91 Fixed test

Posted by vo...@apache.org.
# ignite-91 Fixed test


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

Branch: refs/heads/ignite-16
Commit: 5673e3f979b58f70a18a795be3eb884cc961a93c
Parents: e45b390
Author: sboikov <se...@inria.fr>
Authored: Thu Jan 29 06:04:36 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Thu Jan 29 06:04:36 2015 +0300

----------------------------------------------------------------------
 .../ignite/client/integration/ClientAbstractSelfTest.java    | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/5673e3f9/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractSelfTest.java
index bdb97ba..cee7608 100644
--- a/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractSelfTest.java
@@ -610,7 +610,13 @@ public abstract class ClientAbstractSelfTest extends GridCommonAbstractTest {
             }
         }, proto == GridClientProtocol.TCP ? GridClientException.class : IllegalArgumentException.class, null);
 
-        dfltData.getAll(Collections.singleton(null));
+        assertThrows(log, new Callable<Object>() {
+            @Override public Object call() throws Exception {
+                dfltData.getAll(Collections.singleton(null));
+
+                return null;
+            }
+        }, proto == GridClientProtocol.TCP ? GridClientException.class : IllegalArgumentException.class, null);
     }
 
     /**


[42/50] [abbrv] incubator-ignite git commit: GridNamedInstance -> IgniteNamedInstance GridgainEx -> IgnitionEx

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 35b34f0..0e60832 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -4430,7 +4430,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         try {
             IgniteBiTuple<String, String> t = stash.get();
 
-            return GridGainEx.gridx(t.get1()).cachex(t.get2());
+            return IgnitionEx.gridx(t.get1()).cachex(t.get2());
         }
         catch (IllegalStateException e) {
             throw U.withCause(new InvalidObjectException(e.getMessage()), e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 0f5b06d..1f624f8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -1888,7 +1888,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         try {
             IgniteBiTuple<String, String> t = stash.get();
 
-            IgniteKernal grid = GridGainEx.gridx(t.get1());
+            IgniteKernal grid = IgnitionEx.gridx(t.get1());
 
             GridCacheAdapter<K, V> cache = grid.internalCache(t.get2());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java
index f5db642..369837e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridSpringResourceContext.java
@@ -23,7 +23,7 @@ import org.apache.ignite.internal.*;
 /**
  * Interface was introduced to avoid compile-time dependency on spring framework. Spring resource context
  * provides optional spring resource injectors, it can be passed to factory method
- * starting GridGain {@link GridGainEx#start(GridSpringResourceContext)}.
+ * starting GridGain {@link IgnitionEx#start(GridSpringResourceContext)}.
  */
 public interface GridSpringResourceContext {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsEventsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsEventsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsEventsAbstractSelfTest.java
index a72b151..c96ad41 100644
--- a/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsEventsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsEventsAbstractSelfTest.java
@@ -111,7 +111,7 @@ public abstract class GridGgfsEventsAbstractSelfTest extends GridCommonAbstractT
      * @throws Exception If failed.
      */
     protected IgniteConfiguration getConfiguration(String gridName, IgniteFsConfiguration ggfsCfg) throws Exception {
-        IgniteConfiguration cfg = GridGainEx.loadConfiguration("config/hadoop/default-config.xml").get1();
+        IgniteConfiguration cfg = IgnitionEx.loadConfiguration("config/hadoop/default-config.xml").get1();
 
         assert cfg != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCachePutRemoveLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCachePutRemoveLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCachePutRemoveLoadTest.java
index e11963d..6eca2c8 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCachePutRemoveLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCachePutRemoveLoadTest.java
@@ -96,7 +96,7 @@ public class GridCachePutRemoveLoadTest {
     protected void startNodes() throws Exception {
         for (int i = 0; i < args.nodes(); i++) {
             IgniteConfiguration cfg =
-                GridGainEx.loadConfiguration("modules/core/src/test/config/spring-cache-put-remove-load.xml").get1();
+                IgnitionEx.loadConfiguration("modules/core/src/test/config/spring-cache-put-remove-load.xml").get1();
 
             assert cfg != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index fca61d1..84883f9 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -653,7 +653,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      */
     protected Ignite startGrid(String gridName, GridSpringResourceContext ctx) throws Exception {
-        return GridGainEx.start(optimize(getConfiguration(gridName)), ctx);
+        return IgnitionEx.start(optimize(getConfiguration(gridName)), ctx);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java b/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java
index 74f9e1b..cbc54dc 100644
--- a/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java
+++ b/modules/spring/src/main/java/org/apache/ignite/IgniteSpring.java
@@ -48,7 +48,7 @@ public class IgniteSpring {
      *      also if default grid has already been started.
      */
     public static Ignite start(@Nullable ApplicationContext springCtx) throws IgniteCheckedException {
-        return GridGainEx.start(new GridSpringResourceContextImpl(springCtx));
+        return IgnitionEx.start(new GridSpringResourceContextImpl(springCtx));
     }
 
     /**
@@ -64,7 +64,7 @@ public class IgniteSpring {
      *      also if named grid has already been started.
      */
     public static Ignite start(IgniteConfiguration cfg, @Nullable ApplicationContext springCtx) throws IgniteCheckedException {
-        return GridGainEx.start(cfg, new GridSpringResourceContextImpl(springCtx));
+        return IgnitionEx.start(cfg, new GridSpringResourceContextImpl(springCtx));
     }
 
     /**
@@ -88,7 +88,7 @@ public class IgniteSpring {
      *      been started or Spring XML configuration file is invalid.
      */
     public static Ignite start(String springCfgPath, @Nullable ApplicationContext springCtx) throws IgniteCheckedException {
-        return GridGainEx.start(springCfgPath, null, new GridSpringResourceContextImpl(springCtx));
+        return IgnitionEx.start(springCfgPath, null, new GridSpringResourceContextImpl(springCtx));
     }
 
     /**
@@ -112,6 +112,6 @@ public class IgniteSpring {
      *      been started or Spring XML configuration file is invalid.
      */
     public static Ignite start(URL springCfgUrl, @Nullable ApplicationContext springCtx) throws IgniteCheckedException {
-        return GridGainEx.start(springCfgUrl, null, new GridSpringResourceContextImpl(springCtx));
+        return IgnitionEx.start(springCfgUrl, null, new GridSpringResourceContextImpl(springCtx));
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletContextListenerStartup.java
----------------------------------------------------------------------
diff --git a/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletContextListenerStartup.java b/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletContextListenerStartup.java
index 928958d..4759907 100644
--- a/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletContextListenerStartup.java
+++ b/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletContextListenerStartup.java
@@ -121,7 +121,7 @@ public class GridServletContextListenerStartup implements ServletContextListener
             IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> t;
 
             try {
-                t = GridGainEx.loadConfigurations(cfgUrl);
+                t = IgnitionEx.loadConfigurations(cfgUrl);
             }
             catch (IgniteCheckedException e) {
                 throw new IgniteException("Failed to load GridGain configuration.", e);
@@ -149,7 +149,7 @@ public class GridServletContextListenerStartup implements ServletContextListener
                         ignite = G.ignite(cfg.getGridName());
                     }
                     catch (IgniteIllegalStateException ignored) {
-                        ignite = GridGainEx.start(new IgniteConfiguration(cfg), rsrcCtx);
+                        ignite = IgnitionEx.start(new IgniteConfiguration(cfg), rsrcCtx);
                     }
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletStartup.java
----------------------------------------------------------------------
diff --git a/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletStartup.java b/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletStartup.java
index 8e806d3..0cc5701 100644
--- a/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletStartup.java
+++ b/modules/web/src/main/java/org/apache/ignite/startup/servlet/GridServletStartup.java
@@ -141,7 +141,7 @@ public class GridServletStartup extends HttpServlet {
 
         try {
             IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> t =
-                GridGainEx.loadConfigurations(cfgUrl);
+                IgnitionEx.loadConfigurations(cfgUrl);
 
             Collection<IgniteConfiguration> cfgs = t.get1();
 
@@ -153,7 +153,7 @@ public class GridServletStartup extends HttpServlet {
 
                 IgniteConfiguration adapter = new IgniteConfiguration(cfg);
 
-                Ignite ignite = GridGainEx.start(adapter, t.get2());
+                Ignite ignite = IgnitionEx.start(adapter, t.get2());
 
                 // Test if grid is not null - started properly.
                 if (ignite != null)


[35/50] [abbrv] incubator-ignite git commit: GridEx -> IgniteEx GridKernal -> IgniteKernal

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java
index 4bf8d42..3387ba6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java
@@ -87,7 +87,7 @@ public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest {
     /** {@inheritDoc} */
     @Override protected void afterTest() throws Exception {
         for (int i = 0; i < GRID_CNT; i++) {
-            final GridKernal grid = (GridKernal)grid(i);
+            final IgniteKernal grid = (IgniteKernal)grid(i);
 
             TestCommunicationSpi commSpi = (TestCommunicationSpi)grid.configuration().getCommunicationSpi();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheEventAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheEventAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheEventAbstractTest.java
index becab72..48e69aa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheEventAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheEventAbstractTest.java
@@ -106,7 +106,7 @@ public abstract class GridCacheEventAbstractTest extends GridCacheAbstractSelfTe
     private void waitForEvents(int gridIdx, IgniteBiTuple<Integer, Integer>... evtCnts) throws Exception {
         if (!F.isEmpty(evtCnts))
             try {
-                TestEventListener.waitForEventCount(((GridKernal)grid(0)).context(), evtCnts);
+                TestEventListener.waitForEventCount(((IgniteKernal)grid(0)).context(), evtCnts);
             }
             catch (IgniteCheckedException e) {
                 printEventCounters(gridIdx, evtCnts);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheExpiredEntriesPreloadAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheExpiredEntriesPreloadAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheExpiredEntriesPreloadAbstractSelfTest.java
index 39a5aba..45fcfd5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheExpiredEntriesPreloadAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheExpiredEntriesPreloadAbstractSelfTest.java
@@ -99,7 +99,7 @@ public abstract class GridCacheExpiredEntriesPreloadAbstractSelfTest extends Gri
         // Start another node.
         Ignite g1 = startGrid(1);
 
-        final GridCacheAdapter<String, Integer> cache1 = ((GridKernal)g1).context().cache().internalCache();
+        final GridCacheAdapter<String, Integer> cache1 = ((IgniteKernal)g1).context().cache().internalCache();
 
         cache1.preloader().syncFuture().get();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
index 8796d69..7681046 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeLockAbstractTest.java
@@ -110,7 +110,7 @@ public abstract class GridCacheMultiNodeLockAbstractTest extends GridCommonAbstr
             cache(i).clearAll();
 
             assertTrue(
-                "Cache isn't empty [i=" + i + ", entries=" + ((GridKernal)grid(i)).internalCache().entries() + "]",
+                "Cache isn't empty [i=" + i + ", entries=" + ((IgniteKernal)grid(i)).internalCache().entries() + "]",
                 cache(i).isEmpty());
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxConsistencyRestartAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxConsistencyRestartAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxConsistencyRestartAbstractSelfTest.java
index faf4591..01f7718 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxConsistencyRestartAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxConsistencyRestartAbstractSelfTest.java
@@ -143,7 +143,7 @@ public abstract class IgniteTxConsistencyRestartAbstractSelfTest extends GridCom
                 info("Running iteration: " + i);
 
             try {
-                GridKernal grid = (GridKernal)grid(idx);
+                IgniteKernal grid = (IgniteKernal)grid(idx);
 
                 GridCache<Integer, Integer> cache = grid.cache(null);
 
@@ -181,7 +181,7 @@ public abstract class IgniteTxConsistencyRestartAbstractSelfTest extends GridCom
             Integer val = null;
 
             for (int i = 0; i < GRID_CNT; i++) {
-                GridEx grid = grid(i);
+                IgniteEx grid = grid(i);
 
                 GridCache<Integer, Integer> cache = grid.cache(null);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
index 476b465..8860c04 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxOriginatingNodeFailureAbstractSelfTest.java
@@ -109,12 +109,12 @@ public abstract class IgniteTxOriginatingNodeFailureAbstractSelfTest extends Gri
     protected void testTxOriginatingNodeFails(Collection<Integer> keys, final boolean partial) throws Exception {
         assertFalse(keys.isEmpty());
 
-        final Collection<GridKernal> grids = new ArrayList<>();
+        final Collection<IgniteKernal> grids = new ArrayList<>();
 
         ClusterNode txNode = grid(originatingNode()).localNode();
 
         for (int i = 1; i < gridCount(); i++)
-            grids.add((GridKernal)grid(i));
+            grids.add((IgniteKernal)grid(i));
 
         final Map<Integer, String> map = new HashMap<>();
 
@@ -128,7 +128,7 @@ public abstract class IgniteTxOriginatingNodeFailureAbstractSelfTest extends Gri
 
         Map<Integer, Collection<ClusterNode>> nodeMap = new HashMap<>();
 
-        GridCacheAdapter<Integer, String> cache = ((GridKernal)grid(1)).internalCache();
+        GridCacheAdapter<Integer, String> cache = ((IgniteKernal)grid(1)).internalCache();
 
         info("Node being checked: " + grid(1).localNode().id());
 
@@ -143,7 +143,7 @@ public abstract class IgniteTxOriginatingNodeFailureAbstractSelfTest extends Gri
         }
 
         info("Starting tx [values=" + map + ", topVer=" +
-            ((GridKernal)grid(1)).context().discovery().topologyVersion() + ']');
+            ((IgniteKernal)grid(1)).context().discovery().topologyVersion() + ']');
 
         if (partial)
             ignoreMessages(grid(1).localNode().id(), ignoreMessageClass());
@@ -181,7 +181,7 @@ public abstract class IgniteTxOriginatingNodeFailureAbstractSelfTest extends Gri
 
         boolean txFinished = GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
-                for (GridKernal g : grids) {
+                for (IgniteKernal g : grids) {
                     GridCacheSharedContext<Object, Object> ctx = g.context().cache().context();
 
                     int txNum = ctx.tm().idMapSize();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
index 2240552..3605705 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
@@ -128,12 +128,12 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
     protected void testTxOriginatingNodeFails(Collection<Integer> keys, final boolean fullFailure) throws Exception {
         assertFalse(keys.isEmpty());
 
-        final Collection<GridKernal> grids = new ArrayList<>();
+        final Collection<IgniteKernal> grids = new ArrayList<>();
 
         ClusterNode txNode = grid(originatingNode()).localNode();
 
         for (int i = 1; i < gridCount(); i++)
-            grids.add((GridKernal)grid(i));
+            grids.add((IgniteKernal)grid(i));
 
         failingNodeId = grid(0).localNode().id();
 
@@ -149,7 +149,7 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
 
         Map<Integer, Collection<ClusterNode>> nodeMap = new HashMap<>();
 
-        GridCacheAdapter<Integer, String> cache = ((GridKernal)grid(1)).internalCache();
+        GridCacheAdapter<Integer, String> cache = ((IgniteKernal)grid(1)).internalCache();
 
         info("Node being checked: " + grid(1).localNode().id());
 
@@ -164,14 +164,14 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
         }
 
         info("Starting tx [values=" + map + ", topVer=" +
-            ((GridKernal)grid(1)).context().discovery().topologyVersion() + ']');
+            ((IgniteKernal)grid(1)).context().discovery().topologyVersion() + ']');
 
         if (fullFailure)
             ignoreMessages(ignoreMessageClasses(), allNodeIds());
         else
             ignoreMessages(ignoreMessageClasses(), F.asList(grid(1).localNode().id()));
 
-        final GridEx originatingNodeGrid = grid(originatingNode());
+        final IgniteEx originatingNodeGrid = grid(originatingNode());
 
         GridTestUtils.runAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
@@ -214,7 +214,7 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
 
         boolean txFinished = GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
-                for (GridKernal g : grids) {
+                for (IgniteKernal g : grids) {
                     GridCacheAdapter<?, ?> cache = g.internalCache();
 
                     IgniteTxManager txMgr = cache.isNear() ?
@@ -282,13 +282,13 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
         for (int i = 0; i < 20; i++)
             keys.add(i);
 
-        final Collection<GridKernal> grids = new ArrayList<>();
+        final Collection<IgniteKernal> grids = new ArrayList<>();
 
         ClusterNode primaryNode = grid(1).localNode();
 
         for (int i = 0; i < gridCount(); i++) {
             if (i != 1)
-                grids.add((GridKernal)grid(i));
+                grids.add((IgniteKernal)grid(i));
         }
 
         failingNodeId = primaryNode.id();
@@ -320,7 +320,7 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
         }
 
         info("Starting tx [values=" + map + ", topVer=" +
-            ((GridKernal)grid(1)).context().discovery().topologyVersion() + ']');
+            ((IgniteKernal)grid(1)).context().discovery().topologyVersion() + ']');
 
         assertNotNull(cache);
 
@@ -345,7 +345,7 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
 
         boolean txFinished = GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
-                for (GridKernal g : grids) {
+                for (IgniteKernal g : grids) {
                     GridCacheAdapter<?, ?> cache = g.internalCache();
 
                     IgniteTxManager txMgr = cache.isNear() ?

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java
index 4f297b8..e8aeaf7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicNearCacheSelfTest.java
@@ -693,7 +693,7 @@ public class GridCacheAtomicNearCacheSelfTest extends GridCommonAbstractTest {
     @SuppressWarnings("ConstantConditions")
     private void checkEntry(Ignite ignite, Integer key, @Nullable Integer val, boolean expectNear, UUID... expReaders)
         throws Exception {
-        GridCacheAdapter<Integer, Integer> near = ((GridKernal) ignite).internalCache();
+        GridCacheAdapter<Integer, Integer> near = ((IgniteKernal) ignite).internalCache();
 
         assertTrue(near.isNear());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
index 433bda5..32c2f7a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
@@ -320,7 +320,7 @@ public class GridCacheColocatedDebugTest extends GridCommonAbstractTest {
             Thread.sleep(1000);
             // Check that all transactions are committed.
             for (int i = 0; i < 3; i++) {
-                GridCacheAdapter<Object, Object> cache = ((GridKernal)grid(i)).internalCache();
+                GridCacheAdapter<Object, Object> cache = ((IgniteKernal)grid(i)).internalCache();
 
                 for (Integer key : keys) {
                     GridCacheEntryEx<Object, Object> entry = cache.peekEx(key);
@@ -770,7 +770,7 @@ public class GridCacheColocatedDebugTest extends GridCommonAbstractTest {
     private void checkStore(Ignite ignite, Map<Integer, String> map) throws Exception {
         String cacheName = ignite.configuration().getCacheConfiguration()[0].getName();
 
-        GridCacheContext ctx = ((GridKernal)grid()).context().cache().internalCache(cacheName).context();
+        GridCacheContext ctx = ((IgniteKernal)grid()).context().cache().internalCache(cacheName).context();
 
         CacheStore store = ctx.store().configuredStore();
 
@@ -786,7 +786,7 @@ public class GridCacheColocatedDebugTest extends GridCommonAbstractTest {
         for (int i = 0; i < cnt; i++) {
             String cacheName = grid(i).configuration().getCacheConfiguration()[0].getName();
 
-            GridCacheContext ctx = ((GridKernal)grid()).context().cache().internalCache(cacheName).context();
+            GridCacheContext ctx = ((IgniteKernal)grid()).context().cache().internalCache(cacheName).context();
 
             CacheStore store = ctx.store().configuredStore();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
index 1e0ae74..4182d8b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEntrySelfTest.java
@@ -131,7 +131,7 @@ public class GridCacheDhtEntrySelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"unchecked", "TypeMayBeWeakened"})
     private GridDhtCacheAdapter<Integer, String> dht(Ignite g) {
-        return ((GridNearCacheAdapter)((GridKernal)g).internalCache()).dht();
+        return ((GridNearCacheAdapter)((IgniteKernal)g).internalCache()).dht();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
index b3fa9ee..4e6e21d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
@@ -151,7 +151,7 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT
      */
     @SuppressWarnings({"unchecked"})
     private GridNearCacheAdapter<Integer, String> near(Ignite g) {
-        return (GridNearCacheAdapter)((GridKernal)g).internalCache();
+        return (GridNearCacheAdapter)((IgniteKernal)g).internalCache();
     }
 
     /**
@@ -160,7 +160,7 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT
      */
     @SuppressWarnings({"unchecked", "TypeMayBeWeakened"})
     private GridDhtCacheAdapter<Integer, String> dht(Ignite g) {
-        return ((GridNearCacheAdapter)((GridKernal)g).internalCache()).dht();
+        return ((GridNearCacheAdapter)((IgniteKernal)g).internalCache()).dht();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
index b518b96..223edcb 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
@@ -145,7 +145,7 @@ public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"unchecked"})
     private GridNearCacheAdapter<Integer, String> near(Ignite g) {
-        return (GridNearCacheAdapter)((GridKernal)g).internalCache();
+        return (GridNearCacheAdapter)((IgniteKernal)g).internalCache();
     }
 
     /**
@@ -154,7 +154,7 @@ public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"unchecked", "TypeMayBeWeakened"})
     private GridDhtCacheAdapter<Integer, String> dht(Ignite g) {
-        return ((GridNearCacheAdapter)((GridKernal)g).internalCache()).dht();
+        return ((GridNearCacheAdapter)((IgniteKernal)g).internalCache()).dht();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
index 21d4da6..c4b7dc0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtMappingSelfTest.java
@@ -90,7 +90,7 @@ public class GridCacheDhtMappingSelfTest extends GridCommonAbstractTest {
             Ignite g = grid(i);
 
             GridDhtCacheAdapter<Integer, Integer> dht = ((GridNearCacheAdapter<Integer, Integer>)
-                ((GridKernal)g).<Integer, Integer>internalCache()).dht();
+                ((IgniteKernal)g).<Integer, Integer>internalCache()).dht();
 
             if (dht.peek(kv) != null) {
                 info("Key found on node: " + g.cluster().localNode().id());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
index 32bfad3..ffe8552 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadDelayedSelfTest.java
@@ -369,7 +369,7 @@ public class GridCacheDhtPreloadDelayedSelfTest extends GridCommonAbstractTest {
      * @return Topology.
      */
     private GridDhtPartitionTopology<Integer, String> topology(Ignite g) {
-        return ((GridNearCacheAdapter<Integer, String>)((GridKernal)g).<Integer, String>internalCache()).dht().topology();
+        return ((GridNearCacheAdapter<Integer, String>)((IgniteKernal)g).<Integer, String>internalCache()).dht().topology();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
index e81c3ec..d59158e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
@@ -212,7 +212,7 @@ public class GridCacheDhtPreloadStartStopSelfTest extends GridCommonAbstractTest
             info(">>> Waiting for preload futures...");
 
             GridCachePartitionExchangeManager<Object, Object> exchMgr
-                = ((GridKernal)g1).context().cache().context().exchange();
+                = ((IgniteKernal)g1).context().cache().context().exchange();
 
             // Wait for exchanges to complete.
             for (IgniteInternalFuture<?> fut : exchMgr.exchangeFutures())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
index 7865567..6ff01f7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
@@ -134,13 +134,13 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
         try {
             startGridsMultiThreaded(2);
 
-            GridKernal[] nodes = new GridKernal[] {(GridKernal)grid(0), (GridKernal)grid(1)};
+            IgniteKernal[] nodes = new IgniteKernal[] {(IgniteKernal)grid(0), (IgniteKernal)grid(1)};
 
             Collection<IgniteInternalFuture> futs = new ArrayList<>();
 
             final CountDownLatch startLatch = new CountDownLatch(1);
 
-            for (final GridKernal node : nodes) {
+            for (final IgniteKernal node : nodes) {
                 List<Integer> parts = partitions(node, PARTITION_PRIMARY);
 
                 Map<Integer, Integer> keyMap = keysFor(node, parts);
@@ -217,11 +217,11 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
     private void checkTxNodeJoined(int nodeType) throws Exception {
         startGrids(3);
 
-        final GridKernal g0 = (GridKernal)grid(0);
-        final GridKernal g1 = (GridKernal)grid(1);
-        final GridKernal g2 = (GridKernal)grid(2);
+        final IgniteKernal g0 = (IgniteKernal)grid(0);
+        final IgniteKernal g1 = (IgniteKernal)grid(1);
+        final IgniteKernal g2 = (IgniteKernal)grid(2);
 
-        GridKernal[] nodes = new GridKernal[] {g0, g1, g2};
+        IgniteKernal[] nodes = new IgniteKernal[] {g0, g1, g2};
 
         try {
             info(">>> Started nodes [g0=" + g0.localNode().id() + ", g1=" + g1.localNode().id() + ", g2=" +
@@ -231,7 +231,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
 
             Collection<IgniteInternalFuture> futs = new ArrayList<>();
 
-            for (final GridKernal node : nodes) {
+            for (final IgniteKernal node : nodes) {
                 printDistribution(node);
 
                 // Get partitions that does not reside on g0.
@@ -366,12 +366,12 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
     private void checkTxNodeLeft(int nodeType) throws Exception {
         startGridsMultiThreaded(4);
 
-        final GridKernal g0 = (GridKernal)grid(0);
-        final GridKernal g1 = (GridKernal)grid(1);
-        final GridKernal g2 = (GridKernal)grid(2);
-        final GridKernal g3 = (GridKernal)grid(3);
+        final IgniteKernal g0 = (IgniteKernal)grid(0);
+        final IgniteKernal g1 = (IgniteKernal)grid(1);
+        final IgniteKernal g2 = (IgniteKernal)grid(2);
+        final IgniteKernal g3 = (IgniteKernal)grid(3);
 
-        GridKernal[] nodes = new GridKernal[] {g0, g1, g2};
+        IgniteKernal[] nodes = new IgniteKernal[] {g0, g1, g2};
 
         final CountDownLatch commitLatch = new CountDownLatch(1);
 
@@ -385,7 +385,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
 
             printDistribution(g3);
 
-            for (final GridKernal node : nodes) {
+            for (final IgniteKernal node : nodes) {
                 printDistribution(node);
 
                 // Get partitions that does not reside on g0.
@@ -488,7 +488,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
                 txFut.get(1000);
 
             for (int i = 0; i < 3; i++) {
-                CacheConsistentHashAffinityFunction affinity = (CacheConsistentHashAffinityFunction)((GridKernal)grid(i))
+                CacheConsistentHashAffinityFunction affinity = (CacheConsistentHashAffinityFunction)((IgniteKernal)grid(i))
                     .internalCache().context().config().getAffinity();
 
                 ConcurrentMap addedNodes = U.field(affinity, "addedNodes");
@@ -512,7 +512,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
      *
      * @param node Node to detect partitions for.
      */
-    private void printDistribution(GridKernal node) {
+    private void printDistribution(IgniteKernal node) {
         List<Integer> primary = partitions(node, PARTITION_PRIMARY);
         List<Integer> backup = partitions(node, PARTITION_BACKUP);
         List<Integer> reader = partitions(node, PARTITION_READER);
@@ -529,7 +529,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
      * @param parts Partitions to get keys for.
      * @return Map from partition to key.
      */
-    private Map<Integer, Integer> keysFor(GridKernal node, Iterable<Integer> parts) {
+    private Map<Integer, Integer> keysFor(IgniteKernal node, Iterable<Integer> parts) {
         GridCacheContext<Object, Object> ctx = node.internalCache().context();
 
         Map<Integer, Integer> res = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java
index 5ab4365..5aca193 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTxOriginatingNodeFailureSelfTest.java
@@ -62,7 +62,7 @@ public class GridCachePartitionedTxOriginatingNodeFailureSelfTest extends
      * @throws Exception If failed.
      */
     public void testTxFromPrimary() throws Exception {
-        GridCacheAdapter<Integer, String> cache = ((GridKernal)grid(originatingNode())).internalCache();
+        GridCacheAdapter<Integer, String> cache = ((IgniteKernal)grid(originatingNode())).internalCache();
 
         ClusterNode txNode = grid(originatingNode()).localNode();
 
@@ -85,7 +85,7 @@ public class GridCachePartitionedTxOriginatingNodeFailureSelfTest extends
      * @throws Exception If failed.
      */
     public void testTxFromBackup() throws Exception {
-        GridCacheAdapter<Integer, String> cache = ((GridKernal)grid(originatingNode())).internalCache();
+        GridCacheAdapter<Integer, String> cache = ((IgniteKernal)grid(originatingNode())).internalCache();
 
         ClusterNode txNode = grid(originatingNode()).localNode();
 
@@ -108,7 +108,7 @@ public class GridCachePartitionedTxOriginatingNodeFailureSelfTest extends
      * @throws Exception If failed.
      */
     public void testTxFromNotColocated() throws Exception {
-        GridCacheAdapter<Integer, String> cache = ((GridKernal)grid(originatingNode())).internalCache();
+        GridCacheAdapter<Integer, String> cache = ((IgniteKernal)grid(originatingNode())).internalCache();
 
         ClusterNode txNode = grid(originatingNode()).localNode();
 
@@ -131,7 +131,7 @@ public class GridCachePartitionedTxOriginatingNodeFailureSelfTest extends
      * @throws Exception If failed.
      */
     public void testTxAllNodes() throws Exception {
-        GridCacheAdapter<Integer, String> cache = ((GridKernal)grid(originatingNode())).internalCache();
+        GridCacheAdapter<Integer, String> cache = ((IgniteKernal)grid(originatingNode())).internalCache();
 
         List<ClusterNode> allNodes = new ArrayList<>(GRID_CNT);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
index 12b2aba..3d0e6c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
@@ -241,7 +241,7 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
                 for (int i = 0; i < gridCnt; i++) {
                     ClusterNode locNode = grid(i).localNode();
 
-                    GridCacheAdapter<Object, Object> c = ((GridKernal)grid(i)).internalCache();
+                    GridCacheAdapter<Object, Object> c = ((IgniteKernal)grid(i)).internalCache();
 
                     GridCacheEntryEx<Object, Object> entry = c.peekEx(k);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPreloadSelfTest.java
index 0a2c83e..457567f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicPreloadSelfTest.java
@@ -149,7 +149,7 @@ public class GridCacheAtomicPreloadSelfTest extends GridCommonAbstractTest {
      */
     private void checkTransactions() {
         for (int i = 0; i < 3; i++) {
-            IgniteTxManager<Object, Object> tm = ((GridKernal)grid(i)).context().cache().context().tm();
+            IgniteTxManager<Object, Object> tm = ((IgniteKernal)grid(i)).context().cache().context().tm();
 
             assertEquals("Uncommitted transactions found on node [idx=" + i + ", mapSize=" + tm.idMapSize() + ']',
                 0, tm.idMapSize());
@@ -162,7 +162,7 @@ public class GridCacheAtomicPreloadSelfTest extends GridCommonAbstractTest {
      */
     private void checkValues(int key, int val) {
         for (int i = 0; i < 3; i++) {
-            GridEx grid = grid(i);
+            IgniteEx grid = grid(i);
 
             ClusterNode node = grid.localNode();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
index aff7efa..7aa9482 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearMultiNodeSelfTest.java
@@ -191,7 +191,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"unchecked", "TypeMayBeWeakened"})
     private GridDhtCacheAdapter<Integer, String> dht(Ignite g) {
-        return ((GridNearCacheAdapter)((GridKernal)g).internalCache()).dht();
+        return ((GridNearCacheAdapter)((IgniteKernal)g).internalCache()).dht();
     }
 
     /**
@@ -200,7 +200,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings({"unchecked", "TypeMayBeWeakened"})
     private GridNearCacheAdapter<Integer, String> near(Ignite g) {
-        return (GridNearCacheAdapter)((GridKernal)g).internalCache();
+        return (GridNearCacheAdapter)((IgniteKernal)g).internalCache();
     }
 
     /**
@@ -219,7 +219,7 @@ public class GridCacheNearMultiNodeSelfTest extends GridCommonAbstractTest {
         Map<UUID, T2<Set<Integer>, Set<Integer>>> map = new HashMap<>();
 
         for (int i = 0; i < GRID_CNT; i++) {
-            GridEx grid = grid(i);
+            IgniteEx grid = grid(i);
 
             map.put(grid.cluster().localNode().id(), new T2<Set<Integer>, Set<Integer>>(new HashSet<Integer>(),
                 new HashSet<Integer>()));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
index fde4e58..f658a15 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearReadersSelfTest.java
@@ -215,8 +215,8 @@ public class GridCacheNearReadersSelfTest extends GridCommonAbstractTest {
 
         awaitPartitionMapExchange();
 
-        ((GridKernal)g1).internalCache(null).preloader().request(F.asList(1, 2), 2).get();
-        ((GridKernal)g2).internalCache(null).preloader().request(F.asList(1, 2), 2).get();
+        ((IgniteKernal)g1).internalCache(null).preloader().request(F.asList(1, 2), 2).get();
+        ((IgniteKernal)g2).internalCache(null).preloader().request(F.asList(1, 2), 2).get();
 
         GridCache<Integer, String> cache1 = g1.cache(null);
         GridCache<Integer, String> cache2 = g2.cache(null);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
index 5e14bc7..fe06d27 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheNearTxMultiNodeSelfTest.java
@@ -163,7 +163,7 @@ public class GridCacheNearTxMultiNodeSelfTest extends GridCommonAbstractTest {
             ignites = F.asList(otherIgnite, newIgnite);
 
             for (Ignite g : ignites) {
-                GridNearCacheAdapter near = ((GridKernal)g).internalCache().context().near();
+                GridNearCacheAdapter near = ((IgniteKernal)g).internalCache().context().near();
                 GridDhtCacheAdapter dht = near.dht();
 
                 checkTm(g, near.context().tm());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
index 631d0de..f3ce6fa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedFullApiSelfTest.java
@@ -64,7 +64,7 @@ public class GridCachePartitionedFullApiSelfTest extends GridCacheAbstractFullAp
      * @throws Exception If failed.
      */
     public void testPartitionEntrySetToString() throws Exception {
-        GridCacheAdapter<String, Integer> cache = ((GridKernal)grid(0)).internalCache();
+        GridCacheAdapter<String, Integer> cache = ((IgniteKernal)grid(0)).internalCache();
 
         for (int i = 0; i < 100; i++) {
             String key = String.valueOf(i);
@@ -82,7 +82,7 @@ public class GridCachePartitionedFullApiSelfTest extends GridCacheAbstractFullAp
      * @throws Exception If failed.
      */
     public void testPartitionEntrySetIterator() throws Exception {
-        GridCacheAdapter<String, Integer> cache = ((GridKernal)grid(0)).internalCache();
+        GridCacheAdapter<String, Integer> cache = ((IgniteKernal)grid(0)).internalCache();
 
         Map<Integer, Collection<String>> partMap = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
index 825e612..a8be626 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeCounterSelfTest.java
@@ -129,7 +129,7 @@ public class GridCachePartitionedMultiNodeCounterSelfTest extends GridCommonAbst
      * @return Near cache.
      */
     private static GridNearCacheAdapter<String, Integer> near(Ignite g) {
-        return (GridNearCacheAdapter<String, Integer>)((GridKernal)g).<String, Integer>internalCache();
+        return (GridNearCacheAdapter<String, Integer>)((IgniteKernal)g).<String, Integer>internalCache();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedProjectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedProjectionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedProjectionSelfTest.java
index 29c4005..bbfb552 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedProjectionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedProjectionSelfTest.java
@@ -52,7 +52,7 @@ public class GridCachePartitionedProjectionSelfTest extends GridCacheAbstractPro
      * @throws Exception In case of error.
      */
     public void testInvalidateFlag() throws Exception {
-        long topVer = ((GridKernal)grid(0)).context().discovery().topologyVersion();
+        long topVer = ((IgniteKernal)grid(0)).context().discovery().topologyVersion();
 
         try {
             // Top ver + 2.
@@ -105,7 +105,7 @@ public class GridCachePartitionedProjectionSelfTest extends GridCacheAbstractPro
                 stopGrid(i);
 
             for (Ignite g : G.allGrids()) {
-                GridKernal grid = (GridKernal)g;
+                IgniteKernal grid = (IgniteKernal)g;
 
                 // Wait until all nodes get topology version event.
                 grid.context().discovery().topologyFuture(topVer + 4).get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
index 50eefad..793b690 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
@@ -241,7 +241,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
     private Collection<Integer> nearKeys(Ignite ignite) {
         final Collection<Integer> keys = new ArrayList<>(KEY_CNT);
 
-        GridKernal kernal = (GridKernal) ignite;
+        IgniteKernal kernal = (IgniteKernal) ignite;
 
         GridCacheAffinityManager<Object, Object> affMgr = kernal.internalCache().context().affinity();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java
index 8e5f331..c490156 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/expiry/IgniteCacheExpiryPolicyAbstractTest.java
@@ -854,7 +854,7 @@ public abstract class IgniteCacheExpiryPolicyAbstractTest extends IgniteCacheAbs
         boolean found = false;
 
         for (int i = 0; i < gridCount(); i++) {
-            GridKernal grid = (GridKernal)grid(i);
+            IgniteKernal grid = (IgniteKernal)grid(i);
 
             GridCacheAdapter<Object, Object> cache = grid.context().cache().internalCache();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
index a3d69c5..0111754 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryAbstractSelfTest.java
@@ -162,7 +162,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             assertEquals("Cache is not empty: " + grid(i).cache(null).entrySet(), 0, grid(i).cache(null).size());
 
         for (int i = 0; i < gridCount(); i++) {
-            GridContinuousProcessor proc = ((GridKernal)grid(i)).context().continuous();
+            GridContinuousProcessor proc = ((IgniteKernal)grid(i)).context().continuous();
 
             assertEquals(String.valueOf(i), 2, ((Map)U.field(proc, "locInfos")).size());
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "rmtInfos")).size());
@@ -173,7 +173,7 @@ public abstract class GridCacheContinuousQueryAbstractSelfTest extends GridCommo
             assertEquals(String.valueOf(i), 0, ((Map)U.field(proc, "pending")).size());
 
             GridCacheContinuousQueryManager mgr =
-                ((GridKernal)grid(i)).context().cache().internalCache().context().continuousQueries();
+                ((IgniteKernal)grid(i)).context().cache().internalCache().context().continuousQueries();
 
             assertEquals(0, ((Map)U.field(mgr, "lsnrs")).size());
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/clock/GridTimeSyncProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/clock/GridTimeSyncProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/clock/GridTimeSyncProcessorSelfTest.java
index 019182b..5b198d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/clock/GridTimeSyncProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/clock/GridTimeSyncProcessorSelfTest.java
@@ -69,7 +69,7 @@ public class GridTimeSyncProcessorSelfTest extends GridCommonAbstractTest {
 
         try {
             // Check coordinator time deltas.
-            final GridKernal kernal = (GridKernal)grid(0);
+            final IgniteKernal kernal = (IgniteKernal)grid(0);
 
             // Wait for latest time sync history.
             GridTestUtils.waitForCondition(new PA() {
@@ -128,7 +128,7 @@ public class GridTimeSyncProcessorSelfTest extends GridCommonAbstractTest {
             }
 
             // Check coordinator time deltas.
-            final GridKernal kernal = (GridKernal)grid(0);
+            final IgniteKernal kernal = (IgniteKernal)grid(0);
 
             assertEquals(6, kernal.localNode().order());
 
@@ -197,7 +197,7 @@ public class GridTimeSyncProcessorSelfTest extends GridCommonAbstractTest {
         /** {@inheritDoc} */
         @Override public void onLifecycleEvent(LifecycleEventType evt) throws IgniteCheckedException {
             if (evt == LifecycleEventType.BEFORE_GRID_START)
-                ((GridKernalContextImpl)((GridKernal)g).context()).timeSource(new TimeShiftClockSource(delta));
+                ((GridKernalContextImpl)((IgniteKernal)g).context()).timeSource(new TimeShiftClockSource(delta));
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
index 8109f5e..24e1f7f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
@@ -101,7 +101,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
         assertEquals(GRID_CNT, grid(0).nodes().size());
 
         for (int i = 0; i < GRID_CNT; i++) {
-            GridKernal grid = (GridKernal)grid(i);
+            IgniteKernal grid = (IgniteKernal)grid(i);
 
             GridContinuousProcessor proc = grid.context().continuous();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java
index 8fa33d2..0213de3 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java
@@ -771,7 +771,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
     // TODO Enable after GG-8578.
     @SuppressWarnings("ConstantConditions")
     public void _testFormat() throws Exception {
-        GridKernal grid = (GridKernal)G.ignite("grid");
+        IgniteKernal grid = (IgniteKernal)G.ignite("grid");
         GridCache cache = grid.internalCache("dataCache");
 
         if (dual)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsModesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsModesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsModesSelfTest.java
index 28b417e..12f16d6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsModesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsModesSelfTest.java
@@ -39,7 +39,7 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridGgfsModesSelfTest extends GridGgfsCommonAbstractTest {
     /** Grid instance hosting primary GGFS. */
-    private GridEx grid;
+    private IgniteEx grid;
 
     /** Primary GGFS. */
     private GridGgfsImpl ggfs;
@@ -133,7 +133,7 @@ public class GridGgfsModesSelfTest extends GridGgfsCommonAbstractTest {
         cfg.setLocalHost("127.0.0.1");
         cfg.setRestEnabled(false);
 
-        grid = (GridEx)G.start(cfg);
+        grid = (IgniteEx)G.start(cfg);
 
         ggfs = (GridGgfsImpl)grid.fileSystem("ggfs");
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorSelfTest.java
index 094ba2a..2de8e74 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsProcessorSelfTest.java
@@ -68,7 +68,7 @@ public class GridGgfsProcessorSelfTest extends GridGgfsCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        GridEx grid = grid(0);
+        IgniteEx grid = grid(0);
 
         ggfs = grid.fileSystem(ggfsName());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
index 9761287..f325a5e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsServerManagerIpcEndpointRegistrationAbstractSelfTest.java
@@ -96,7 +96,7 @@ public abstract class GridGgfsServerManagerIpcEndpointRegistrationAbstractSelfTe
      * @return Tuple2 where (tcp endpoints count, shmem endpoints count).
      */
     protected T2<Integer, Integer> checkRegisteredIpcEndpoints() throws Exception {
-        GridKernalContext ctx = ((GridKernal)grid()).context();
+        GridKernalContext ctx = ((IgniteKernal)grid()).context();
 
         int tcp = 0;
         int shmem = 0;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsSizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsSizeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsSizeSelfTest.java
index f279f86..b4f866d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsSizeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsSizeSelfTest.java
@@ -307,7 +307,7 @@ public class GridGgfsSizeSelfTest extends GridGgfsCommonAbstractTest {
 
         // Ensure that cache was marked as GGFS data cache.
         for (int i = 0; i < GRID_CNT; i++) {
-            GridEx g = grid(i);
+            IgniteEx g = grid(i);
 
             GridCacheProjectionEx cache = (GridCacheProjectionEx)g.cachex(DATA_CACHE_NAME).cache();
 
@@ -688,7 +688,7 @@ public class GridGgfsSizeSelfTest extends GridGgfsCommonAbstractTest {
      * @return Node ID.
      */
     private UUID primary(GridGgfsBlockKey key) {
-        GridEx grid = grid(0);
+        IgniteEx grid = grid(0);
 
         for (ClusterNode node : grid.nodes()) {
             if (grid.cachex(DATA_CACHE_NAME).affinity().isPrimary(node, key))
@@ -705,7 +705,7 @@ public class GridGgfsSizeSelfTest extends GridGgfsCommonAbstractTest {
      * @return Collection of node IDs.
      */
     private Collection<UUID> primaryOrBackups(GridGgfsBlockKey key) {
-        GridEx grid = grid(0);
+        IgniteEx grid = grid(0);
 
         Collection<UUID> ids = new HashSet<>();
 
@@ -746,7 +746,7 @@ public class GridGgfsSizeSelfTest extends GridGgfsCommonAbstractTest {
      * @return Data cache.
      */
     private GridCacheAdapter<GridGgfsBlockKey, byte[]> cache(UUID nodeId) {
-        return (GridCacheAdapter<GridGgfsBlockKey, byte[]>)((GridEx)G.ignite(nodeId)).cachex(DATA_CACHE_NAME)
+        return (GridCacheAdapter<GridGgfsBlockKey, byte[]>)((IgniteEx)G.ignite(nodeId)).cachex(DATA_CACHE_NAME)
             .<GridGgfsBlockKey, byte[]>cache();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
index dd2f623..d36726c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
@@ -75,7 +75,7 @@ public class GridCacheCommandHandlerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCacheGetFailsSyncNotify() throws Exception {
-        GridRestCommandHandler hnd = new TestableGridCacheCommandHandler(((GridKernal)grid()).context(), "getAsync",
+        GridRestCommandHandler hnd = new TestableGridCacheCommandHandler(((IgniteKernal)grid()).context(), "getAsync",
             true);
 
         GridRestCacheRequest req = new GridRestCacheRequest();
@@ -100,7 +100,7 @@ public class GridCacheCommandHandlerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCacheGetFailsAsyncNotify() throws Exception {
-        GridRestCommandHandler hnd = new TestableGridCacheCommandHandler(((GridKernal)grid()).context(), "getAsync",
+        GridRestCommandHandler hnd = new TestableGridCacheCommandHandler(((IgniteKernal)grid()).context(), "getAsync",
             false);
 
         GridRestCacheRequest req = new GridRestCacheRequest();
@@ -175,7 +175,7 @@ public class GridCacheCommandHandlerSelfTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException In case of any grid exception.
      */
     private <T> T testAppend(T curVal, T newVal, boolean append) throws IgniteCheckedException {
-        GridRestCommandHandler hnd = new GridCacheCommandHandler(((GridKernal)grid()).context());
+        GridRestCommandHandler hnd = new GridCacheCommandHandler(((IgniteKernal)grid()).context());
 
         String key = UUID.randomUUID().toString();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceReassignmentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceReassignmentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceReassignmentSelfTest.java
index a619813..7f2f0ce 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceReassignmentSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceReassignmentSelfTest.java
@@ -145,7 +145,7 @@ public class GridServiceReassignmentSelfTest extends GridServiceProcessorAbstrac
      * @throws Exception If failed.
      */
     private void checkServices(int total, int maxPerNode, int gridIdx) throws Exception {
-        GridEx grid = grid(gridIdx);
+        IgniteEx grid = grid(gridIdx);
 
         GridCacheProjectionEx<GridServiceAssignmentsKey, GridServiceAssignments> cache = grid.
             utilityCache(GridServiceAssignmentsKey.class, GridServiceAssignments.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java
index 43a8cf0..4a292d9 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java
@@ -63,7 +63,7 @@ public class GridFinishedFutureSelfTest extends GridCommonAbstractTest {
      * @throws Exception In case of any exception.
      */
     private void testExternalizable(@Nullable Object t, @Nullable Throwable ex, boolean syncNotify) throws Exception {
-        GridKernalContext ctx = ((GridKernal)grid()).context();
+        GridKernalContext ctx = ((IgniteKernal)grid()).context();
 
         IgniteMarshaller m = new IgniteOptimizedMarshaller();
         ClassLoader clsLdr = getClass().getClassLoader();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java
index ffbda7a..4756425 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark.java
@@ -118,7 +118,7 @@ public class GridIoManagerBenchmark {
         X.println("Duration: " + duration);
         X.println("Output file name: " + outputFilename);
 
-        GridKernal g = (GridKernal)G.start(path);
+        IgniteKernal g = (IgniteKernal)G.start(path);
 
         if (g.localNode().order() > 1) {
             try {
@@ -139,7 +139,7 @@ public class GridIoManagerBenchmark {
      * @param outputFilename Output file name.
      */
     @SuppressWarnings("deprecation")
-    private static void sendMessages(GridKernal g, int threads, int duration, @Nullable final String outputFilename) {
+    private static void sendMessages(IgniteKernal g, int threads, int duration, @Nullable final String outputFilename) {
         X.println(">>> Sending messages.");
 
         g.context().io().addMessageListener(TEST_TOPIC, new SenderMessageListener());
@@ -221,7 +221,7 @@ public class GridIoManagerBenchmark {
      * @param g Kernal.
      */
     @SuppressWarnings("deprecation")
-    private static void receiveMessages(final GridKernal g) {
+    private static void receiveMessages(final IgniteKernal g) {
         X.println(">>> Receiving messages.");
 
         final GridIoManager io = g.context().io();
@@ -254,12 +254,12 @@ public class GridIoManagerBenchmark {
      */
     private static class SendThread extends Thread {
         /** */
-        private final GridKernal g;
+        private final IgniteKernal g;
 
         /**
          * @param g Kernal.
          */
-        SendThread(GridKernal g) {
+        SendThread(IgniteKernal g) {
             this.g = g;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
index 7442d25..4bca8dd 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
@@ -101,8 +101,8 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
      */
     @SuppressWarnings("deprecation")
     public void testThroughput() throws Exception {
-        final GridKernal sndKernal = (GridKernal)grid(0);
-        final GridKernal rcvKernal = (GridKernal)grid(1);
+        final IgniteKernal sndKernal = (IgniteKernal)grid(0);
+        final IgniteKernal rcvKernal = (IgniteKernal)grid(1);
 
         final ClusterNode sndNode = sndKernal.localNode();
         final ClusterNode rcvNode = rcvKernal.localNode();
@@ -197,8 +197,8 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
      */
     @SuppressWarnings("deprecation")
     public void testLatency() throws Exception {
-        final GridKernal sndKernal = (GridKernal)grid(0);
-        final GridKernal rcvKernal = (GridKernal)grid(1);
+        final IgniteKernal sndKernal = (IgniteKernal)grid(0);
+        final IgniteKernal rcvKernal = (IgniteKernal)grid(1);
 
         final ClusterNode sndNode = sndKernal.localNode();
         final ClusterNode rcvNode = rcvKernal.localNode();
@@ -293,8 +293,8 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
      */
     @SuppressWarnings("deprecation")
     public void testVariableLoad() throws Exception {
-        final GridKernal sndKernal = (GridKernal)grid(0);
-        final GridKernal rcvKernal = (GridKernal)grid(1);
+        final IgniteKernal sndKernal = (IgniteKernal)grid(0);
+        final IgniteKernal rcvKernal = (IgniteKernal)grid(1);
 
         final ClusterNode sndNode = sndKernal.localNode();
         final ClusterNode rcvNode = rcvKernal.localNode();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java b/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java
index 483cbb5..bee3f5f 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/dsi/GridDsiPerfJob.java
@@ -105,7 +105,7 @@ public class GridDsiPerfJob extends ComputeJobAdapter {
 
         long cnt = cntrs.get1().incrementAndGet();
 
-        GridNearCacheAdapter near = (GridNearCacheAdapter)((GridKernal) ignite).internalCache(cacheName);
+        GridNearCacheAdapter near = (GridNearCacheAdapter)((IgniteKernal) ignite).internalCache(cacheName);
         GridDhtCacheAdapter dht = near.dht();
 
         doWork();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
index bd6a75c..f8a62f9 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
@@ -445,7 +445,7 @@ public abstract class GridMarshallerAbstractTest extends GridCommonAbstractTest
     }
 
     /**
-     * Tests marshal {@link GridKernal} instance.
+     * Tests marshal {@link IgniteKernal} instance.
      *
      * @throws Exception If test failed.
      */
@@ -459,8 +459,8 @@ public abstract class GridMarshallerAbstractTest extends GridCommonAbstractTest
         assert inBean.getObjectField() != null;
         assert outBean.getObjectField() != null;
 
-        assert inBean.getObjectField().getClass().equals(GridKernal.class);
-        assert outBean.getObjectField().getClass().equals(GridKernal.class);
+        assert inBean.getObjectField().getClass().equals(IgniteKernal.class);
+        assert outBean.getObjectField().getClass().equals(IgniteKernal.class);
 
         assert inBean != outBean;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java
index e81fcba..3d91ce8 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java
@@ -124,7 +124,7 @@ public class GridTcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
         long prev = 0;
 
         for (Ignite g : G.allGrids()) {
-            GridKernal kernal = (GridKernal)g;
+            IgniteKernal kernal = (IgniteKernal)g;
 
             long ver = kernal.context().discovery().topologyVersion();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySelfTest.java
index 41e86fb..6db7f95 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoverySelfTest.java
@@ -714,7 +714,7 @@ public class GridTcpDiscoverySelfTest extends GridCommonAbstractTest {
 
                 boolean found = false;
 
-                for (GridPortRecord rec : ((GridKernal) g).context().ports().records()) {
+                for (GridPortRecord rec : ((IgniteKernal) g).context().ports().records()) {
                     if ((rec.protocol() == UDP) && rec.port() == ipFinder.getMulticastPort()) {
                         found = true;
 
@@ -897,12 +897,12 @@ public class GridTcpDiscoverySelfTest extends GridCommonAbstractTest {
 
             Long startTime = null;
 
-            GridKernal firstGrid = null;
+            IgniteKernal firstGrid = null;
 
-            Collection<GridKernal> grids = new ArrayList<>();
+            Collection<IgniteKernal> grids = new ArrayList<>();
 
             for (int i = 0; i < 5 ; i++) {
-                GridKernal grid = (GridKernal)grid(i);
+                IgniteKernal grid = (IgniteKernal)grid(i);
 
                 assertTrue(grid.context().discovery().gridStartTime() > 0);
 
@@ -921,12 +921,12 @@ public class GridTcpDiscoverySelfTest extends GridCommonAbstractTest {
 
             stopGrid(firstGrid.name());
 
-            for (GridKernal grid : grids)
+            for (IgniteKernal grid : grids)
                 assertEquals(startTime, (Long)grid.context().discovery().gridStartTime());
 
-            grids.add((GridKernal) startGrid(5));
+            grids.add((IgniteKernal) startGrid(5));
 
-            for (GridKernal grid : grids)
+            for (IgniteKernal grid : grids)
                 assertEquals(startTime, (Long)grid.context().discovery().gridStartTime());
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index 433034d..29a623e 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -843,7 +843,7 @@ public final class GridTestUtils {
      * @return Cache context.
      */
     public static <K, V> GridCacheContext<K, V> cacheContext(CacheProjection<K, V> cache) {
-        return ((GridKernal)cache.gridProjection().ignite()).<K, V>internalCache().context();
+        return ((IgniteKernal)cache.gridProjection().ignite()).<K, V>internalCache().context();
     }
 
     /**
@@ -880,7 +880,7 @@ public final class GridTestUtils {
     public static <K, V> void waitTopologyUpdate(@Nullable String cacheName, int backups, IgniteLogger log)
         throws Exception {
         for (Ignite g : Ignition.allGrids()) {
-            GridCache<K, V> cache = ((GridEx)g).cachex(cacheName);
+            GridCache<K, V> cache = ((IgniteEx)g).cachex(cacheName);
 
             GridDhtPartitionTopology<?, ?> top = dht(cache).topology();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 4a4ca95..fca61d1 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -794,8 +794,8 @@ public abstract class GridAbstractTest extends TestCase {
      * @param name Name.
      * @return Grid instance.
      */
-    protected GridEx grid(String name) {
-        return (GridEx)G.ignite(name);
+    protected IgniteEx grid(String name) {
+        return (IgniteEx)G.ignite(name);
     }
 
     /**
@@ -804,8 +804,8 @@ public abstract class GridAbstractTest extends TestCase {
      * @param idx Index.
      * @return Grid instance.
      */
-    protected GridEx grid(int idx) {
-        return (GridEx)G.ignite(getTestGridName(idx));
+    protected IgniteEx grid(int idx) {
+        return (IgniteEx)G.ignite(getTestGridName(idx));
     }
 
     /**
@@ -821,8 +821,8 @@ public abstract class GridAbstractTest extends TestCase {
      *
      * @return Grid for given test.
      */
-    protected GridEx grid() {
-        return (GridEx)G.ignite(getTestGridName());
+    protected IgniteEx grid() {
+        return (IgniteEx)G.ignite(getTestGridName());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
index 14d8817..f07fa98 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridTestKernalContext.java
@@ -34,7 +34,7 @@ public class GridTestKernalContext extends GridKernalContextImpl {
      */
     public GridTestKernalContext() {
         super(null,
-            new GridKernal(null),
+            new IgniteKernal(null),
             new IgniteConfiguration(),
             new GridKernalGatewayImpl(null),
             null,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index b5d7c94..3dc8f85 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -102,7 +102,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return Cache.
      */
     protected <K, V> GridLocalCache<K, V> local() {
-        return (GridLocalCache<K, V>)((GridKernal)grid()).<K, V>internalCache();
+        return (GridLocalCache<K, V>)((IgniteKernal)grid()).<K, V>internalCache();
     }
 
     /**
@@ -111,7 +111,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      */
     protected static <K, V> GridDhtCacheAdapter<K, V> dht(CacheProjection<K,V> cache) {
         return nearEnabled(cache) ? near(cache).dht() :
-            ((GridKernal)cache.gridProjection().ignite()).<K, V>internalCache(cache.name()).context().dht();
+            ((IgniteKernal)cache.gridProjection().ignite()).<K, V>internalCache(cache.name()).context().dht();
     }
 
     /**
@@ -144,7 +144,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return Colocated cache.
      */
     protected <K, V> GridDhtColocatedCache<K, V> colocated(int idx, String cache) {
-        return (GridDhtColocatedCache<K, V>)((GridKernal)grid(idx)).internalCache(cache);
+        return (GridDhtColocatedCache<K, V>)((IgniteKernal)grid(idx)).internalCache(cache);
     }
 
     /**
@@ -152,7 +152,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return {@code True} if near cache is enabled.
      */
     protected static <K, V> boolean nearEnabled(CacheProjection<K,V> cache) {
-        CacheConfiguration cfg = ((GridKernal)cache.gridProjection().ignite()).
+        CacheConfiguration cfg = ((IgniteKernal)cache.gridProjection().ignite()).
             <K, V>internalCache(cache.name()).context().config();
 
         return isNearEnabled(cfg);
@@ -163,7 +163,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return Near cache.
      */
     protected static <K, V> GridNearCacheAdapter<K, V> near(CacheProjection<K,V> cache) {
-        return ((GridKernal)cache.gridProjection().ignite()).<K, V>internalCache(cache.name()).context().near();
+        return ((IgniteKernal)cache.gridProjection().ignite()).<K, V>internalCache(cache.name()).context().near();
     }
 
     /**
@@ -171,14 +171,14 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return Colocated cache.
      */
     protected static <K, V> GridDhtColocatedCache<K, V> colocated(CacheProjection<K,V> cache) {
-        return ((GridKernal)cache.gridProjection().ignite()).<K, V>internalCache(cache.name()).context().colocated();
+        return ((IgniteKernal)cache.gridProjection().ignite()).<K, V>internalCache(cache.name()).context().colocated();
     }
 
     /**
      * @return Near cache.
      */
     protected <K, V> GridNearCacheAdapter<K, V> near() {
-        return ((GridKernal)grid()).<K, V>internalCache().context().near();
+        return ((IgniteKernal)grid()).<K, V>internalCache().context().near();
     }
 
     /**
@@ -186,7 +186,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return Near cache.
      */
     protected <K, V> GridNearCacheAdapter<K, V> near(int idx) {
-        return ((GridKernal)grid(idx)).<K, V>internalCache().context().near();
+        return ((IgniteKernal)grid(idx)).<K, V>internalCache().context().near();
     }
 
     /**
@@ -194,7 +194,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return Colocated cache.
      */
     protected <K, V> GridDhtColocatedCache<K, V> colocated(int idx) {
-        return (GridDhtColocatedCache<K, V>)((GridKernal)grid(idx)).<K, V>internalCache();
+        return (GridDhtColocatedCache<K, V>)((IgniteKernal)grid(idx)).<K, V>internalCache();
     }
 
     /**
@@ -203,7 +203,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return Near cache.
      */
     protected <K, V> GridNearCacheAdapter<K, V> near(int idx, String cache) {
-        return ((GridKernal)grid(idx)).<K, V>internalCache(cache).context().near();
+        return ((IgniteKernal)grid(idx)).<K, V>internalCache(cache).context().near();
     }
 
     /** {@inheritDoc} */
@@ -247,7 +247,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
     @SuppressWarnings("BusyWait")
     protected void awaitPartitionMapExchange() throws InterruptedException {
         for (Ignite g : G.allGrids()) {
-            for (GridCache<?, ?> c : ((GridEx)g).cachesx()) {
+            for (GridCache<?, ?> c : ((IgniteEx)g).cachesx()) {
                 CacheConfiguration cfg = c.configuration();
 
                 if (cfg.getCacheMode() == PARTITIONED && cfg.getPreloadMode() != NONE && g.cluster().nodes().size() > 1) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
index 182796b..f098759 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteBasicTestSuite.java
@@ -68,7 +68,7 @@ public class IgniteBasicTestSuite extends TestSuite {
         suite.addTestSuite(GridStopWithCancelSelfTest.class);
         suite.addTestSuite(GridReduceSelfTest.class);
         suite.addTestSuite(GridEventConsumeSelfTest.class);
-        suite.addTestSuite(GridExceptionHelpLinksSelfTest.class);
+        suite.addTestSuite(IgniteExceptionHelpLinksSelfTest.class);
         suite.addTestSuite(GridSuppressedExceptionSelfTest.class);
         suite.addTestSuite(GridLifecycleAwareSelfTest.class);
         suite.addTestSuite(GridMessageListenSelfTest.class);


[48/50] [abbrv] incubator-ignite git commit: #Tests: fix ignite-readme.pdf

Posted by vo...@apache.org.
#Tests: fix ignite-readme.pdf


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

Branch: refs/heads/ignite-16
Commit: 45353228310224dbf4f86fb7b278e315b87bd0b2
Parents: d5bef13
Author: ivasilinets <iv...@gridgain.com>
Authored: Thu Jan 29 12:49:22 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Thu Jan 29 12:49:22 2015 +0300

----------------------------------------------------------------------
 assembly/release-base-fabric.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/45353228/assembly/release-base-fabric.xml
----------------------------------------------------------------------
diff --git a/assembly/release-base-fabric.xml b/assembly/release-base-fabric.xml
index a13c6dc..bd70839 100644
--- a/assembly/release-base-fabric.xml
+++ b/assembly/release-base-fabric.xml
@@ -29,7 +29,7 @@
         </file>
 
         <file>
-            <source>docs/gridgain_readme.pdf</source>
+            <source>docs/ignite_readme.pdf</source>
             <outputDirectory>/docs</outputDirectory>
         </file>
     </files>


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

Posted by vo...@apache.org.
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());


[44/50] [abbrv] incubator-ignite git commit: GridNamedInstance -> IgniteNamedInstance GridgainEx -> IgnitionEx

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 776cda6..88dbef6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -888,7 +888,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
      */
     protected Object readResolve() throws ObjectStreamException {
         try {
-            return GridGainEx.gridx(stash.get()).context();
+            return IgnitionEx.gridx(stash.get()).context();
         }
         catch (IllegalStateException e) {
             throw U.withCause(new InvalidObjectException(e.getMessage()), e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java
index 042e3ec..d8e6816 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridLoggerProxy.java
@@ -200,7 +200,7 @@ public class GridLoggerProxy implements IgniteLogger, LifecycleAware, Externaliz
             String gridNameR = t.get1();
             Object ctgrR = t.get2();
 
-            return GridGainEx.gridx(gridNameR).log().getLogger(ctgrR);
+            return IgnitionEx.gridx(gridNameR).log().getLogger(ctgrR);
         }
         catch (IllegalStateException e) {
             throw U.withCause(new InvalidObjectException(e.getMessage()), e);


[46/50] [abbrv] incubator-ignite git commit: GridNamedInstance -> IgniteNamedInstance GridgainEx -> IgnitionEx

Posted by vo...@apache.org.
GridNamedInstance -> IgniteNamedInstance
GridgainEx -> IgnitionEx


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

Branch: refs/heads/ignite-16
Commit: c5247ab37b5233520bb610570d9a1dd2b793d0bb
Parents: c323a8e
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jan 29 12:44:57 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jan 29 12:44:57 2015 +0300

----------------------------------------------------------------------
 .../main/java/org/apache/ignite/Ignition.java   |   38 +-
 .../apache/ignite/cache/CachingProvider.java    |    2 +-
 .../ignite/internal/ClusterGroupAdapter.java    |    2 +-
 .../internal/ClusterNodeLocalMapImpl.java       |    2 +-
 .../org/apache/ignite/internal/GridGainEx.java  | 2399 ------------------
 .../ignite/internal/GridKernalContextImpl.java  |    2 +-
 .../apache/ignite/internal/GridLoggerProxy.java |    2 +-
 .../org/apache/ignite/internal/IgnitionEx.java  | 2399 ++++++++++++++++++
 .../processors/cache/GridCacheAdapter.java      |    2 +-
 .../processors/cache/GridCacheContext.java      |    2 +-
 .../resource/GridSpringResourceContext.java     |    2 +-
 .../fs/GridGgfsEventsAbstractSelfTest.java      |    2 +-
 .../cache/GridCachePutRemoveLoadTest.java       |    2 +-
 .../testframework/junits/GridAbstractTest.java  |    2 +-
 .../java/org/apache/ignite/IgniteSpring.java    |    8 +-
 .../GridServletContextListenerStartup.java      |    4 +-
 .../startup/servlet/GridServletStartup.java     |    4 +-
 17 files changed, 2437 insertions(+), 2437 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/Ignition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/Ignition.java b/modules/core/src/main/java/org/apache/ignite/Ignition.java
index 568b520..a8c3619 100644
--- a/modules/core/src/main/java/org/apache/ignite/Ignition.java
+++ b/modules/core/src/main/java/org/apache/ignite/Ignition.java
@@ -120,7 +120,7 @@ public class Ignition {
      * @param daemon Daemon flag to set.
      */
     public static void setDaemon(boolean daemon) {
-        GridGainEx.setDaemon(daemon);
+        IgnitionEx.setDaemon(daemon);
     }
 
     /**
@@ -134,7 +134,7 @@ public class Ignition {
      * @return Daemon flag.
      */
     public static boolean isDaemon() {
-        return GridGainEx.isDaemon();
+        return IgnitionEx.isDaemon();
     }
 
     /**
@@ -143,7 +143,7 @@ public class Ignition {
      * @return Default grid state.
      */
     public static IgniteState state() {
-        return GridGainEx.state();
+        return IgnitionEx.state();
     }
 
     /**
@@ -155,7 +155,7 @@ public class Ignition {
      * @return Grid state.
      */
     public static IgniteState state(@Nullable String name) {
-        return GridGainEx.state(name);
+        return IgnitionEx.state(name);
     }
 
     /**
@@ -170,7 +170,7 @@ public class Ignition {
      *      {@code false} otherwise (if it was not started).
      */
     public static boolean stop(boolean cancel) {
-        return GridGainEx.stop(cancel);
+        return IgnitionEx.stop(cancel);
     }
 
     /**
@@ -193,7 +193,7 @@ public class Ignition {
      *      not found).
      */
     public static boolean stop(@Nullable String name, boolean cancel) {
-        return GridGainEx.stop(name, cancel);
+        return IgnitionEx.stop(name, cancel);
     }
 
     /**
@@ -212,7 +212,7 @@ public class Ignition {
      *      up to the actual job to exit from execution
      */
     public static void stopAll(boolean cancel) {
-        GridGainEx.stopAll(cancel);
+        IgnitionEx.stopAll(cancel);
     }
 
     /**
@@ -236,7 +236,7 @@ public class Ignition {
      * @see #RESTART_EXIT_CODE
      */
     public static void restart(boolean cancel) {
-        GridGainEx.restart(cancel);
+        IgnitionEx.restart(cancel);
     }
 
     /**
@@ -259,7 +259,7 @@ public class Ignition {
      * @see #KILL_EXIT_CODE
      */
     public static void kill(boolean cancel) {
-        GridGainEx.kill(cancel);
+        IgnitionEx.kill(cancel);
     }
 
     /**
@@ -272,7 +272,7 @@ public class Ignition {
      *      also if default grid has already been started.
      */
     public static Ignite start() throws IgniteCheckedException {
-        return GridGainEx.start();
+        return IgnitionEx.start();
     }
 
     /**
@@ -285,7 +285,7 @@ public class Ignition {
      *      also if named grid has already been started.
      */
     public static Ignite start(IgniteConfiguration cfg) throws IgniteCheckedException {
-        return GridGainEx.start(cfg);
+        return IgnitionEx.start(cfg);
     }
 
     /**
@@ -305,7 +305,7 @@ public class Ignition {
      *      been started or Spring XML configuration file is invalid.
      */
     public static Ignite start(@Nullable String springCfgPath) throws IgniteCheckedException {
-        return GridGainEx.start(springCfgPath);
+        return IgnitionEx.start(springCfgPath);
     }
 
     /**
@@ -325,7 +325,7 @@ public class Ignition {
      *      been started or Spring XML configuration file is invalid.
      */
     public static Ignite start(URL springCfgUrl) throws IgniteCheckedException {
-        return GridGainEx.start(springCfgUrl);
+        return IgnitionEx.start(springCfgUrl);
     }
 
     /**
@@ -341,7 +341,7 @@ public class Ignition {
      *      initialized or grid instance was stopped or was not started.
      */
     public static Ignite ignite() throws IgniteIllegalStateException {
-        return GridGainEx.grid();
+        return IgnitionEx.grid();
     }
 
     /**
@@ -350,7 +350,7 @@ public class Ignition {
      * @return List of all grids started so far.
      */
     public static List<Ignite> allGrids() {
-        return GridGainEx.allGrids();
+        return IgnitionEx.allGrids();
     }
 
     /**
@@ -366,7 +366,7 @@ public class Ignition {
      *      initialized or grid instance was stopped or was not started.
      */
     public static Ignite ignite(UUID locNodeId) throws IgniteIllegalStateException {
-        return GridGainEx.grid(locNodeId);
+        return IgnitionEx.grid(locNodeId);
     }
 
     /**
@@ -386,7 +386,7 @@ public class Ignition {
      *      initialized or grid instance was stopped or was not started.
      */
     public static Ignite ignite(@Nullable String name) throws IgniteIllegalStateException {
-        return GridGainEx.grid(name);
+        return IgnitionEx.grid(name);
     }
 
     /**
@@ -402,7 +402,7 @@ public class Ignition {
      *      this method is no-op.
      */
     public static void addListener(IgniteListener lsnr) {
-        GridGainEx.addListener(lsnr);
+        IgnitionEx.addListener(lsnr);
     }
 
     /**
@@ -412,6 +412,6 @@ public class Ignition {
      * @return {@code true} if lsnr was added before, {@code false} otherwise.
      */
     public static boolean removeListener(IgniteListener lsnr) {
-        return GridGainEx.removeListener(lsnr);
+        return IgnitionEx.removeListener(lsnr);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/cache/CachingProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CachingProvider.java b/modules/core/src/main/java/org/apache/ignite/cache/CachingProvider.java
index 1b6c945..c594594 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CachingProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CachingProvider.java
@@ -40,7 +40,7 @@ public class CachingProvider implements javax.cache.spi.CachingProvider {
         URI uri = null;
 
         try {
-            URL dfltCfgURL = U.resolveGridGainUrl(GridGainEx.DFLT_CFG);
+            URL dfltCfgURL = U.resolveGridGainUrl(IgnitionEx.DFLT_CFG);
 
             if (dfltCfgURL != null)
                 uri = dfltCfgURL.toURI();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
index b3f8cc0..820735e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
@@ -638,7 +638,7 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable {
      */
     protected Object readResolve() throws ObjectStreamException {
         try {
-            IgniteKernal g = GridGainEx.gridx(gridName);
+            IgniteKernal g = IgnitionEx.gridx(gridName);
 
             return ids != null ? new ClusterGroupAdapter(g, g.context(), subjId, ids) :
                 p != null ? new ClusterGroupAdapter(g, g.context(), subjId, p) : g;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/internal/ClusterNodeLocalMapImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ClusterNodeLocalMapImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/ClusterNodeLocalMapImpl.java
index 5487f0a..c5378a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/ClusterNodeLocalMapImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/ClusterNodeLocalMapImpl.java
@@ -85,7 +85,7 @@ public class ClusterNodeLocalMapImpl<K, V> extends ConcurrentHashMap8<K, V> impl
      */
     protected Object readResolve() throws ObjectStreamException {
         try {
-            return GridGainEx.gridx(stash.get()).nodeLocalMap();
+            return IgnitionEx.gridx(stash.get()).nodeLocalMap();
         }
         catch (IllegalStateException e) {
             throw U.withCause(new InvalidObjectException(e.getMessage()), e);


[14/50] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-1' into ignite-91

Posted by vo...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-1' into ignite-91


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

Branch: refs/heads/ignite-16
Commit: 645aa34e83326b7f626fb832d311fda20e829e9b
Parents: 5673e3f 673d11b
Author: sboikov <se...@inria.fr>
Authored: Thu Jan 29 06:07:46 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Thu Jan 29 06:07:46 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheStoreManager.java   | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------



[06/50] [abbrv] incubator-ignite git commit: fixed timers stop

Posted by vo...@apache.org.
fixed timers stop


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

Branch: refs/heads/ignite-16
Commit: 38164d05670f920123078865bd6f0fec72020422
Parents: be6686d
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jan 28 19:00:11 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jan 28 19:00:11 2015 +0300

----------------------------------------------------------------------
 .../org/apache/ignite/internal/GridKernal.java  | 30 ++++++++++----------
 1 file changed, 15 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/38164d05/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
index e13f48a..35484a8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
@@ -1923,6 +1923,21 @@ public class GridKernal extends ClusterGroupAdapter implements GridEx, IgniteMXB
                 }
             }
 
+            // Cancel update notification timer.
+            if (updateNtfTimer != null)
+                updateNtfTimer.cancel();
+
+            if (starveTimer != null)
+                starveTimer.cancel();
+
+            // Cancel license timer.
+            if (licTimer != null)
+                licTimer.cancel();
+
+            // Cancel metrics log timer.
+            if (metricsLogTimer != null)
+                metricsLogTimer.cancel();
+
             gw.writeLock();
 
             try {
@@ -1931,21 +1946,6 @@ public class GridKernal extends ClusterGroupAdapter implements GridEx, IgniteMXB
                 // No more kernal calls from this point on.
                 gw.setState(STOPPING);
 
-                // Cancel update notification timer.
-                if (updateNtfTimer != null)
-                    updateNtfTimer.cancel();
-
-                if (starveTimer != null)
-                    starveTimer.cancel();
-
-                // Cancel license timer.
-                if (licTimer != null)
-                    licTimer.cancel();
-
-                // Cancel metrics log timer.
-                if (metricsLogTimer != null)
-                    metricsLogTimer.cancel();
-
                 // Clear node local store.
                 nodeLoc.clear();
 


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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
index 88dc193..c9681c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheAdapter.java
@@ -258,7 +258,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
      * @param topVer Topology version.
      * @return Finish future.
      */
-    @Nullable public IgniteFuture<?> multiUpdateFinishFuture(long topVer) {
+    @Nullable public IgniteInternalFuture<?> multiUpdateFinishFuture(long topVer) {
         GridCompoundFuture<IgniteUuid, Object> fut = null;
 
         for (MultiUpdateFuture multiFut : multiTxFuts.values()) {
@@ -487,7 +487,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
      * @param filter {@inheritDoc}
      * @return {@inheritDoc}
      */
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(
+    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
         @Nullable Collection<? extends K> keys,
         boolean forcePrimary,
         boolean skipTx,
@@ -530,7 +530,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
      * @param expiry Expiry policy.
      * @return Get future.
      */
-    IgniteFuture<Map<K, V>> getDhtAllAsync(@Nullable Collection<? extends K> keys,
+    IgniteInternalFuture<Map<K, V>> getDhtAllAsync(@Nullable Collection<? extends K> keys,
         boolean readThrough,
         @Nullable UUID subjId,
         String taskName,
@@ -605,7 +605,7 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
 
         final GetExpiryPolicy expiryPlc = ttl == -1L ? null : new GetExpiryPolicy(ttl);
 
-        IgniteFuture<Collection<GridCacheEntryInfo<K, V>>> fut =
+        IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>> fut =
             getDhtAsync(nodeId,
                 req.messageId(),
                 req.keys(),
@@ -618,8 +618,8 @@ public abstract class GridDhtCacheAdapter<K, V> extends GridDistributedCacheAdap
                 req.filter(),
                 expiryPlc);
 
-        fut.listenAsync(new CI1<IgniteFuture<Collection<GridCacheEntryInfo<K, V>>>>() {
-            @Override public void apply(IgniteFuture<Collection<GridCacheEntryInfo<K, V>>> f) {
+        fut.listenAsync(new CI1<IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>>>() {
+            @Override public void apply(IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>> f) {
                 GridNearGetResponse<K, V> res = new GridNearGetResponse<>(ctx.cacheId(),
                     req.futureId(),
                     req.miniId(),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
index 52be7a5..550a693 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtCacheEntry.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -355,7 +356,7 @@ public class GridDhtCacheEntry<K, V> extends GridDistributedCacheEntry<K, V> {
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     @SuppressWarnings("unchecked")
-    @Nullable public IgniteFuture<Boolean> addReader(UUID nodeId, long msgId, long topVer)
+    @Nullable public IgniteInternalFuture<Boolean> addReader(UUID nodeId, long msgId, long topVer)
         throws GridCacheEntryRemovedException {
         // Don't add local node as reader.
         if (cctx.nodeId().equals(nodeId))
@@ -450,8 +451,8 @@ public class GridDhtCacheEntry<K, V> extends GridDistributedCacheEntry<K, V> {
             if (!txFut.isDone()) {
                 final ReaderId<K, V> reader0 = reader;
 
-                txFut.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> f) {
+                txFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> f) {
                         synchronized (this) {
                             // Release memory.
                             reader0.resetTxFuture();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java
index 15ca4a0..326a813 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtEmbeddedFuture.java
@@ -49,7 +49,7 @@ public class GridDhtEmbeddedFuture<A, B> extends GridEmbeddedFuture<A, B> implem
      * @param embedded Embedded.
      * @param c Closure.
      */
-    public GridDhtEmbeddedFuture(GridKernalContext ctx, IgniteFuture<B> embedded, IgniteBiClosure<B, Exception, A> c) {
+    public GridDhtEmbeddedFuture(GridKernalContext ctx, IgniteInternalFuture<B> embedded, IgniteBiClosure<B, Exception, A> c) {
         super(ctx, embedded, c);
 
         invalidParts = Collections.emptyList();
@@ -60,8 +60,8 @@ public class GridDhtEmbeddedFuture<A, B> extends GridEmbeddedFuture<A, B> implem
      * @param c Embedding closure.
      * @param ctx Kernal context.
      */
-    public GridDhtEmbeddedFuture(IgniteFuture<B> embedded,
-        IgniteBiClosure<B, Exception, IgniteFuture<A>> c, GridKernalContext ctx) {
+    public GridDhtEmbeddedFuture(IgniteInternalFuture<B> embedded,
+        IgniteBiClosure<B, Exception, IgniteInternalFuture<A>> c, GridKernalContext ctx) {
         super(embedded, c, ctx);
 
         invalidParts = Collections.emptyList();
@@ -73,7 +73,7 @@ public class GridDhtEmbeddedFuture<A, B> extends GridEmbeddedFuture<A, B> implem
      * @param c Closure.
      * @param invalidParts Retries.
      */
-    public GridDhtEmbeddedFuture(GridKernalContext ctx, IgniteFuture<B> embedded, IgniteBiClosure<B, Exception, A> c,
+    public GridDhtEmbeddedFuture(GridKernalContext ctx, IgniteInternalFuture<B> embedded, IgniteBiClosure<B, Exception, A> c,
         Collection<Integer> invalidParts) {
         super(ctx, embedded, c);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtFuture.java
index af494d5..9044bfb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtFuture.java
@@ -17,14 +17,14 @@
 
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 
 import java.util.*;
 
 /**
  * Keys to retry.
  */
-public interface GridDhtFuture<T> extends IgniteFuture<T> {
+public interface GridDhtFuture<T> extends IgniteInternalFuture<T> {
     /**
      * Node that future should be able to provide keys to retry before
      * it completes, so it's not necessary to wait till future is done

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
index 251d550..b6282c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtGetFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -286,7 +287,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
      * @return Future for local get.
      */
     @SuppressWarnings( {"unchecked", "IfMayBeConditional"})
-    private IgniteFuture<Collection<GridCacheEntryInfo<K, V>>> getAsync(final LinkedHashMap<? extends K, Boolean> keys) {
+    private IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>> getAsync(final LinkedHashMap<? extends K, Boolean> keys) {
         if (F.isEmpty(keys))
             return new GridFinishedFuture<Collection<GridCacheEntryInfo<K, V>>>(cctx.kernalContext(),
                 Collections.<GridCacheEntryInfo<K, V>>emptyList());
@@ -321,7 +322,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
                     // TODO: To fix, check that reader is contained in the list of readers once
                     // TODO: again after the returned future completes - if not, try again.
                     // TODO: Also, why is info read before transactions are complete, and not after?
-                    IgniteFuture<Boolean> f = (!e.deleted() && k.getValue()) ? e.addReader(reader, msgId, topVer) : null;
+                    IgniteInternalFuture<Boolean> f = (!e.deleted() && k.getValue()) ? e.addReader(reader, msgId, topVer) : null;
 
                     if (f != null) {
                         if (txFut == null)
@@ -347,7 +348,7 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
         if (txFut != null)
             txFut.markInitialized();
 
-        IgniteFuture<Map<K, V>> fut;
+        IgniteInternalFuture<Map<K, V>> fut;
 
         if (txFut == null || txFut.isDone()) {
             if (reload && cctx.readThrough() && cctx.store().configured()) {
@@ -382,8 +383,8 @@ public final class GridDhtGetFuture<K, V> extends GridCompoundIdentityFuture<Col
             // transactions to complete.
             fut = new GridEmbeddedFuture<>(
                 txFut,
-                new C2<Boolean, Exception, IgniteFuture<Map<K, V>>>() {
-                    @Override public IgniteFuture<Map<K, V>> apply(Boolean b, Exception e) {
+                new C2<Boolean, Exception, IgniteInternalFuture<Map<K, V>>>() {
+                    @Override public IgniteInternalFuture<Map<K, V>> apply(Boolean b, Exception e) {
                         if (e != null)
                             throw new GridClosureException(e);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
index 373b488..a3a0ab7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLocalPartition.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -390,7 +391,7 @@ public class GridDhtLocalPartition<K, V> implements Comparable<GridDhtLocalParti
      * @param updateSeq Update sequence.
      * @return Future to signal that this node is no longer an owner or backup.
      */
-    IgniteFuture<?> rent(boolean updateSeq) {
+    IgniteInternalFuture<?> rent(boolean updateSeq) {
         while (true) {
             int reservations = state.getStamp();
 
@@ -418,7 +419,7 @@ public class GridDhtLocalPartition<K, V> implements Comparable<GridDhtLocalParti
      * @param updateSeq Update sequence.
      * @return Future for evict attempt.
      */
-    private IgniteFuture<Boolean> tryEvictAsync(boolean updateSeq) {
+    private IgniteInternalFuture<Boolean> tryEvictAsync(boolean updateSeq) {
         if (map.isEmpty() && state.compareAndSet(RENTING, EVICTED, 0, 0)) {
             if (log.isDebugEnabled())
                 log.debug("Evicted partition: " + this);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
index 8174985..f0da6b4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtLockFuture.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -234,8 +235,8 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
      * @return Participating nodes.
      */
     @Override public Collection<? extends ClusterNode> nodes() {
-        return F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-            @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+        return F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<?>, ClusterNode>() {
+            @Nullable @Override public ClusterNode apply(IgniteInternalFuture<?> f) {
                 if (isMini(f))
                     return ((MiniFuture)f).node();
 
@@ -470,7 +471,7 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
     @Override public boolean onNodeLeft(UUID nodeId) {
         boolean found = false;
 
-        for (IgniteFuture<?> fut : futures()) {
+        for (IgniteInternalFuture<?> fut : futures()) {
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -496,7 +497,7 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
 
             boolean found = false;
 
-            for (IgniteFuture<Boolean> fut : pending()) {
+            for (IgniteInternalFuture<Boolean> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture mini = (MiniFuture)fut;
 
@@ -712,7 +713,7 @@ public final class GridDhtLockFuture<K, V> extends GridCompoundIdentityFuture<Bo
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
index 45eda34..2d2f431 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTopologyFuture.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.managers.discovery.*;
 
 /**
@@ -29,7 +29,7 @@ import org.apache.ignite.internal.managers.discovery.*;
  * When new new transaction is started, it will wait for this future before acquiring new locks on particular
  * topology version.
  */
-public interface GridDhtTopologyFuture extends IgniteFuture<Long> {
+public interface GridDhtTopologyFuture extends IgniteInternalFuture<Long> {
     /**
      * Gets a topology snapshot for the topology version represented by the future. Note that by the time
      * partition exchange completes some nodes from the snapshot may leave the grid. One should use discovery

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
index 3d5477b..64c0811 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTransactionalCacheAdapter.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -343,14 +344,14 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
      * @param req Request.
      */
     protected final void processDhtLockRequest(final UUID nodeId, final GridDhtLockRequest<K, V> req) {
-        IgniteFuture<Object> keyFut = F.isEmpty(req.keys()) ? null :
+        IgniteInternalFuture<Object> keyFut = F.isEmpty(req.keys()) ? null :
             ctx.dht().dhtPreloader().request(req.keys(), req.topologyVersion());
 
         if (keyFut == null || keyFut.isDone())
             processDhtLockRequest0(nodeId, req);
         else {
-            keyFut.listenAsync(new CI1<IgniteFuture<Object>>() {
-                @Override public void apply(IgniteFuture<Object> t) {
+            keyFut.listenAsync(new CI1<IgniteInternalFuture<Object>>() {
+                @Override public void apply(IgniteInternalFuture<Object> t) {
                     processDhtLockRequest0(nodeId, req);
                 }
             });
@@ -521,7 +522,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
         }
 
         // Group lock can be only started from local node, so we never start group lock transaction on remote node.
-        IgniteFuture<?> f = lockAllAsync(ctx, nearNode, req, null);
+        IgniteInternalFuture<?> f = lockAllAsync(ctx, nearNode, req, null);
 
         // Register listener just so we print out errors.
         // Exclude lock timeout exception since it's not a fatal exception.
@@ -550,7 +551,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys,
         long timeout,
         IgniteTxLocalEx<K, V> txx,
         boolean isInvalidate,
@@ -665,14 +666,14 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
      * @param filter0 Filter.
      * @return Future.
      */
-    public IgniteFuture<GridNearLockResponse<K, V>> lockAllAsync(
+    public IgniteInternalFuture<GridNearLockResponse<K, V>> lockAllAsync(
         final GridCacheContext<K, V> cacheCtx,
         final ClusterNode nearNode,
         final GridNearLockRequest<K, V> req,
         @Nullable final IgnitePredicate<CacheEntry<K, V>>[] filter0) {
         final List<K> keys = req.keys();
 
-        IgniteFuture<Object> keyFut = null;
+        IgniteInternalFuture<Object> keyFut = null;
 
         if (req.onePhaseCommit()) {
             boolean forceKeys = req.hasTransforms() || req.filter() != null;
@@ -690,8 +691,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
             keyFut = new GridFinishedFutureEx<>();
 
         return new GridEmbeddedFuture<>(true, keyFut,
-            new C2<Object, Exception, IgniteFuture<GridNearLockResponse<K,V>>>() {
-                @Override public IgniteFuture<GridNearLockResponse<K, V>> apply(Object o, Exception exx) {
+            new C2<Object, Exception, IgniteInternalFuture<GridNearLockResponse<K,V>>>() {
+                @Override public IgniteInternalFuture<GridNearLockResponse<K, V>> apply(Object o, Exception exx) {
                     if (exx != null)
                         return new GridDhtFinishedFuture<>(ctx.kernalContext(), exx);
 
@@ -831,7 +832,7 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
 
                             assert req.writeEntries() == null || req.writeEntries().size() == entries.size();
 
-                            IgniteFuture<GridCacheReturn<V>> txFut = tx.lockAllAsync(
+                            IgniteInternalFuture<GridCacheReturn<V>> txFut = tx.lockAllAsync(
                                 cacheCtx,
                                 entries,
                                 req.writeEntries(),
@@ -846,8 +847,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
 
                             return new GridDhtEmbeddedFuture<>(
                                 txFut,
-                                new C2<GridCacheReturn<V>, Exception, IgniteFuture<GridNearLockResponse<K, V>>>() {
-                                    @Override public IgniteFuture<GridNearLockResponse<K, V>> apply(
+                                new C2<GridCacheReturn<V>, Exception, IgniteInternalFuture<GridNearLockResponse<K, V>>>() {
+                                    @Override public IgniteInternalFuture<GridNearLockResponse<K, V>> apply(
                                         GridCacheReturn<V> o, Exception e) {
                                         if (e != null)
                                             e = U.unwrap(e);
@@ -866,8 +867,8 @@ public abstract class GridDhtTransactionalCacheAdapter<K, V> extends GridDhtCach
                                             assert t.implicit();
 
                                             return t.commitAsync().chain(
-                                                new C1<IgniteFuture<IgniteTx>, GridNearLockResponse<K, V>>() {
-                                                    @Override public GridNearLockResponse<K, V> apply(IgniteFuture<IgniteTx> f) {
+                                                new C1<IgniteInternalFuture<IgniteTx>, GridNearLockResponse<K, V>>() {
+                                                    @Override public GridNearLockResponse<K, V> apply(IgniteInternalFuture<IgniteTx> f) {
                                                         try {
                                                             // Check for error.
                                                             f.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index 2672d30..b5c7927 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -122,8 +123,8 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
      */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<?>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(IgniteInternalFuture<?> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -134,7 +135,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        for (IgniteFuture<?> fut : futures())
+        for (IgniteInternalFuture<?> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -199,7 +200,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
      */
     public void onResult(UUID nodeId, GridDhtTxFinishResponse<K, V> res) {
         if (!isDone()) {
-            for (IgniteFuture<IgniteTx> fut : futures()) {
+            for (IgniteInternalFuture<IgniteTx> fut : futures()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -239,7 +240,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
index 38bc4ca..4d62ecf 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocal.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -246,7 +247,7 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
     }
 
     /** {@inheritDoc} */
-    @Override @Nullable protected IgniteFuture<Boolean> addReader(long msgId, GridDhtCacheEntry<K, V> cached,
+    @Override @Nullable protected IgniteInternalFuture<Boolean> addReader(long msgId, GridDhtCacheEntry<K, V> cached,
         IgniteTxEntry<K, V> entry, long topVer) {
         // Don't add local node as reader.
         if (!cctx.localNodeId().equals(nearNodeId)) {
@@ -276,7 +277,7 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<IgniteTxEx<K, V>> prepareAsync() {
+    @Override public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync() {
         if (optimistic()) {
             assert isSystemInvalidate();
 
@@ -348,7 +349,7 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
      * @param lastBackups IDs of backup nodes receiving last prepare request.
      * @return Future that will be completed when locks are acquired.
      */
-    public IgniteFuture<IgniteTxEx<K, V>> prepareAsync(@Nullable Iterable<IgniteTxEntry<K, V>> reads,
+    public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync(@Nullable Iterable<IgniteTxEntry<K, V>> reads,
         @Nullable Iterable<IgniteTxEntry<K, V>> writes,
         Map<IgniteTxKey<K>, GridCacheVersion> verMap,
         long msgId,
@@ -449,7 +450,7 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
 
     /** {@inheritDoc} */
     @SuppressWarnings({"ThrowableInstanceNeverThrown"})
-    @Override public IgniteFuture<IgniteTx> commitAsync() {
+    @Override public IgniteInternalFuture<IgniteTx> commitAsync() {
         if (log.isDebugEnabled())
             log.debug("Committing dht local tx: " + this);
 
@@ -486,8 +487,8 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
                 }
             }
             else
-                prep.listenAsync(new CI1<IgniteFuture<IgniteTxEx<K, V>>>() {
-                    @Override public void apply(IgniteFuture<IgniteTxEx<K, V>> f) {
+                prep.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
+                    @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
                         try {
                             f.get(); // Check for errors of a parent future.
 
@@ -544,7 +545,7 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<IgniteTx> rollbackAsync() {
+    @Override public IgniteInternalFuture<IgniteTx> rollbackAsync() {
         GridDhtTxPrepareFuture<K, V> prepFut = this.prepFut.get();
 
         final GridDhtTxFinishFuture<K, V> fut = new GridDhtTxFinishFuture<>(cctx, this, /*rollback*/false);
@@ -574,8 +575,8 @@ public class GridDhtTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> implements
         else {
             prepFut.complete();
 
-            prepFut.listenAsync(new CI1<IgniteFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteFuture<IgniteTxEx<K, V>> f) {
+            prepFut.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
                     try {
                         f.get(); // Check for errors of a parent future.
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
index d03c8d0..c3e8729 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxLocalAdapter.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -138,7 +139,7 @@ public abstract class GridDhtTxLocalAdapter<K, V> extends IgniteTxLocalAdapter<K
      * @param topVer Topology version.
      * @return {@code True} if reader was added as a result of this call.
      */
-    @Nullable protected abstract IgniteFuture<Boolean> addReader(long msgId,
+    @Nullable protected abstract IgniteInternalFuture<Boolean> addReader(long msgId,
         GridDhtCacheEntry<K, V> cached,
         IgniteTxEntry<K, V> entry,
         long topVer);
@@ -389,7 +390,7 @@ public abstract class GridDhtTxLocalAdapter<K, V> extends IgniteTxLocalAdapter<K
      * @return Future for active transactions for the time when reader was added.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgniteFuture<Boolean> addEntry(long msgId, IgniteTxEntry<K, V> e) throws IgniteCheckedException {
+    @Nullable public IgniteInternalFuture<Boolean> addEntry(long msgId, IgniteTxEntry<K, V> e) throws IgniteCheckedException {
         init();
 
         IgniteTxState state = state();
@@ -481,7 +482,7 @@ public abstract class GridDhtTxLocalAdapter<K, V> extends IgniteTxLocalAdapter<K
      * @param accessTtl TTL for read operation.
      * @return Lock future.
      */
-    IgniteFuture<GridCacheReturn<V>> lockAllAsync(
+    IgniteInternalFuture<GridCacheReturn<V>> lockAllAsync(
         GridCacheContext<K, V> cacheCtx,
         Collection<GridCacheEntryEx<K, V>> entries,
         List<IgniteTxEntry<K, V>> writeEntries,
@@ -609,7 +610,7 @@ public abstract class GridDhtTxLocalAdapter<K, V> extends IgniteTxLocalAdapter<K
      * @param filter Entry write filter.
      * @return Future for lock acquisition.
      */
-    private IgniteFuture<GridCacheReturn<V>> obtainLockAsync(
+    private IgniteInternalFuture<GridCacheReturn<V>> obtainLockAsync(
         final GridCacheContext<K, V> cacheCtx,
         GridCacheReturn<V> ret,
         final Collection<? extends K> passedKeys,
@@ -626,7 +627,7 @@ public abstract class GridDhtTxLocalAdapter<K, V> extends IgniteTxLocalAdapter<K
 
         GridDhtTransactionalCacheAdapter<K, V> dhtCache = cacheCtx.isNear() ? cacheCtx.nearTx().dht() : cacheCtx.dhtTx();
 
-        IgniteFuture<Boolean> fut = dhtCache.lockAllAsyncInternal(passedKeys,
+        IgniteInternalFuture<Boolean> fut = dhtCache.lockAllAsyncInternal(passedKeys,
             lockTimeout(),
             this,
             isInvalidate(),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
index cd4ab2b..947bec4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridDhtTxPrepareFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -175,8 +176,8 @@ public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFutu
      */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<?>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(IgniteInternalFuture<?> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -247,7 +248,7 @@ public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFutu
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        for (IgniteFuture<?> fut : futures())
+        for (IgniteInternalFuture<?> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -305,7 +306,7 @@ public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFutu
      */
     public void onResult(UUID nodeId, GridDhtTxPrepareResponse<K, V> res) {
         if (!isDone()) {
-            for (IgniteFuture<IgniteTxEx<K, V>> fut : pending()) {
+            for (IgniteInternalFuture<IgniteTxEx<K, V>> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -516,7 +517,7 @@ public final class GridDhtTxPrepareFuture<K, V> extends GridCompoundIdentityFutu
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 7227026..5c3d0ae 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -202,8 +203,8 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
     @SuppressWarnings("unchecked")
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<Map<K, V>>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<Map<K, V>> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<Map<K, V>>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(IgniteInternalFuture<Map<K, V>> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -214,7 +215,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        for (IgniteFuture<Map<K, V>> fut : futures())
+        for (IgniteInternalFuture<Map<K, V>> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -233,7 +234,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
      * @param res Result.
      */
     public void onResult(UUID nodeId, GridNearGetResponse<K, V> res) {
-        for (IgniteFuture<Map<K, V>> fut : futures())
+        for (IgniteInternalFuture<Map<K, V>> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -264,7 +265,7 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<Map<K, V>> f) {
+    private boolean isMini(IgniteInternalFuture<Map<K, V>> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -357,8 +358,8 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
                 }
 
                 // Add new future.
-                add(fut.chain(new C1<IgniteFuture<Collection<GridCacheEntryInfo<K, V>>>, Map<K, V>>() {
-                    @Override public Map<K, V> apply(IgniteFuture<Collection<GridCacheEntryInfo<K, V>>> fut) {
+                add(fut.chain(new C1<IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>>, Map<K, V>>() {
+                    @Override public Map<K, V> apply(IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>> fut) {
                         try {
                             return createResultMap(fut.get());
                         }
@@ -693,11 +694,11 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
                     log.debug("Remapping mini get future [invalidParts=" + invalidParts + ", fut=" + this + ']');
 
                 // Need to wait for next topology version to remap.
-                IgniteFuture<Long> topFut = ctx.discovery().topologyFuture(rmtTopVer);
+                IgniteInternalFuture<Long> topFut = ctx.discovery().topologyFuture(rmtTopVer);
 
-                topFut.listenAsync(new CIX1<IgniteFuture<Long>>() {
+                topFut.listenAsync(new CIX1<IgniteInternalFuture<Long>>() {
                     @SuppressWarnings("unchecked")
-                    @Override public void applyx(IgniteFuture<Long> fut) throws IgniteCheckedException {
+                    @Override public void applyx(IgniteInternalFuture<Long> fut) throws IgniteCheckedException {
                         long topVer = fut.get();
 
                         // This will append new futures to compound list.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index d2150de..3bdd976 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -255,7 +256,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(
+    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
         @Nullable final Collection<? extends K> keys,
         final boolean forcePrimary,
         boolean skipTx,
@@ -273,8 +274,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
         final ExpiryPolicy expiryPlc = prj != null ? prj.expiry() : null;
 
-        return asyncOp(new CO<IgniteFuture<Map<K, V>>>() {
-            @Override public IgniteFuture<Map<K, V>> apply() {
+        return asyncOp(new CO<IgniteInternalFuture<Map<K, V>>>() {
+            @Override public IgniteInternalFuture<Map<K, V>> apply() {
                 return getAllAsync0(keys,
                     false,
                     forcePrimary,
@@ -307,7 +308,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry,
         long ttl, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         A.notNull(key, "key");
 
@@ -324,7 +325,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> putxAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
+    @Override public IgniteInternalFuture<Boolean> putxAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         A.notNull(key, "key");
 
@@ -345,7 +346,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> putIfAbsentAsync(K key, V val) {
         A.notNull(key, "key", val, "val");
 
         return putAsync(key, val, ctx.noPeekArray());
@@ -357,7 +358,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> putxIfAbsentAsync(K key, V val) {
         A.notNull(key, "key", val, "val");
 
         return putxAsync(key, val, ctx.noPeekArray());
@@ -369,7 +370,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(K key, V val) {
         A.notNull(key, "key", val, "val");
 
         return putAsync(key, val, ctx.hasPeekArray());
@@ -381,7 +382,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(K key, V val) {
         A.notNull(key, "key", val, "val");
 
         return putxAsync(key, val, ctx.hasPeekArray());
@@ -393,7 +394,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
         A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
 
         return putxAsync(key, newVal, ctx.equalsPeekArray(oldVal));
@@ -411,7 +412,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
         A.notNull(key, "key", val, "val");
 
         return removeAllAsync0(F.asList(key), null, null, true, true, ctx.equalsPeekArray(val));
@@ -419,7 +420,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
         return updateAllAsync0(F.asMap(key, newVal),
             null,
             null,
@@ -438,7 +439,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
+    @Override public IgniteInternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return updateAllAsync0(m,
             null,
@@ -457,7 +458,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap) {
+    @Override public IgniteInternalFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap) {
         ctx.dr().onReceiveCacheEntriesReceived(drMap.size());
 
         return updateAllAsync0(null,
@@ -479,7 +480,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         A.notNull(key, "key");
 
@@ -493,7 +494,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<?> removeAllAsync(Collection<? extends K> keys,
         IgnitePredicate<CacheEntry<K, V>>[] filter) {
         A.notNull(keys, "keys");
 
@@ -508,7 +509,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         A.notNull(key, "key");
 
@@ -521,7 +522,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
         A.notNull(key, "key", val, "val");
 
         return removexAsync(key, ctx.equalsPeekArray(val));
@@ -533,7 +534,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<?> removeAllAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return removeAllAsync(keySet(filter), filter);
     }
 
@@ -543,7 +544,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) {
+    @Override public IgniteInternalFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) {
         ctx.dr().onReceiveCacheEntriesReceived(drMap.size());
 
         return removeAllAsync0(null, drMap, null, false, false, null);
@@ -561,8 +562,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    protected <T> IgniteFuture<T> asyncOp(final CO<IgniteFuture<T>> op) {
-        IgniteFuture<T> fail = asyncOpAcquire();
+    protected <T> IgniteInternalFuture<T> asyncOp(final CO<IgniteInternalFuture<T>> op) {
+        IgniteInternalFuture<T> fail = asyncOpAcquire();
 
         if (fail != null)
             return fail;
@@ -572,12 +573,12 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         holder.lock();
 
         try {
-            IgniteFuture fut = holder.future();
+            IgniteInternalFuture fut = holder.future();
 
             if (fut != null && !fut.isDone()) {
-                IgniteFuture<T> f = new GridEmbeddedFuture<>(fut,
-                    new C2<T, Exception, IgniteFuture<T>>() {
-                        @Override public IgniteFuture<T> apply(T t, Exception e) {
+                IgniteInternalFuture<T> f = new GridEmbeddedFuture<>(fut,
+                    new C2<T, Exception, IgniteInternalFuture<T>>() {
+                        @Override public IgniteInternalFuture<T> apply(T t, Exception e) {
                             return op.apply();
                         }
                     }, ctx.kernalContext());
@@ -587,7 +588,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                 return f;
             }
 
-            IgniteFuture<T> f = op.apply();
+            IgniteInternalFuture<T> f = op.apply();
 
             saveFuture(holder, f);
 
@@ -599,7 +600,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
+    @Override protected IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
         long timeout,
         @Nullable IgniteTxLocalEx<K, V> tx,
         boolean isInvalidate,
@@ -628,7 +629,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <T> IgniteFuture<EntryProcessorResult<T>> invokeAsync(K key,
+    @Override public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
         A.notNull(key, "key", entryProcessor, "entryProcessor");
@@ -641,7 +642,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         Map<? extends K, EntryProcessor> invokeMap =
             Collections.singletonMap(key, (EntryProcessor)entryProcessor);
 
-        IgniteFuture<Map<K, EntryProcessorResult<T>>> fut = updateAllAsync0(null,
+        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut = updateAllAsync0(null,
             invokeMap,
             args,
             null,
@@ -651,8 +652,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             null,
             null);
 
-        return fut.chain(new CX1<IgniteFuture<Map<K, EntryProcessorResult<T>>>, EntryProcessorResult<T>>() {
-            @Override public EntryProcessorResult<T> applyx(IgniteFuture<Map<K, EntryProcessorResult<T>>> fut)
+        return fut.chain(new CX1<IgniteInternalFuture<Map<K, EntryProcessorResult<T>>>, EntryProcessorResult<T>>() {
+            @Override public EntryProcessorResult<T> applyx(IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut)
                 throws IgniteCheckedException {
                 Map<K, EntryProcessorResult<T>> resMap = fut.get();
 
@@ -669,7 +670,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
         final EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
         A.notNull(keys, "keys", entryProcessor, "entryProcessor");
@@ -704,7 +705,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args) {
         A.notNull(map, "map");
@@ -739,7 +740,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param filter Cache entry filter for atomic updates.
      * @return Completion future.
      */
-    private IgniteFuture updateAllAsync0(
+    private IgniteInternalFuture updateAllAsync0(
         @Nullable final Map<? extends K, ? extends V> map,
         @Nullable final Map<? extends K, ? extends EntryProcessor> invokeMap,
         @Nullable Object[] invokeArgs,
@@ -780,8 +781,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
             subjId,
             taskNameHash);
 
-        return asyncOp(new CO<IgniteFuture<Object>>() {
-            @Override public IgniteFuture<Object> apply() {
+        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+            @Override public IgniteInternalFuture<Object> apply() {
                 updateFut.map();
 
                 return updateFut;
@@ -800,7 +801,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param filter Cache entry filter for atomic removes.
      * @return Completion future.
      */
-    private IgniteFuture removeAllAsync0(
+    private IgniteInternalFuture removeAllAsync0(
         @Nullable final Collection<? extends K> keys,
         @Nullable final Map<? extends K, GridCacheVersion> drMap,
         @Nullable GridCacheEntryEx<K, V> cached,
@@ -846,8 +847,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         if (statsEnabled)
             updateFut.listenAsync(new UpdateRemoveTimeStatClosure<>(metrics0(), start));
 
-        return asyncOp(new CO<IgniteFuture<Object>>() {
-            @Override public IgniteFuture<Object> apply() {
+        return asyncOp(new CO<IgniteInternalFuture<Object>>() {
+            @Override public IgniteInternalFuture<Object> apply() {
                 updateFut.map();
 
                 return updateFut;
@@ -868,7 +869,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
      * @param expiryPlc Expiry policy.
      * @return Get future.
      */
-    private IgniteFuture<Map<K, V>> getAllAsync0(@Nullable Collection<? extends K> keys,
+    private IgniteInternalFuture<Map<K, V>> getAllAsync0(@Nullable Collection<? extends K> keys,
         boolean reload,
         boolean forcePrimary,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter,
@@ -1015,13 +1016,13 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
         @Nullable final GridCacheEntryEx<K, V> cached,
         final CI2<GridNearAtomicUpdateRequest<K, V>, GridNearAtomicUpdateResponse<K, V>> completionCb
     ) {
-        IgniteFuture<Object> forceFut = preldr.request(req.keys(), req.topologyVersion());
+        IgniteInternalFuture<Object> forceFut = preldr.request(req.keys(), req.topologyVersion());
 
         if (forceFut.isDone())
             updateAllAsyncInternal0(nodeId, req, completionCb);
         else {
-            forceFut.listenAsync(new CI1<IgniteFuture<Object>>() {
-                @Override public void apply(IgniteFuture<Object> t) {
+            forceFut.listenAsync(new CI1<IgniteInternalFuture<Object>>() {
+                @Override public void apply(IgniteInternalFuture<Object> t) {
                     updateAllAsyncInternal0(nodeId, req, completionCb);
                 }
             });
@@ -1776,7 +1777,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                 res.addNearTtl(i, ttl, expireTime);
 
                             if (updRes.newValue() != null || newValBytes != null) {
-                                IgniteFuture<Boolean> f = entry.addReader(node.id(), req.messageId(), topVer);
+                                IgniteInternalFuture<Boolean> f = entry.addReader(node.id(), req.messageId(), topVer);
 
                                 assert f == null : f;
                             }
@@ -2049,7 +2050,7 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
                                     res.addNearTtl(idx, updRes.newTtl(), -1);
 
                                 if (writeVal != null || !entry.valueBytes().isNull()) {
-                                    IgniteFuture<Boolean> f = entry.addReader(node.id(), req.messageId(), topVer);
+                                    IgniteInternalFuture<Boolean> f = entry.addReader(node.id(), req.messageId(), topVer);
 
                                     assert f == null : f;
                                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 539a462..5cd8d54 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.atomic;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.lang.*;
@@ -440,8 +441,8 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
                 snapshot = fut.topologySnapshot();
             }
             else {
-                fut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                    @Override public void apply(IgniteFuture<Long> t) {
+                fut.listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                    @Override public void apply(IgniteInternalFuture<Long> t) {
                         mapOnTopology(keys, remap, oldNodeId);
                     }
                 });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index 668f6fe..7811713 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.colocated;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -152,7 +153,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(
+    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
         @Nullable final Collection<? extends K> keys,
         boolean forcePrimary,
         boolean skipTx,
@@ -172,7 +173,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
 
         if (tx != null && !tx.implicit() && !skipTx) {
             return asyncOp(tx, new AsyncOp<Map<K, V>>(keys) {
-                @Override public IgniteFuture<Map<K, V>> op(IgniteTxLocalAdapter<K, V> tx) {
+                @Override public IgniteInternalFuture<Map<K, V>> op(IgniteTxLocalAdapter<K, V> tx) {
                     return ctx.wrapCloneMap(tx.getAllAsync(ctx, keys, entry, deserializePortable, filter));
                 }
             });
@@ -238,7 +239,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
      * @param expiryPlc Expiry policy.
      * @return Loaded values.
      */
-    public IgniteFuture<Map<K, V>> loadAsync(@Nullable Collection<? extends K> keys,
+    public IgniteInternalFuture<Map<K, V>> loadAsync(@Nullable Collection<? extends K> keys,
         boolean readThrough,
         boolean reload,
         boolean forcePrimary,
@@ -373,7 +374,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
      *
      * {@inheritDoc}
      */
-    @Override public IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
         long timeout,
         @Nullable IgniteTxLocalEx<K, V> tx,
         boolean isInvalidate,
@@ -606,7 +607,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
      * @param filter filter Optional filter.
      * @return Lock future.
      */
-    IgniteFuture<Exception> lockAllAsync(
+    IgniteInternalFuture<Exception> lockAllAsync(
         final GridCacheContext<K, V> cacheCtx,
         @Nullable final GridNearTxLocal<K, V> tx,
         final long threadId,
@@ -620,7 +621,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
     ) {
         assert keys != null;
 
-        IgniteFuture<Object> keyFut = ctx.dht().dhtPreloader().request(keys, topVer);
+        IgniteInternalFuture<Object> keyFut = ctx.dht().dhtPreloader().request(keys, topVer);
 
         // Prevent embedded future creation if possible.
         if (keyFut.isDone()) {
@@ -645,8 +646,8 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
         }
         else {
             return new GridEmbeddedFuture<>(true, keyFut,
-                new C2<Object, Exception, IgniteFuture<Exception>>() {
-                    @Override public IgniteFuture<Exception> apply(Object o, Exception exx) {
+                new C2<Object, Exception, IgniteInternalFuture<Exception>>() {
+                    @Override public IgniteInternalFuture<Exception> apply(Object o, Exception exx) {
                         if (exx != null)
                             return new GridDhtFinishedFuture<>(ctx.kernalContext(), exx);
 
@@ -679,7 +680,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
      * @param filter filter Optional filter.
      * @return Lock future.
      */
-    private IgniteFuture<Exception> lockAllAsync0(
+    private IgniteInternalFuture<Exception> lockAllAsync0(
         GridCacheContext<K, V> cacheCtx,
         @Nullable final GridNearTxLocal<K, V> tx,
         long threadId,
@@ -766,7 +767,7 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
             if (log.isDebugEnabled())
                 log.debug("Performing colocated lock [tx=" + tx + ", keys=" + keys + ']');
 
-            IgniteFuture<GridCacheReturn<V>> txFut = tx.lockAllAsync(cacheCtx,
+            IgniteInternalFuture<GridCacheReturn<V>> txFut = tx.lockAllAsync(cacheCtx,
                 keys,
                 tx.implicit(),
                 txRead,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
index c88311c..e0edcad 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedLockFuture.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.colocated;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -173,8 +174,8 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
      * @return Participating nodes.
      */
     @Override public Collection<? extends ClusterNode> nodes() {
-        return F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-            @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+        return F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<?>, ClusterNode>() {
+            @Nullable @Override public ClusterNode apply(IgniteInternalFuture<?> f) {
                 if (isMini(f))
                     return ((MiniFuture)f).node();
 
@@ -371,7 +372,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
     @Override public boolean onNodeLeft(UUID nodeId) {
         boolean found = false;
 
-        for (IgniteFuture<?> fut : futures()) {
+        for (IgniteInternalFuture<?> fut : futures()) {
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -404,7 +405,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                 log.debug("Received lock response from node [nodeId=" + nodeId + ", res=" + res + ", fut=" +
                     this + ']');
 
-            for (IgniteFuture<Boolean> fut : pending()) {
+            for (IgniteInternalFuture<Boolean> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture mini = (MiniFuture)fut;
 
@@ -536,7 +537,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -596,8 +597,8 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                     markInitialized();
                 }
                 else {
-                    fut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                        @Override public void apply(IgniteFuture<Long> t) {
+                    fut.listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                        @Override public void apply(IgniteInternalFuture<Long> t) {
                             mapOnTopology();
                         }
                     });
@@ -859,7 +860,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
 
             add(fut); // Append new future.
 
-            IgniteFuture<?> txSync = null;
+            IgniteInternalFuture<?> txSync = null;
 
             if (inTx())
                 txSync = cctx.tm().awaitFinishAckAsync(node.id(), tx.threadId());
@@ -878,8 +879,8 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
                 }
             }
             else {
-                txSync.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> t) {
+                txSync.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> t) {
                         try {
                             if (log.isDebugEnabled())
                                 log.debug("Sending near lock request [node=" + node.id() + ", req=" + req + ']');
@@ -912,7 +913,7 @@ public final class GridDhtColocatedLockFuture<K, V> extends GridCompoundIdentity
         if (log.isDebugEnabled())
             log.debug("Before locally locking keys : " + keys);
 
-        IgniteFuture<Exception> fut = cctx.colocated().lockAllAsync(cctx,
+        IgniteInternalFuture<Exception> fut = cctx.colocated().lockAllAsync(cctx,
             tx,
             threadId,
             lockVer,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
index 67a5905..16a56e2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtForceKeysFuture.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.preloader;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
@@ -131,7 +132,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -156,7 +157,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
 
         int type = evt.type();
 
-        for (IgniteFuture<?> f : futures()) {
+        for (IgniteInternalFuture<?> f : futures()) {
             if (isMini(f)) {
                 MiniFuture mini = (MiniFuture)f;
 
@@ -180,7 +181,7 @@ public final class GridDhtForceKeysFuture<K, V> extends GridCompoundFuture<Objec
      */
     @SuppressWarnings( {"unchecked"})
     public void onResult(UUID nodeId, GridDhtForceKeysResponse<K, V> res) {
-        for (IgniteFuture<Object> f : futures())
+        for (IgniteInternalFuture<Object> f : futures())
             if (isMini(f)) {
                 MiniFuture mini = (MiniFuture)f;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
index 9c7652c..8a6fef8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionDemandPool.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.preloader;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
@@ -161,7 +162,7 @@ public class GridDhtPartitionDemandPool<K, V> {
     /**
      * @return Future for {@link org.apache.ignite.cache.CachePreloadMode#SYNC} mode.
      */
-    IgniteFuture<?> syncFuture() {
+    IgniteInternalFuture<?> syncFuture() {
         return syncFut;
     }
 
@@ -206,8 +207,8 @@ public class GridDhtPartitionDemandPool<K, V> {
             if (log.isDebugEnabled())
                 log.debug("Forcing preload event for future: " + exchFut);
 
-            exchFut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                @Override public void apply(IgniteFuture<Long> t) {
+            exchFut.listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                @Override public void apply(IgniteInternalFuture<Long> t) {
                     cctx.shared().exchange().forcePreloadExchange(exchFut);
                 }
             });
@@ -355,8 +356,8 @@ public class GridDhtPartitionDemandPool<K, V> {
 
             obj = new GridTimeoutObjectAdapter(delay) {
                 @Override public void onTimeout() {
-                    exchFut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                        @Override public void apply(IgniteFuture<Long> f) {
+                    exchFut.listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                        @Override public void apply(IgniteInternalFuture<Long> f) {
                             cctx.shared().exchange().forcePreloadExchange(exchFut);
                         }
                     });
@@ -810,7 +811,7 @@ public class GridDhtPartitionDemandPool<K, V> {
                 int preloadOrder = cctx.config().getPreloadOrder();
 
                 if (preloadOrder > 0) {
-                    IgniteFuture<?> fut = cctx.kernalContext().cache().orderedPreloadFuture(preloadOrder);
+                    IgniteInternalFuture<?> fut = cctx.kernalContext().cache().orderedPreloadFuture(preloadOrder);
 
                     try {
                         if (fut != null) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
index 0ebd0df..219737f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPartitionsExchangeFuture.java
@@ -21,10 +21,10 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.timeout.*;
@@ -130,7 +130,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
     /** */
     @SuppressWarnings({"FieldCanBeLocal", "UnusedDeclaration"})
     @GridToStringInclude
-    private volatile IgniteFuture<?> partReleaseFut;
+    private volatile IgniteInternalFuture<?> partReleaseFut;
 
     /** */
     private final Object mux = new Object();
@@ -377,7 +377,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
     /**
      * @return Init future.
      */
-    IgniteFuture<?> initFuture() {
+    IgniteInternalFuture<?> initFuture() {
         return initFut;
     }
 
@@ -453,7 +453,7 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
                     cacheCtx.preloader().updateLastExchangeFuture(this);
                 }
 
-                IgniteFuture<?> partReleaseFut = cctx.partitionReleaseFuture(topVer);
+                IgniteInternalFuture<?> partReleaseFut = cctx.partitionReleaseFuture(topVer);
 
                 // Assign to class variable so it will be included into toString() method.
                 this.partReleaseFut = partReleaseFut;
@@ -728,8 +728,8 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
             }
         }
         else {
-            initFut.listenAsync(new CI1<IgniteFuture<Boolean>>() {
-                @Override public void apply(IgniteFuture<Boolean> t) {
+            initFut.listenAsync(new CI1<IgniteInternalFuture<Boolean>>() {
+                @Override public void apply(IgniteInternalFuture<Boolean> t) {
                     try {
                         if (!t.get()) // Just to check if there was an error.
                             return;
@@ -826,8 +826,8 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
 
         assert exchId.topologyVersion() == msg.topologyVersion();
 
-        initFut.listenAsync(new CI1<IgniteFuture<Boolean>>() {
-            @Override public void apply(IgniteFuture<Boolean> t) {
+        initFut.listenAsync(new CI1<IgniteInternalFuture<Boolean>>() {
+            @Override public void apply(IgniteInternalFuture<Boolean> t) {
                 assert msg.lastVersion() != null;
 
                 cctx.versions().onReceived(nodeId, msg.lastVersion());
@@ -886,8 +886,8 @@ public class GridDhtPartitionsExchangeFuture<K, V> extends GridFutureAdapter<Lon
 
         try {
             // Wait for initialization part of this future to complete.
-            initFut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> f) {
+            initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> f) {
                     if (isDone())
                         return;
 


[15/50] [abbrv] incubator-ignite git commit: # sprint-1 Removed GridGain from readme files.

Posted by vo...@apache.org.
# sprint-1 Removed GridGain from readme files.


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

Branch: refs/heads/ignite-16
Commit: 029e96b28647ae9b4b31e4afb941d6959c180d99
Parents: 673d11b
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Jan 29 05:34:01 2015 +0000
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Jan 29 05:34:01 2015 +0000

----------------------------------------------------------------------
 assembly/libs/readme.txt                   | 111 +++++++++---------------
 examples/config/filesystem/readme.txt      |   4 +-
 examples/config/hibernate/readme.txt       |   4 +-
 examples/config/servlet/readme.txt         |   2 +-
 ipc/readme.txt                             |   4 +-
 ipc/shmem/readme.txt                       |  10 +--
 modules/aop/readme.txt                     |  12 +--
 modules/aws/readme.txt                     |  12 +--
 modules/core/src/test/resources/readme.txt |   2 +-
 modules/email/readme.txt                   |  12 +--
 modules/hadoop/readme.txt                  |  12 +--
 modules/hibernate/readme.txt               |  14 +--
 modules/indexing/readme.txt                |  12 +--
 modules/jcl/readme.txt                     |  12 +--
 modules/jta/readme.txt                     |  12 +--
 modules/log4j/readme.txt                   |  12 +--
 modules/rest-http/readme.txt               |  12 +--
 modules/scalar/readme.txt                  |   8 +-
 modules/schedule/readme.txt                |  12 +--
 modules/slf4j/readme.txt                   |  12 +--
 modules/spring/readme.txt                  |  12 +--
 modules/ssh/readme.txt                     |  12 +--
 modules/urideploy/readme.txt               |  12 +--
 modules/web/readme.txt                     |  16 ++--
 24 files changed, 157 insertions(+), 186 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/assembly/libs/readme.txt
----------------------------------------------------------------------
diff --git a/assembly/libs/readme.txt b/assembly/libs/readme.txt
index ef6aed2..7595da9 100644
--- a/assembly/libs/readme.txt
+++ b/assembly/libs/readme.txt
@@ -1,14 +1,14 @@
-GridGain Dependencies
----------------------
+Apache Ignite Dependencies
+--------------------------
 
-Current folder contains JAR files for all GridGain modules along with their dependencies.
+Current folder contains JAR files for all Apache Ignite modules along with their dependencies.
 When node is started using 'ggstart.{sh|bat}' script, all JARs and classes located in
 'libs' folder and all its sub-folders except 'optional' are added to classpath of the node.
 
-By default, only GridGain core JAR and a minimum set of modules is enabled, while other
+By default, only Apache Ignite core JAR and a minimum set of modules is enabled, while other
 modules are located in 'optional' folder and therefore disabled.
 
-To enable any of optional GridGain modules when starting a standalone node,
+To enable any of optional Ignite modules when starting a standalone node,
 move corresponding module folder from 'libs/optional' to 'libs' before running
 'ggstart.{sh|bat}' script. The content of the module folder will be added to
 classpath in this case.
@@ -17,43 +17,15 @@ If you need to add your own classes to classpath of the node (e.g., task classes
 to 'libs' folder. You can create a subfolder for convenience as well.
 
 
-Importing GridGain Dependencies In Maven Project
+Importing Ignite Dependencies In Maven Project
 ------------------------------------------------
 
 If you are using Maven to manage dependencies of your project, there are two options:
 
-1. Import one of the following predefined GridGain editions:
-  - gridgain-hpc
-  - gridgain-datagrid
-  - gridgain-streaming
-  - gridgain-fabric (all inclusive)
+1. Import fabric edition:
+  - ignite-fabric (all inclusive)
 
-2. Or import individual GridGain modules a la carte.
-
-
-Importing GridGain Edition Dependencies
----------------------------------------
-
-When importing editions, each edition automatically imports GridGain core module and
-a set of additional modules needed for this edition to work. Specifically:
-
-- gridgain-hpc
-  - gridgain-core
-  - gridgain-spring (optional, add if you plan to use Spring configuration)
-
-- gridgain-datagrid
-  - gridgain-core
-  - gridgain-indexing (optional, add if you need SQL indexing)
-  - gridgain-spring (optional, add if you plan to use Spring configuration)
-
-- gridgain-streaming
-  - gridgain-core
-  - gridgain-spring (optional, add if you plan to use Spring configuration)
-
-- gridgain-fabric
-  - (all dependencies included in other editions)
-
-Here is how 'gridgain-datagrid' can be added to your POM file (replace '${ignite.version}'
+Here is how 'ignite-fabric' can be added to your POM file (replace '${ignite.version}'
 with actual Ignite version you are interested in):
 
 <project xmlns="http://maven.apache.org/POM/4.0.0"
@@ -64,8 +36,8 @@ with actual Ignite version you are interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-datagrid</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-fabric</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...
@@ -73,12 +45,11 @@ with actual Ignite version you are interested in):
     ...
 </project>
 
+2. Or import individual Apache Ignite modules a la carte.
 
-Importing Individual Modules A La Carte
----------------------------------------
 
-Alternatively you can import GridGain modules a la carte, one by one.
-The only required module is 'gridgain-core', all others are optional.
+Alternatively you can import Ignite modules a la carte, one by one.
+The only required module is 'ignite-core', all others are optional.
 Here is how it can be imported into your POM file:
 
 <project xmlns="http://maven.apache.org/POM/4.0.0"
@@ -89,8 +60,8 @@ Here is how it can be imported into your POM file:
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-core</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...
@@ -101,27 +72,27 @@ Here is how it can be imported into your POM file:
 All optional modules can be imported just like the core module, but with different artifact IDs.
 
 The following modules are available:
-- gridgain-spring (for Spring-based configuration support)
-- gridgain-indexing (for SQL querying and indexing)
-- gridgain-hibernate (for Hibernate integration)
-- gridgain-web (for Web Sessions Clustering)
-- gridgain-schedule (for Cron-based task scheduling)
-- gridgain-logj4 (for Log4j logging)
-- gridgain-jcl (for Apache Commons logging)
-- gridgain-jta (for XA integration)
-- gridgain-hadoop (for Apache Hadoop Accelerator)
-- gridgain-rest-http (for HTTP REST messages)
-- gridgain-scalar (for GridGain Scala API)
-- gridgain-sl4j (for SL4J logging)
-- gridgain-ssh (for starting grid nodes on remote machines)
-- gridgain-urideploy (for URI-based deployment)
-- gridgain-aws (for seemless cluster discovery on AWS S3)
-- gridgain-email (for email alerts)
-- gridgain-aop (for AOP-based grid-enabling)
-- gridgain-visor-console (open source command line management and monitoring tool)
-
-For example, if you want to use GridGain Spring-based configuration,
-you should add 'gridgain-spring' module like this:
+- ignite-spring (for Spring-based configuration support)
+- ignite-indexing (for SQL querying and indexing)
+- ignite-hibernate (for Hibernate integration)
+- ignite-web (for Web Sessions Clustering)
+- ignite-schedule (for Cron-based task scheduling)
+- ignite-logj4 (for Log4j logging)
+- ignite-jcl (for Apache Commons logging)
+- ignite-jta (for XA integration)
+- ignite-hadoop (for Apache Hadoop Accelerator)
+- ignite-rest-http (for HTTP REST messages)
+- ignite-scalar (for ignite Scala API)
+- ignite-sl4j (for SL4J logging)
+- ignite-ssh (for starting grid nodes on remote machines)
+- ignite-urideploy (for URI-based deployment)
+- ignite-aws (for seemless cluster discovery on AWS S3)
+- ignite-email (for email alerts)
+- ignite-aop (for AOP-based grid-enabling)
+- ignite-visor-console (open source command line management and monitoring tool)
+
+For example, if you want to use Apache Ignite Spring-based configuration,
+you should add 'ignite-spring' module like this:
 
 <project xmlns="http://maven.apache.org/POM/4.0.0"
     xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
@@ -132,15 +103,15 @@ you should add 'gridgain-spring' module like this:
         ...
         <!-- Core module. -->
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-core</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-core</artifactId>
             <version>${ignite.version}</version>
         </dependency>
 
         <!-- Optional. -->
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-spring</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/examples/config/filesystem/readme.txt
----------------------------------------------------------------------
diff --git a/examples/config/filesystem/readme.txt b/examples/config/filesystem/readme.txt
index e4e74ec..c69b8fd 100644
--- a/examples/config/filesystem/readme.txt
+++ b/examples/config/filesystem/readme.txt
@@ -2,7 +2,7 @@ FileSystem Configuration Example
 --------------------------------
 
 This folder contains configuration files for GGFS examples located in
-org.gridgain.examples.ggfs package.
+org.apache.ignite.examples.ggfs package.
 
-- example-ggfs.xml file is used to start GridGain nodes with GGFS configured
+- example-ggfs.xml file is used to start Apache Ignite nodes with GGFS configured
 - core-site.xml file is used to run Hadoop FS driver over GGFS

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/examples/config/hibernate/readme.txt
----------------------------------------------------------------------
diff --git a/examples/config/hibernate/readme.txt b/examples/config/hibernate/readme.txt
index 216a6ec..5b7ab29 100644
--- a/examples/config/hibernate/readme.txt
+++ b/examples/config/hibernate/readme.txt
@@ -2,7 +2,7 @@ Hibernate L2 Cache Configuration Example
 ----------------------------------------
 
 This folder contains example-hibernate-L2-cache.xml file that demonstrates
-how to configure Hibernate to use GridGain cache as an L2 cache provider.
+how to configure Hibernate to use Apache Ignite cache as an L2 cache provider.
 
-This file is also used in Hibernate example located in org.gridgain.examples.datagrid.hibernate
+This file is also used in Hibernate example located in org.apache.ignite.examples.datagrid.hibernate
 package.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/examples/config/servlet/readme.txt
----------------------------------------------------------------------
diff --git a/examples/config/servlet/readme.txt b/examples/config/servlet/readme.txt
index ed8785e..9bd1538 100644
--- a/examples/config/servlet/readme.txt
+++ b/examples/config/servlet/readme.txt
@@ -2,7 +2,7 @@ Servlet Configuration Example
 -----------------------------
 
 This folder contains web.xml file that demonstrates how to configure any servlet container
-to start a GridGain node inside a Web application.
+to start a Apache Ignite node inside a Web application.
 
 For more information on available configuration properties, etc. refer to our documentation:
 http://doc.gridgain.org/latest/Web+Sessions+Caching

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/ipc/readme.txt
----------------------------------------------------------------------
diff --git a/ipc/readme.txt b/ipc/readme.txt
index dadece2..3adc0dc 100644
--- a/ipc/readme.txt
+++ b/ipc/readme.txt
@@ -1,4 +1,4 @@
-GridGain IPC
+Apache Ignite IPC
 -------------------
 
-Contains different Inter-process communication (IPC) implementations for GridGain.
+Contains different Inter-process communication (IPC) implementations for Apache Ignite.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/ipc/shmem/readme.txt
----------------------------------------------------------------------
diff --git a/ipc/shmem/readme.txt b/ipc/shmem/readme.txt
index b0abe37..a21cbf8 100644
--- a/ipc/shmem/readme.txt
+++ b/ipc/shmem/readme.txt
@@ -1,7 +1,7 @@
-GridGain Shared memory IPC library
--------------------
+Apache Ignite Shared memory IPC library
+---------------------------------------
 
-GridGain shared memory IPC library implement exchange via shared memory for GridGain.
+Apache Ignite shared memory IPC library implement exchange via shared memory for Apache Ignite.
 
 Building on Linux and Mac OS X
 -------------------
@@ -22,9 +22,9 @@ to see all available command line options. Once the configure script finishes, y
 This will build and install the shared memory library and the headers into the default location on your
 system (which is usually '/usr/local').
 
-Usage with GridGain
+Usage with Apache Ignite
 -------------------
 
 Copy compiled library to folder that already listed in 'java.library.path'
-with name in form: 'libggshmem-<gridgain-version>.<extention>'.
+with name in form: 'libggshmem-<ignite-version>.<extention>'.
 Note: Grid should be restarted.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/aop/readme.txt
----------------------------------------------------------------------
diff --git a/modules/aop/readme.txt b/modules/aop/readme.txt
index b72ad81..75c5f49 100644
--- a/modules/aop/readme.txt
+++ b/modules/aop/readme.txt
@@ -1,10 +1,10 @@
-GridGain AOP Module
--------------------
+Apache Ignite AOP Module
+------------------------
 
-GridGain AOP module provides capability to turn any Java method to a distributed closure by
+Apache Ignite AOP module provides capability to turn any Java method to a distributed closure by
 adding @Gridify annotation to it.
 
-To enable AOP module when starting a standalone node, move 'optional/gridgain-aop' folder to
+To enable AOP module when starting a standalone node, move 'optional/ignite-aop' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -23,8 +23,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-aop</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-aop</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/aws/readme.txt
----------------------------------------------------------------------
diff --git a/modules/aws/readme.txt b/modules/aws/readme.txt
index faa80a4..317369c 100644
--- a/modules/aws/readme.txt
+++ b/modules/aws/readme.txt
@@ -1,9 +1,9 @@
-GridGain AWS Module
--------------------
+Apache Ignite AWS Module
+------------------------
 
-GridGain AWS module provides S3-based implementations of checkpoint SPI and IP finder for TCP discovery.
+Apache Ignite AWS module provides S3-based implementations of checkpoint SPI and IP finder for TCP discovery.
 
-To enable AWS module when starting a standalone node, move 'optional/gridgain-aws' folder to
+To enable AWS module when starting a standalone node, move 'optional/ignite-aws' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -22,8 +22,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-aws</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-aws</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/core/src/test/resources/readme.txt
----------------------------------------------------------------------
diff --git a/modules/core/src/test/resources/readme.txt b/modules/core/src/test/resources/readme.txt
index c8664e0..195d655 100644
--- a/modules/core/src/test/resources/readme.txt
+++ b/modules/core/src/test/resources/readme.txt
@@ -1,5 +1,5 @@
 This folder is created for test GridTaskUriDeploymentDeadlockSelfTest. It contains helloworld.gar and helloworld1.gar
-which are the same and were copied from GridGain GAR example.
+which are the same and were copied from Apache Ignite GAR example.
 
 We put two files here to have a collision and make deployment SPI to unregister class loaders.
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/email/readme.txt
----------------------------------------------------------------------
diff --git a/modules/email/readme.txt b/modules/email/readme.txt
index 789ebd6..9fd9108 100644
--- a/modules/email/readme.txt
+++ b/modules/email/readme.txt
@@ -1,11 +1,11 @@
-GridGain Email Module
----------------------
+Apache Ignite Email Module
+--------------------------
 
-GridGain email module enables GridGain to send emails in critical situations such as license
+Apache Ignite email module enables Apache Ignite to send emails in critical situations such as license
 expiration or fatal system errors (this should be also configured via 'GridConfiguration.setSmtpXXX(..)'
 configuration properties).
 
-To enable email module when starting a standalone node, move 'optional/gridgain-email' folder to
+To enable email module when starting a standalone node, move 'optional/ignite-email' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -24,8 +24,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-email</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-email</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/hadoop/readme.txt
----------------------------------------------------------------------
diff --git a/modules/hadoop/readme.txt b/modules/hadoop/readme.txt
index 5fcc040..0212371 100644
--- a/modules/hadoop/readme.txt
+++ b/modules/hadoop/readme.txt
@@ -1,10 +1,10 @@
-GridGain Hadoop Module
-----------------------
+Apache Ignite Hadoop Module
+---------------------------
 
-GridGain Hadoop module provides In-Memory MapReduce engine and driver to use GGFS as Hadoop file system
+Apache Ignite Hadoop module provides In-Memory MapReduce engine and driver to use GGFS as Hadoop file system
 which are 100% compatible with HDFS and YARN.
 
-To enable Hadoop module when starting a standalone node, move 'optional/gridgain-hadoop' folder to
+To enable Hadoop module when starting a standalone node, move 'optional/ignite-hadoop' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -23,8 +23,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-hadoop</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-hadoop</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/hibernate/readme.txt
----------------------------------------------------------------------
diff --git a/modules/hibernate/readme.txt b/modules/hibernate/readme.txt
index e1a913a..7b4b70a 100644
--- a/modules/hibernate/readme.txt
+++ b/modules/hibernate/readme.txt
@@ -1,10 +1,10 @@
-GridGain Hibernate Module
--------------------------
+Apache Ignite Hibernate Module
+------------------------------
 
-GridGain Hibernate module provides Hibernate second-level cache (L2 cache) implementation based
-on GridGain In-Memory Data Grid.
+Apache Ignite Hibernate module provides Hibernate second-level cache (L2 cache) implementation based
+on Apache Ignite In-Memory Data Grid.
 
-To enable Hibernate module when starting a standalone node, move 'optional/gridgain-hibernate' folder to
+To enable Hibernate module when starting a standalone node, move 'optional/ignite-hibernate' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -23,8 +23,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-hibernate</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-hibernate</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/indexing/readme.txt
----------------------------------------------------------------------
diff --git a/modules/indexing/readme.txt b/modules/indexing/readme.txt
index 9e87ede..699b2ee 100644
--- a/modules/indexing/readme.txt
+++ b/modules/indexing/readme.txt
@@ -1,10 +1,10 @@
-GridGain Indexing Module
-------------------------
+Apache Ignite Indexing Module
+-----------------------------
 
-GridGain indexing module provides capabilities to index cache context and run SQL, full text or
+Apache Ignite indexing module provides capabilities to index cache context and run SQL, full text or
 individual field queries against these indexes.
 
-To enable indexing module when starting a standalone node, move 'optional/gridgain-indexing' folder to
+To enable indexing module when starting a standalone node, move 'optional/ignite-indexing' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -23,8 +23,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-indexing</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-indexing</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/jcl/readme.txt
----------------------------------------------------------------------
diff --git a/modules/jcl/readme.txt b/modules/jcl/readme.txt
index 878c7d3..0f713dc 100644
--- a/modules/jcl/readme.txt
+++ b/modules/jcl/readme.txt
@@ -1,10 +1,10 @@
-GridGain JCL Module
--------------------
+Apache Ignite JCL Module
+------------------------
 
-GridGain JCL module provides GridLogger implementation that can delegate to any logger based
+Apache Ignite JCL module provides GridLogger implementation that can delegate to any logger based
 on Jakarta Commons Logging (JCL).
 
-To enable JCL module when starting a standalone node, move 'optional/gridgain-jcl' folder to
+To enable JCL module when starting a standalone node, move 'optional/ignite-jcl' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -23,8 +23,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-jcl</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-jcl</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/jta/readme.txt
----------------------------------------------------------------------
diff --git a/modules/jta/readme.txt b/modules/jta/readme.txt
index 67cf2ea..5893824 100644
--- a/modules/jta/readme.txt
+++ b/modules/jta/readme.txt
@@ -1,9 +1,9 @@
-GridGain JTA Module
--------------------
+Apache Ignite JTA Module
+------------------------
 
-GridGain JTA module provides capabilities to integrate GridGain cache transactions with JTA.
+Apache Ignite JTA module provides capabilities to integrate Apache Ignite cache transactions with JTA.
 
-To enable JTA module when starting a standalone node, move 'optional/gridgain-jta' folder to
+To enable JTA module when starting a standalone node, move 'optional/ignite-jta' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -22,8 +22,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-jta</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-jta</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/log4j/readme.txt
----------------------------------------------------------------------
diff --git a/modules/log4j/readme.txt b/modules/log4j/readme.txt
index b03294e..8d8ff4f 100644
--- a/modules/log4j/readme.txt
+++ b/modules/log4j/readme.txt
@@ -1,9 +1,9 @@
-GridGain Log4J Module
----------------------
+Apache Ignite Log4J Module
+--------------------------
 
-GridGain Log4J module provides GridLogger implementation based on Apache Log4J.
+Apache Ignite Log4J module provides GridLogger implementation based on Apache Log4J.
 
-To enable Log4J module when starting a standalone node, move 'optional/gridgain-log4j' folder to
+To enable Log4J module when starting a standalone node, move 'optional/ignite-log4j' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -22,8 +22,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-log4j</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-log4j</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/rest-http/readme.txt
----------------------------------------------------------------------
diff --git a/modules/rest-http/readme.txt b/modules/rest-http/readme.txt
index 07c35ad..139f287 100644
--- a/modules/rest-http/readme.txt
+++ b/modules/rest-http/readme.txt
@@ -1,10 +1,10 @@
-GridGain REST-HTTP Module
--------------------------
+Apache Ignite REST-HTTP Module
+------------------------------
 
-GridGain REST-HTTP module provides Jetty-based server which can be used to execute tasks and/or cache commands
+Apache Ignite REST-HTTP module provides Jetty-based server which can be used to execute tasks and/or cache commands
 in grid using REST approach via HTTP protocol.
 
-To enable REST-HTTP module when starting a standalone node, move 'optional/gridgain-rest-http' folder to
+To enable REST-HTTP module when starting a standalone node, move 'optional/ignite-rest-http' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -23,8 +23,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-rest-http</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-rest-http</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/scalar/readme.txt
----------------------------------------------------------------------
diff --git a/modules/scalar/readme.txt b/modules/scalar/readme.txt
index 6f32a08..88bb921 100644
--- a/modules/scalar/readme.txt
+++ b/modules/scalar/readme.txt
@@ -1,9 +1,9 @@
-GridGain Scalar Module
-----------------------
+Apache Ignite Scalar Module
+---------------------------
 
-GridGain Scalar module provides Scala-based DSL with extensions and shortcuts for GridGain API.
+Apache Ignite Scalar module provides Scala-based DSL with extensions and shortcuts for Apache Ignite API.
 
-To enable Scalar module when starting a standalone node, move 'optional/gridgain-scalar' folder to
+To enable Scalar module when starting a standalone node, move 'optional/ignite-scalar' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/schedule/readme.txt
----------------------------------------------------------------------
diff --git a/modules/schedule/readme.txt b/modules/schedule/readme.txt
index 43c01e3..ba51203 100644
--- a/modules/schedule/readme.txt
+++ b/modules/schedule/readme.txt
@@ -1,9 +1,9 @@
-GridGain Schedule Module
-------------------------
+Apache Ignite Schedule Module
+-----------------------------
 
-GridGain Schedule module provides functionality for scheduling jobs locally using UNIX cron-based syntax.
+Apache Ignite Schedule module provides functionality for scheduling jobs locally using UNIX cron-based syntax.
 
-To enable Schedule module when starting a standalone node, move 'optional/gridgain-schedule' folder to
+To enable Schedule module when starting a standalone node, move 'optional/ignite-schedule' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -22,8 +22,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-schedule</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-schedule</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/slf4j/readme.txt
----------------------------------------------------------------------
diff --git a/modules/slf4j/readme.txt b/modules/slf4j/readme.txt
index c4255ae..c6fadc3 100644
--- a/modules/slf4j/readme.txt
+++ b/modules/slf4j/readme.txt
@@ -1,9 +1,9 @@
-GridGain SLF4J Module
----------------------
+Apache Ignite SLF4J Module
+--------------------------
 
-GridGain SLF4J module provides GridLogger implementation based on SLF4J.
+Apache Ignite SLF4J module provides GridLogger implementation based on SLF4J.
 
-To enable SLF4J module when starting a standalone node, move 'optional/gridgain-slf4j' folder to
+To enable SLF4J module when starting a standalone node, move 'optional/ignite-slf4j' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -22,8 +22,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-slf4j</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-slf4j</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/spring/readme.txt
----------------------------------------------------------------------
diff --git a/modules/spring/readme.txt b/modules/spring/readme.txt
index 6b9cbd8..93eade9 100644
--- a/modules/spring/readme.txt
+++ b/modules/spring/readme.txt
@@ -1,10 +1,10 @@
-GridGain Spring Module
-----------------------
+Apache Ignite Spring Module
+---------------------------
 
-GridGain Spring module provides resources injection capabilities and parser for Spring-based
+Apache Ignite Spring module provides resources injection capabilities and parser for Spring-based
 configuration XML files.
 
-To enable Spring module when starting a standalone node, move 'optional/gridgain-spring' folder to
+To enable Spring module when starting a standalone node, move 'optional/ignite-spring' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -23,8 +23,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-spring</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-spring</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/ssh/readme.txt
----------------------------------------------------------------------
diff --git a/modules/ssh/readme.txt b/modules/ssh/readme.txt
index 073ae51..e69c896 100644
--- a/modules/ssh/readme.txt
+++ b/modules/ssh/readme.txt
@@ -1,9 +1,9 @@
-GridGain SSH Module
--------------------
+Apache Ignite SSH Module
+------------------------
 
-GridGain SSH module provides capabilities to start GridGain nodes on remote machines via SSH.
+Apache Ignite SSH module provides capabilities to start Apache Ignite nodes on remote machines via SSH.
 
-To enable SSH module when starting a standalone node, move 'optional/gridgain-ssh' folder to
+To enable SSH module when starting a standalone node, move 'optional/ignite-ssh' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -22,8 +22,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-ssh</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-ssh</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/urideploy/readme.txt
----------------------------------------------------------------------
diff --git a/modules/urideploy/readme.txt b/modules/urideploy/readme.txt
index b334838..a641ddf 100644
--- a/modules/urideploy/readme.txt
+++ b/modules/urideploy/readme.txt
@@ -1,10 +1,10 @@
-GridGain URI Deploy Module
---------------------------
+Apache Ignite URI Deploy Module
+-------------------------------
 
-GridGain URI Deploy module provides capabilities to deploy tasks from different sources like
+Apache Ignite URI Deploy module provides capabilities to deploy tasks from different sources like
 File System, HTTP, FTP, or even Email.
 
-To enable URI Deploy module when starting a standalone node, move 'optional/gridgain-urideploy' folder to
+To enable URI Deploy module when starting a standalone node, move 'optional/ignite-urideploy' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -23,8 +23,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-urideploy</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-urideploy</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/029e96b2/modules/web/readme.txt
----------------------------------------------------------------------
diff --git a/modules/web/readme.txt b/modules/web/readme.txt
index 86c8620..5acdf40 100644
--- a/modules/web/readme.txt
+++ b/modules/web/readme.txt
@@ -1,11 +1,11 @@
-GridGain Web Module
--------------------
+Apache Ignite Web Module
+------------------------
 
-GridGain Web module provides GridGain node startups based on servlet and servlet context listener
-which allow to start GridGain inside any web container. Additionally this module provides
-capabilities to cache web sessions in GridGain cache.
+Apache Ignite Web module provides Apache Ignite node startups based on servlet and servlet context listener
+which allow to start Apache Ignite inside any web container. Additionally this module provides
+capabilities to cache web sessions in Apache Ignite cache.
 
-To enable Web module when starting a standalone node, move 'optional/gridgain-web' folder to
+To enable Web module when starting a standalone node, move 'optional/ignite-web' folder to
 'libs' folder before running 'ggstart.{sh|bat}' script. The content of the module folder will
 be added to classpath in this case.
 
@@ -24,8 +24,8 @@ interested in):
     <dependencies>
         ...
         <dependency>
-            <groupId>org.gridgain</groupId>
-            <artifactId>gridgain-web</artifactId>
+            <groupId>org.apache.ignite</groupId>
+            <artifactId>ignite-web</artifactId>
             <version>${ignite.version}</version>
         </dependency>
         ...


[12/50] [abbrv] incubator-ignite git commit: # Cache store manager fix

Posted by vo...@apache.org.
# Cache store manager fix


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

Branch: refs/heads/ignite-16
Commit: 673d11b591789d37f5f5240ad267e875f76d3821
Parents: 6cab420
Author: Valentin Kulichenko <vk...@gridgain.com>
Authored: Wed Jan 28 17:45:30 2015 -0800
Committer: Valentin Kulichenko <vk...@gridgain.com>
Committed: Wed Jan 28 17:45:30 2015 -0800

----------------------------------------------------------------------
 .../processors/cache/GridCacheStoreManager.java   | 18 +++++++++---------
 1 file changed, 9 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/673d11b5/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
index 741a8c9..61408cb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
@@ -176,17 +176,17 @@ public class GridCacheStoreManager<K, V> extends GridCacheManagerAdapter<K, V> {
             }
         }
 
-        if (!cctx.config().isKeepPortableInStore()) {
-            if (cctx.config().isPortableEnabled()) {
-                if (store instanceof GridInteropAware)
-                    ((GridInteropAware)store).configure(true);
-                else
-                    convertPortable = true;
-            }
+        boolean convertPortable = !cctx.config().isKeepPortableInStore();
+
+        if (cctx.config().isPortableEnabled()) {
+            if (store instanceof GridInteropAware)
+                ((GridInteropAware)store).configure(cctx.cache().name(), convertPortable);
             else
-                U.warn(log, "GridCacheConfiguration.isKeepPortableInStore() configuration property will " +
-                    "be ignored because portable mode is not enabled for cache: " + cctx.namex());
+                this.convertPortable = convertPortable;
         }
+        else if (convertPortable)
+            U.warn(log, "GridCacheConfiguration.isKeepPortableInStore() configuration property will " +
+                "be ignored because portable mode is not enabled for cache: " + cctx.namex());
     }
 
     /** {@inheritDoc} */


[40/50] [abbrv] incubator-ignite git commit: GridEx -> IgniteEx GridKernal -> IgniteKernal

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
deleted file mode 100644
index 8ffafac..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
+++ /dev/null
@@ -1,3322 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.internal.processors.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.internal.processors.portable.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.lifecycle.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.mxbean.*;
-import org.apache.ignite.plugin.*;
-import org.apache.ignite.internal.product.*;
-import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.authentication.*;
-import org.apache.ignite.spi.authentication.noop.*;
-import org.apache.ignite.hadoop.*;
-import org.apache.ignite.internal.managers.*;
-import org.apache.ignite.internal.managers.checkpoint.*;
-import org.apache.ignite.internal.managers.collision.*;
-import org.apache.ignite.internal.managers.communication.*;
-import org.apache.ignite.internal.managers.deployment.*;
-import org.apache.ignite.internal.managers.discovery.*;
-import org.apache.ignite.internal.managers.eventstorage.*;
-import org.apache.ignite.internal.managers.failover.*;
-import org.apache.ignite.internal.managers.indexing.*;
-import org.apache.ignite.internal.managers.loadbalancer.*;
-import org.apache.ignite.internal.managers.securesession.*;
-import org.apache.ignite.internal.managers.security.*;
-import org.apache.ignite.internal.managers.swapspace.*;
-import org.apache.ignite.internal.processors.affinity.*;
-import org.apache.ignite.internal.processors.clock.*;
-import org.apache.ignite.internal.processors.closure.*;
-import org.apache.ignite.internal.processors.continuous.*;
-import org.apache.ignite.internal.processors.dataload.*;
-import org.apache.ignite.internal.processors.email.*;
-import org.apache.ignite.internal.processors.interop.*;
-import org.apache.ignite.internal.processors.job.*;
-import org.apache.ignite.internal.processors.jobmetrics.*;
-import org.apache.ignite.internal.processors.license.*;
-import org.apache.ignite.internal.processors.offheap.*;
-import org.apache.ignite.internal.processors.plugin.*;
-import org.apache.ignite.internal.processors.port.*;
-import org.apache.ignite.internal.processors.query.*;
-import org.apache.ignite.internal.processors.resource.*;
-import org.apache.ignite.internal.processors.rest.*;
-import org.apache.ignite.internal.processors.segmentation.*;
-import org.apache.ignite.internal.processors.service.*;
-import org.apache.ignite.internal.processors.session.*;
-import org.apache.ignite.internal.processors.streamer.*;
-import org.apache.ignite.internal.processors.task.*;
-import org.apache.ignite.internal.processors.timeout.*;
-import org.apache.ignite.plugin.security.*;
-import org.apache.ignite.spi.securesession.noop.*;
-import org.apache.ignite.internal.util.future.*;
-import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.internal.util.nodestart.*;
-import org.apache.ignite.internal.util.tostring.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jetbrains.annotations.*;
-
-import javax.management.*;
-import java.io.*;
-import java.lang.management.*;
-import java.lang.reflect.*;
-import java.net.*;
-import java.text.*;
-import java.util.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-
-import static org.apache.ignite.internal.GridKernalState.*;
-import static org.apache.ignite.lifecycle.LifecycleEventType.*;
-import static org.apache.ignite.IgniteSystemProperties.*;
-import static org.apache.ignite.internal.IgniteComponentType.*;
-import static org.apache.ignite.internal.GridNodeAttributes.*;
-import static org.apache.ignite.internal.GridProductImpl.*;
-import static org.apache.ignite.internal.processors.license.GridLicenseSubsystem.*;
-import static org.apache.ignite.internal.util.nodestart.GridNodeStartUtils.*;
-
-/**
- * GridGain kernal.
- * <p/>
- * See <a href="http://en.wikipedia.org/wiki/Kernal">http://en.wikipedia.org/wiki/Kernal</a> for information on the
- * misspelling.
- */
-public class GridKernal extends ClusterGroupAdapter implements GridEx, IgniteMXBean {
-    /** */
-    private static final long serialVersionUID = 0L;
-
-    /** Compatible versions. */
-    private static final String COMPATIBLE_VERS = GridProperties.get("gridgain.compatible.vers");
-
-    /** GridGain site that is shown in log messages. */
-    static final String SITE = "www.gridgain.com";
-
-    /** System line separator. */
-    private static final String NL = U.nl();
-
-    /** Periodic version check delay. */
-    private static final long PERIODIC_VER_CHECK_DELAY = 1000 * 60 * 60; // Every hour.
-
-    /** Periodic version check delay. */
-    private static final long PERIODIC_VER_CHECK_CONN_TIMEOUT = 10 * 1000; // 10 seconds.
-
-    /** Periodic version check delay. */
-    private static final long PERIODIC_LIC_CHECK_DELAY = 1000 * 60; // Every minute.
-
-    /** Periodic starvation check interval. */
-    private static final long PERIODIC_STARVATION_CHECK_FREQ = 1000 * 30;
-
-    /** Shutdown delay in msec. when license violation detected. */
-    private static final int SHUTDOWN_DELAY = 60 * 1000;
-
-    /** */
-    private IgniteConfiguration cfg;
-
-    /** */
-    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
-    @GridToStringExclude
-    private GridLoggerProxy log;
-
-    /** */
-    private String gridName;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName kernalMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName locNodeMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName pubExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName sysExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName mgmtExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName p2PExecSvcMBean;
-
-    /** */
-    @GridToStringExclude
-    private ObjectName restExecSvcMBean;
-
-    /** Kernal start timestamp. */
-    private long startTime = U.currentTimeMillis();
-
-    /** Spring context, potentially {@code null}. */
-    private GridSpringResourceContext rsrcCtx;
-
-    /** */
-    @GridToStringExclude
-    private Timer updateNtfTimer;
-
-    /** */
-    @GridToStringExclude
-    private Timer starveTimer;
-
-    /** */
-    @GridToStringExclude
-    private Timer licTimer;
-
-    /** */
-    @GridToStringExclude
-    private Timer metricsLogTimer;
-
-    /** Indicate error on grid stop. */
-    @GridToStringExclude
-    private boolean errOnStop;
-
-    /** Node local store. */
-    @GridToStringExclude
-    private ClusterNodeLocalMap nodeLoc;
-
-    /** Scheduler. */
-    @GridToStringExclude
-    private IgniteScheduler scheduler;
-
-    /** Grid security instance. */
-    @GridToStringExclude
-    private GridSecurity security;
-
-    /** Portables instance. */
-    @GridToStringExclude
-    private IgnitePortables portables;
-
-    /** Kernal gateway. */
-    @GridToStringExclude
-    private final AtomicReference<GridKernalGateway> gw = new AtomicReference<>();
-
-    /** Data Grid edition usage registered flag. */
-    @GridToStringExclude
-    private volatile boolean dbUsageRegistered;
-
-    /** */
-    @GridToStringExclude
-    private final Collection<String> compatibleVers;
-
-    /** Stop guard. */
-    @GridToStringExclude
-    private final AtomicBoolean stopGuard = new AtomicBoolean();
-
-    /**
-     * No-arg constructor is required by externalization.
-     */
-    public GridKernal() {
-        this(null);
-    }
-
-    /**
-     * @param rsrcCtx Optional Spring application context.
-     */
-    public GridKernal(@Nullable GridSpringResourceContext rsrcCtx) {
-        super(null, null, null, (IgnitePredicate<ClusterNode>)null);
-
-        this.rsrcCtx = rsrcCtx;
-
-        String[] compatibleVers = COMPATIBLE_VERS.split(",");
-
-        for (int i = 0; i < compatibleVers.length; i++)
-            compatibleVers[i] = compatibleVers[i].trim();
-
-        this.compatibleVers = Collections.unmodifiableList(Arrays.asList(compatibleVers));
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteCluster cluster() {
-        return this;
-    }
-
-    /** {@inheritDoc} */
-    @Override public final IgniteCompute compute(ClusterGroup prj) {
-        return ((ClusterGroupAdapter)prj).compute();
-    }
-
-    /** {@inheritDoc} */
-    @Override public final IgniteMessaging message(ClusterGroup prj) {
-        return ((ClusterGroupAdapter)prj).message();
-    }
-
-    /** {@inheritDoc} */
-    @Override public final IgniteEvents events(ClusterGroup prj) {
-        return ((ClusterGroupAdapter)prj).events();
-    }
-
-    /** {@inheritDoc} */
-    @Override public IgniteManaged managed(ClusterGroup prj) {
-        return ((ClusterGroupAdapter)prj).managed();
-    }
-
-    /** {@inheritDoc} */
-    @Override public ExecutorService executorService(ClusterGroup prj) {
-        return ((ClusterGroupAdapter)prj).executorService();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String name() {
-        return gridName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getCopyright() {
-        return ctx.product().copyright();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getLicenseFilePath() {
-        assert cfg != null;
-
-        return cfg.getLicenseUrl();
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getStartTimestamp() {
-        return startTime;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getStartTimestampFormatted() {
-        return DateFormat.getDateTimeInstance().format(new Date(startTime));
-    }
-
-    /** {@inheritDoc} */
-    @Override public long getUpTime() {
-        return U.currentTimeMillis() - startTime;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getUpTimeFormatted() {
-        return X.timeSpan2HMSM(U.currentTimeMillis() - startTime);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getFullVersion() {
-        return COMPOUND_VER + '-' + BUILD_TSTAMP_STR;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getCheckpointSpiFormatted() {
-        assert cfg != null;
-
-        return Arrays.toString(cfg.getCheckpointSpi());
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSwapSpaceSpiFormatted() {
-        assert cfg != null;
-
-        return cfg.getSwapSpaceSpi().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getCommunicationSpiFormatted() {
-        assert cfg != null;
-
-        return cfg.getCommunicationSpi().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getDeploymentSpiFormatted() {
-        assert cfg != null;
-
-        return cfg.getDeploymentSpi().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getDiscoverySpiFormatted() {
-        assert cfg != null;
-
-        return cfg.getDiscoverySpi().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getEventStorageSpiFormatted() {
-        assert cfg != null;
-
-        return cfg.getEventStorageSpi().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getCollisionSpiFormatted() {
-        assert cfg != null;
-
-        return cfg.getCollisionSpi().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getFailoverSpiFormatted() {
-        assert cfg != null;
-
-        return Arrays.toString(cfg.getFailoverSpi());
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getLoadBalancingSpiFormatted() {
-        assert cfg != null;
-
-        return Arrays.toString(cfg.getLoadBalancingSpi());
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getAuthenticationSpiFormatted() {
-        assert cfg != null;
-
-        return cfg.getAuthenticationSpi().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getSecureSessionSpiFormatted() {
-        assert cfg != null;
-
-        return cfg.getSecureSessionSpi().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getOsInformation() {
-        return U.osString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getJdkInformation() {
-        return U.jdkString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getOsUser() {
-        return System.getProperty("user.name");
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getVmName() {
-        return ManagementFactory.getRuntimeMXBean().getName();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getInstanceName() {
-        return gridName;
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getExecutorServiceFormatted() {
-        assert cfg != null;
-
-        return cfg.getExecutorService().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getGridGainHome() {
-        assert cfg != null;
-
-        return cfg.getGridGainHome();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getGridLoggerFormatted() {
-        assert cfg != null;
-
-        return cfg.getGridLogger().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public String getMBeanServerFormatted() {
-        assert cfg != null;
-
-        return cfg.getMBeanServer().toString();
-    }
-
-    /** {@inheritDoc} */
-    @Override public UUID getLocalNodeId() {
-        assert cfg != null;
-
-        return cfg.getNodeId();
-    }
-
-    /** {@inheritDoc} */
-    @SuppressWarnings("unchecked")
-    @Override public Collection<String> getUserAttributesFormatted() {
-        assert cfg != null;
-
-        return F.transform(cfg.getUserAttributes().entrySet(), new C1<Map.Entry<String, ?>, String>() {
-            @Override public String apply(Map.Entry<String, ?> e) {
-                return e.getKey() + ", " + e.getValue().toString();
-            }
-        });
-    }
-
-    /** {@inheritDoc} */
-    @Override public boolean isPeerClassLoadingEnabled() {
-        assert cfg != null;
-
-        return cfg.isPeerClassLoadingEnabled();
-    }
-
-    /** {@inheritDoc} */
-    @Override public Collection<String> getLifecycleBeansFormatted() {
-        LifecycleBean[] beans = cfg.getLifecycleBeans();
-
-        return F.isEmpty(beans) ? Collections.<String>emptyList() : F.transform(beans, F.<LifecycleBean>string());
-    }
-
-    /**
-     * @param attrs Current attributes.
-     * @param name  New attribute name.
-     * @param val New attribute value.
-     * @throws IgniteCheckedException If duplicated SPI name found.
-     */
-    private void add(Map<String, Object> attrs, String name, @Nullable Serializable val) throws IgniteCheckedException {
-        assert attrs != null;
-        assert name != null;
-
-        if (attrs.put(name, val) != null) {
-            if (name.endsWith(ATTR_SPI_CLASS))
-                // User defined duplicated names for the different SPIs.
-                throw new IgniteCheckedException("Failed to set SPI attribute. Duplicated SPI name found: " +
-                    name.substring(0, name.length() - ATTR_SPI_CLASS.length()));
-
-            // Otherwise it's a mistake of setting up duplicated attribute.
-            assert false : "Duplicate attribute: " + name;
-        }
-    }
-
-    /**
-     * Notifies life-cycle beans of grid event.
-     *
-     * @param evt Grid event.
-     * @throws IgniteCheckedException If user threw exception during start.
-     */
-    @SuppressWarnings({"CatchGenericClass"})
-    private void notifyLifecycleBeans(LifecycleEventType evt) throws IgniteCheckedException {
-        if (!cfg.isDaemon() && cfg.getLifecycleBeans() != null)
-            for (LifecycleBean bean : cfg.getLifecycleBeans())
-                if (bean != null)
-                    bean.onLifecycleEvent(evt);
-    }
-
-    /**
-     * Notifies life-cycle beans of grid event.
-     *
-     * @param evt Grid event.
-     */
-    @SuppressWarnings({"CatchGenericClass"})
-    private void notifyLifecycleBeansEx(LifecycleEventType evt) {
-        try {
-            notifyLifecycleBeans(evt);
-        }
-        // Catch generic throwable to secure against user assertions.
-        catch (Throwable e) {
-            U.error(log, "Failed to notify lifecycle bean (safely ignored) [evt=" + evt +
-                ", gridName=" + gridName + ']', e);
-        }
-    }
-
-    /**
-     * @param cfg Grid configuration to use.
-     * @param utilityCachePool Utility cache pool.
-     * @param errHnd Error handler to use for notification about startup problems.
-     * @throws IgniteCheckedException Thrown in case of any errors.
-     */
-    @SuppressWarnings({"CatchGenericClass", "unchecked"})
-    public void start(final IgniteConfiguration cfg, ExecutorService utilityCachePool, GridAbsClosure errHnd)
-        throws IgniteCheckedException {
-        gw.compareAndSet(null, new GridKernalGatewayImpl(cfg.getGridName()));
-
-        GridKernalGateway gw = this.gw.get();
-
-        gw.writeLock();
-
-        try {
-            switch (gw.getState()) {
-                case STARTED: {
-                    U.warn(log, "Grid has already been started (ignored).");
-
-                    return;
-                }
-
-                case STARTING: {
-                    U.warn(log, "Grid is already in process of being started (ignored).");
-
-                    return;
-                }
-
-                case STOPPING: {
-                    throw new IgniteCheckedException("Grid is in process of being stopped");
-                }
-
-                case STOPPED: {
-                    break;
-                }
-            }
-
-            gw.setState(STARTING);
-        }
-        finally {
-            gw.writeUnlock();
-        }
-
-        assert cfg != null;
-
-        // Make sure we got proper configuration.
-        validateCommon(cfg);
-
-        gridName = cfg.getGridName();
-
-        this.cfg = cfg;
-
-        log = (GridLoggerProxy)cfg.getGridLogger().getLogger(getClass().getName() +
-            (gridName != null ? '%' + gridName : ""));
-
-        RuntimeMXBean rtBean = ManagementFactory.getRuntimeMXBean();
-
-        // Ack various information.
-        ackAsciiLogo();
-        ackConfigUrl();
-        ackDaemon();
-        ackOsInfo();
-        ackLanguageRuntime();
-        ackRemoteManagement();
-        ackVmArguments(rtBean);
-        ackClassPaths(rtBean);
-        ackSystemProperties();
-        ackEnvironmentVariables();
-        ackSmtpConfiguration();
-        ackCacheConfiguration();
-        ackP2pConfiguration();
-
-        // Run background network diagnostics.
-        GridDiagnostic.runBackgroundCheck(gridName, cfg.getExecutorService(), log);
-
-        boolean notifyEnabled = IgniteSystemProperties.getBoolean(GG_UPDATE_NOTIFIER, true);
-
-        GridUpdateNotifier verChecker0 = null;
-
-        if (notifyEnabled) {
-            try {
-                verChecker0 = new GridUpdateNotifier(gridName, VER, SITE, gw, false);
-
-                verChecker0.checkForNewVersion(cfg.getExecutorService(), log);
-            }
-            catch (IgniteCheckedException e) {
-                if (log.isDebugEnabled())
-                    log.debug("Failed to create GridUpdateNotifier: " + e);
-            }
-        }
-
-        final GridUpdateNotifier verChecker = verChecker0;
-
-        // Ack 3-rd party licenses location.
-        if (log.isInfoEnabled() && cfg.getGridGainHome() != null)
-            log.info("3-rd party licenses can be found at: " + cfg.getGridGainHome() + File.separatorChar + "libs" +
-                File.separatorChar + "licenses");
-
-        // Check that user attributes are not conflicting
-        // with internally reserved names.
-        for (String name : cfg.getUserAttributes().keySet())
-            if (name.startsWith(ATTR_PREFIX))
-                throw new IgniteCheckedException("User attribute has illegal name: '" + name + "'. Note that all names " +
-                    "starting with '" + ATTR_PREFIX + "' are reserved for internal use.");
-
-        // Ack local node user attributes.
-        logNodeUserAttributes();
-
-        // Ack configuration.
-        ackSpis();
-
-        Map<String, Object> attrs = createNodeAttributes(cfg, BUILD_TSTAMP_STR);
-
-        // Spin out SPIs & managers.
-        try {
-            GridKernalContextImpl ctx =
-                new GridKernalContextImpl(log, this, cfg, gw, utilityCachePool, ENT);
-
-            nodeLoc = new ClusterNodeLocalMapImpl(ctx);
-
-            U.onGridStart();
-
-            // Set context into rich adapter.
-            setKernalContext(ctx);
-
-            // Start and configure resource processor first as it contains resources used
-            // by all other managers and processors.
-            GridResourceProcessor rsrcProc = new GridResourceProcessor(ctx);
-
-            rsrcProc.setSpringContext(rsrcCtx);
-
-            ctx.product(new GridProductImpl(ctx, verChecker));
-
-            scheduler = new IgniteSchedulerImpl(ctx);
-
-            startProcessor(ctx, rsrcProc, attrs);
-
-            // Inject resources into lifecycle beans.
-            if (!cfg.isDaemon() && cfg.getLifecycleBeans() != null)
-                for (LifecycleBean bean : cfg.getLifecycleBeans())
-                    if (bean != null)
-                        rsrcProc.inject(bean);
-
-            // Lifecycle notification.
-            notifyLifecycleBeans(BEFORE_GRID_START);
-
-            // Starts lifecycle aware components.
-            U.startLifecycleAware(lifecycleAwares(cfg));
-
-            addHelper(ctx, GGFS_HELPER.create(F.isEmpty(cfg.getGgfsConfiguration())));
-
-            startProcessor(ctx, new IgnitePluginProcessor(ctx, cfg), attrs);
-
-            // Off-heap processor has no dependencies.
-            startProcessor(ctx, new GridOffHeapProcessor(ctx), attrs);
-
-            // Closure processor should be started before all others
-            // (except for resource processor), as many components can depend on it.
-            startProcessor(ctx, new GridClosureProcessor(ctx), attrs);
-
-            // Start some other processors (order & place is important).
-            startProcessor(ctx, (GridProcessor)EMAIL.create(ctx, cfg.getSmtpHost() == null), attrs);
-            startProcessor(ctx, new GridPortProcessor(ctx), attrs);
-            startProcessor(ctx, new GridJobMetricsProcessor(ctx), attrs);
-
-            // Timeout processor needs to be started before managers,
-            // as managers may depend on it.
-            startProcessor(ctx, new GridTimeoutProcessor(ctx), attrs);
-
-            // Start SPI managers.
-            // NOTE: that order matters as there are dependencies between managers.
-            startManager(ctx, createComponent(GridSecurityManager.class, ctx), attrs);
-            startManager(ctx, createComponent(GridSecureSessionManager.class, ctx), attrs);
-            startManager(ctx, new GridIoManager(ctx), attrs);
-            startManager(ctx, new GridCheckpointManager(ctx), attrs);
-
-            startManager(ctx, new GridEventStorageManager(ctx), attrs);
-            startManager(ctx, new GridDeploymentManager(ctx), attrs);
-            startManager(ctx, new GridLoadBalancerManager(ctx), attrs);
-            startManager(ctx, new GridFailoverManager(ctx), attrs);
-            startManager(ctx, new GridCollisionManager(ctx), attrs);
-            startManager(ctx, new GridSwapSpaceManager(ctx), attrs);
-            startManager(ctx, new GridIndexingManager(ctx), attrs);
-
-            ackSecurity(ctx);
-
-            // Start processors before discovery manager, so they will
-            // be able to start receiving messages once discovery completes.
-            startProcessor(ctx, new GridClockSyncProcessor(ctx), attrs);
-            startProcessor(ctx, createComponent(GridLicenseProcessor.class, ctx), attrs);
-            startProcessor(ctx, new GridAffinityProcessor(ctx), attrs);
-            startProcessor(ctx, createComponent(GridSegmentationProcessor.class, ctx), attrs);
-            startProcessor(ctx, new GridQueryProcessor(ctx), attrs);
-            startProcessor(ctx, new GridCacheProcessor(ctx), attrs);
-            startProcessor(ctx, new GridTaskSessionProcessor(ctx), attrs);
-            startProcessor(ctx, new GridJobProcessor(ctx), attrs);
-            startProcessor(ctx, new GridTaskProcessor(ctx), attrs);
-            startProcessor(ctx, (GridProcessor)SCHEDULE.createOptional(ctx), attrs);
-            startProcessor(ctx, createComponent(GridPortableProcessor.class, ctx), attrs);
-            startProcessor(ctx, createComponent(GridInteropProcessor.class, ctx), attrs);
-            startProcessor(ctx, new GridRestProcessor(ctx), attrs);
-            startProcessor(ctx, new GridDataLoaderProcessor(ctx), attrs);
-            startProcessor(ctx, new GridStreamProcessor(ctx), attrs);
-            startProcessor(ctx, (GridProcessor)GGFS.create(ctx, F.isEmpty(cfg.getGgfsConfiguration())), attrs);
-            startProcessor(ctx, new GridContinuousProcessor(ctx), attrs);
-            startProcessor(ctx, (GridProcessor)(cfg.isPeerClassLoadingEnabled() ?
-                IgniteComponentType.HADOOP.create(ctx, true): // No-op when peer class loading is enabled.
-                IgniteComponentType.HADOOP.createIfInClassPath(ctx, cfg.getHadoopConfiguration() != null)), attrs);
-            startProcessor(ctx, new GridServiceProcessor(ctx), attrs);
-
-            // Start plugins.
-            for (PluginProvider provider : ctx.plugins().allProviders()) {
-                ctx.add(new GridPluginComponent(provider));
-
-                provider.start(ctx.plugins().pluginContextForProvider(provider), attrs);
-            }
-
-            ctx.createMessageFactory();
-
-            if (ctx.isEnterprise()) {
-                security = new GridSecurityImpl(ctx);
-                portables = new GridPortablesImpl(ctx);
-            }
-
-            gw.writeLock();
-
-            try {
-                gw.setState(STARTED);
-
-                // Start discovery manager last to make sure that grid is fully initialized.
-                startManager(ctx, new GridDiscoveryManager(ctx), attrs);
-            }
-            finally {
-                gw.writeUnlock();
-            }
-
-            // Check whether physical RAM is not exceeded.
-            checkPhysicalRam();
-
-            // Suggest configuration optimizations.
-            suggestOptimizations(ctx, cfg);
-
-            if (!ctx.isEnterprise())
-                warnNotSupportedFeaturesForOs(cfg);
-
-            // Notify discovery manager the first to make sure that topology is discovered.
-            ctx.discovery().onKernalStart();
-
-            // Notify IO manager the second so further components can send and receive messages.
-            ctx.io().onKernalStart();
-
-            // Callbacks.
-            for (GridComponent comp : ctx) {
-                // Skip discovery manager.
-                if (comp instanceof GridDiscoveryManager)
-                    continue;
-
-                // Skip IO manager.
-                if (comp instanceof GridIoManager)
-                    continue;
-
-                comp.onKernalStart();
-            }
-
-            // Ack the license.
-            ctx.license().ackLicense();
-
-            // Register MBeans.
-            registerKernalMBean();
-            registerLocalNodeMBean();
-            registerExecutorMBeans();
-
-            // Lifecycle bean notifications.
-            notifyLifecycleBeans(AFTER_GRID_START);
-        }
-        catch (Throwable e) {
-            IgniteSpiVersionCheckException verCheckErr = X.cause(e, IgniteSpiVersionCheckException.class);
-
-            if (verCheckErr != null)
-                U.error(log, verCheckErr.getMessage());
-            else if (X.hasCause(e, InterruptedException.class, IgniteInterruptedException.class))
-                U.warn(log, "Grid startup routine has been interrupted (will rollback).");
-            else
-                U.error(log, "Got exception while starting (will rollback startup routine).", e);
-
-            errHnd.apply();
-
-            stop(true);
-
-            if (e instanceof IgniteCheckedException)
-                throw (IgniteCheckedException)e;
-            else
-                throw new IgniteCheckedException(e);
-        }
-
-        // Mark start timestamp.
-        startTime = U.currentTimeMillis();
-
-        // Ack latest version information.
-        if (verChecker != null)
-            verChecker.reportStatus(log);
-
-        if (notifyEnabled) {
-            assert verChecker != null;
-
-            verChecker.reportOnlyNew(true);
-            verChecker.licenseProcessor(ctx.license());
-
-            updateNtfTimer = new Timer("gridgain-update-notifier-timer");
-
-            // Setup periodic version check.
-            updateNtfTimer.scheduleAtFixedRate(new GridTimerTask() {
-                @Override public void safeRun() throws InterruptedException {
-                    verChecker.topologySize(nodes().size());
-
-                    verChecker.checkForNewVersion(cfg.getExecutorService(), log);
-
-                    // Just wait for 10 secs.
-                    Thread.sleep(PERIODIC_VER_CHECK_CONN_TIMEOUT);
-
-                    // Report status if one is available.
-                    // No-op if status is NOT available.
-                    verChecker.reportStatus(log);
-                }
-            }, PERIODIC_VER_CHECK_DELAY, PERIODIC_VER_CHECK_DELAY);
-        }
-
-        String intervalStr = IgniteSystemProperties.getString(GG_STARVATION_CHECK_INTERVAL);
-
-        // Start starvation checker if enabled.
-        boolean starveCheck = !isDaemon() && !"0".equals(intervalStr);
-
-        if (starveCheck) {
-            final long interval = F.isEmpty(intervalStr) ? PERIODIC_STARVATION_CHECK_FREQ : Long.parseLong(intervalStr);
-
-            starveTimer = new Timer("gridgain-starvation-checker");
-
-            starveTimer.scheduleAtFixedRate(new GridTimerTask() {
-                /** Last completed task count. */
-                private long lastCompletedCnt;
-
-                @Override protected void safeRun() {
-                    ExecutorService e = cfg.getExecutorService();
-
-                    if (!(e instanceof ThreadPoolExecutor))
-                        return;
-
-                    ThreadPoolExecutor exec = (ThreadPoolExecutor)e;
-
-                    long completedCnt = exec.getCompletedTaskCount();
-
-                    // If all threads are active and no task has completed since last time and there is
-                    // at least one waiting request, then it is possible starvation.
-                    if (exec.getPoolSize() == exec.getActiveCount() && completedCnt == lastCompletedCnt &&
-                        !exec.getQueue().isEmpty())
-                        LT.warn(log, null, "Possible thread pool starvation detected (no task completed in last " +
-                            interval + "ms, is executorService pool size large enough?)");
-
-                    lastCompletedCnt = completedCnt;
-                }
-            }, interval, interval);
-        }
-
-        if (!isDaemon()) {
-            licTimer = new Timer("gridgain-license-checker");
-
-            // Setup periodic license check.
-            licTimer.scheduleAtFixedRate(new GridTimerTask() {
-                @Override public void safeRun() throws InterruptedException {
-                    try {
-                        ctx.license().checkLicense();
-                    }
-                    // This exception only happens when license processor was unable
-                    // to resolve license violation on its own and this grid instance
-                    // now needs to be shutdown.
-                    //
-                    // Note that in most production configurations the license will
-                    // have certain grace period and license processor will attempt
-                    // to reload the license during the grace period.
-                    //
-                    // This exception thrown here means that grace period, if any,
-                    // has expired and license violation is still unresolved.
-                    catch (IgniteProductLicenseException ignored) {
-                        U.error(log, "License violation is unresolved. GridGain node will shutdown in " +
-                            (SHUTDOWN_DELAY / 1000) + " sec.");
-                        U.error(log, "  ^-- Contact your support for immediate assistance (!)");
-
-                        // Allow interruption to break from here since
-                        // node is stopping anyways.
-                        Thread.sleep(SHUTDOWN_DELAY);
-
-                        G.stop(gridName, true);
-                    }
-                    // Safety net.
-                    catch (Throwable e) {
-                        U.error(log, "Unable to check the license due to system error.", e);
-                        U.error(log, "Grid instance will be stopped...");
-
-                        // Stop the grid if we get unknown license-related error.
-                        // Should never happen. Practically an assertion...
-                        G.stop(gridName, true);
-                    }
-                }
-            }, PERIODIC_LIC_CHECK_DELAY, PERIODIC_LIC_CHECK_DELAY);
-        }
-
-        long metricsLogFreq = cfg.getMetricsLogFrequency();
-
-        if (metricsLogFreq > 0) {
-            metricsLogTimer = new Timer("gridgain-metrics-logger");
-
-            metricsLogTimer.scheduleAtFixedRate(new GridTimerTask() {
-                /** */
-                private final DecimalFormat dblFmt = new DecimalFormat("#.##");
-
-                @Override protected void safeRun() {
-                    if (log.isInfoEnabled()) {
-                        ClusterMetrics m = localNode().metrics();
-
-                        double cpuLoadPct = m.getCurrentCpuLoad() * 100;
-                        double avgCpuLoadPct = m.getAverageCpuLoad() * 100;
-                        double gcPct = m.getCurrentGcCpuLoad() * 100;
-
-                        long heapUsed = m.getHeapMemoryUsed();
-                        long heapMax = m.getHeapMemoryMaximum();
-
-                        long heapUsedInMBytes = heapUsed / 1024 / 1024;
-                        long heapCommInMBytes = m.getHeapMemoryCommitted() / 1024 / 1024;
-
-                        double freeHeapPct = heapMax > 0 ? ((double)((heapMax - heapUsed) * 100)) / heapMax : -1;
-
-                        int hosts = 0;
-                        int nodes = 0;
-                        int cpus = 0;
-
-                        try {
-                            ClusterMetrics metrics = metrics();
-
-                            Collection<ClusterNode> nodes0 = nodes();
-
-                            hosts = U.neighborhood(nodes0).size();
-                            nodes = nodes0.size();
-                            cpus = metrics.getTotalCpus();
-                        }
-                        catch (IgniteCheckedException ignore) {
-                            // No-op.
-                        }
-
-                        int pubPoolActiveThreads = 0;
-                        int pubPoolIdleThreads = 0;
-                        int pubPoolQSize = 0;
-
-                        ExecutorService pubExec = cfg.getExecutorService();
-
-                        if (pubExec instanceof ThreadPoolExecutor) {
-                            ThreadPoolExecutor exec = (ThreadPoolExecutor)pubExec;
-
-                            int poolSize = exec.getPoolSize();
-
-                            pubPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
-                            pubPoolIdleThreads = poolSize - pubPoolActiveThreads;
-                            pubPoolQSize = exec.getQueue().size();
-                        }
-
-                        int sysPoolActiveThreads = 0;
-                        int sysPoolIdleThreads = 0;
-                        int sysPoolQSize = 0;
-
-                        ExecutorService sysExec = cfg.getSystemExecutorService();
-
-                        if (sysExec instanceof ThreadPoolExecutor) {
-                            ThreadPoolExecutor exec = (ThreadPoolExecutor)sysExec;
-
-                            int poolSize = exec.getPoolSize();
-
-                            sysPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
-                            sysPoolIdleThreads = poolSize - sysPoolActiveThreads;
-                            sysPoolQSize = exec.getQueue().size();
-                        }
-
-                        String msg = NL +
-                            "Metrics for local node (to disable set 'metricsLogFrequency' to 0)" + NL +
-                            "    ^-- H/N/C [hosts=" + hosts + ", nodes=" + nodes + ", CPUs=" + cpus + "]" + NL +
-                            "    ^-- CPU [cur=" + dblFmt.format(cpuLoadPct) + "%, avg=" +
-                                dblFmt.format(avgCpuLoadPct) + "%, GC=" + dblFmt.format(gcPct) + "%]" + NL +
-                            "    ^-- Heap [used=" + dblFmt.format(heapUsedInMBytes) + "MB, free=" +
-                                dblFmt.format(freeHeapPct) + "%, comm=" + dblFmt.format(heapCommInMBytes) + "MB]" + NL +
-                            "    ^-- Public thread pool [active=" + pubPoolActiveThreads + ", idle=" +
-                                pubPoolIdleThreads + ", qSize=" + pubPoolQSize + "]" + NL +
-                            "    ^-- System thread pool [active=" + sysPoolActiveThreads + ", idle=" +
-                                sysPoolIdleThreads + ", qSize=" + sysPoolQSize + "]" + NL +
-                            "    ^-- Outbound messages queue [size=" + m.getOutboundMessagesQueueSize() + "]";
-
-                        log.info(msg);
-                    }
-                }
-            }, metricsLogFreq, metricsLogFreq);
-        }
-
-        ctx.performance().logSuggestions(log, gridName);
-
-        ackBenchmarks();
-        ackVisor();
-
-        ackStart(rtBean);
-
-        if (!isDaemon())
-            ctx.discovery().ackTopology();
-
-        // Send node start email notification, if enabled.
-        if (isSmtpEnabled() && isAdminEmailsSet() && cfg.isLifeCycleEmailNotification()) {
-            SB sb = new SB();
-
-            for (GridPortRecord rec : ctx.ports().records())
-                sb.a(rec.protocol()).a(":").a(rec.port()).a(" ");
-
-            String nid = localNode().id().toString().toUpperCase();
-            String nid8 = U.id8(localNode().id()).toUpperCase();
-
-            IgniteProductLicense lic = ctx.license().license();
-
-            String body =
-                "GridGain node started with the following parameters:" + NL +
-                NL +
-                "----" + NL +
-                "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH + NL +
-                "Grid name: " + gridName + NL +
-                "Node ID: " + nid + NL +
-                "Node order: " + localNode().order() + NL +
-                "Node addresses: " + U.addressesAsString(localNode()) + NL +
-                "Local ports: " + sb + NL +
-                "OS name: " + U.osString() + NL +
-                "OS user: " + System.getProperty("user.name") + NL +
-                "CPU(s): " + localNode().metrics().getTotalCpus() + NL +
-                "Heap: " + U.heapSize(localNode(), 2) + "GB" + NL +
-                "JVM name: " + U.jvmName() + NL +
-                "JVM vendor: " + U.jvmVendor() + NL +
-                "JVM version: " + U.jvmVersion() + NL +
-                "VM name: " + rtBean.getName() + NL;
-
-            if (lic != null) {
-                body +=
-                    "License ID: " + lic.id().toString().toUpperCase() + NL +
-                    "Licensed to: " + lic.userOrganization() + NL;
-            }
-            else
-                assert !ENT;
-
-            body +=
-                "----" + NL +
-                NL +
-                "NOTE:" + NL +
-                "This message is sent automatically to all configured admin emails." + NL +
-                "To change this behavior use 'lifeCycleEmailNotify' grid configuration property." +
-                NL + NL +
-                "| " + SITE + NL +
-                "| support@gridgain.com" + NL;
-
-            sendAdminEmailAsync("GridGain node started: " + nid8, body, false);
-        }
-    }
-
-    /**
-     * Validates common configuration parameters.
-     *
-     * @param cfg Configuration.
-     */
-    private void validateCommon(IgniteConfiguration cfg) {
-        A.notNull(cfg.getNodeId(), "cfg.getNodeId()");
-
-        A.notNull(cfg.getMBeanServer(), "cfg.getMBeanServer()");
-        A.notNull(cfg.getGridLogger(), "cfg.getGridLogger()");
-        A.notNull(cfg.getMarshaller(), "cfg.getMarshaller()");
-        A.notNull(cfg.getExecutorService(), "cfg.getExecutorService()");
-        A.notNull(cfg.getUserAttributes(), "cfg.getUserAttributes()");
-
-        // All SPIs should be non-null.
-        A.notNull(cfg.getSwapSpaceSpi(), "cfg.getSwapSpaceSpi()");
-        A.notNull(cfg.getCheckpointSpi(), "cfg.getCheckpointSpi()");
-        A.notNull(cfg.getCommunicationSpi(), "cfg.getCommunicationSpi()");
-        A.notNull(cfg.getDeploymentSpi(), "cfg.getDeploymentSpi()");
-        A.notNull(cfg.getDiscoverySpi(), "cfg.getDiscoverySpi()");
-        A.notNull(cfg.getEventStorageSpi(), "cfg.getEventStorageSpi()");
-        A.notNull(cfg.getAuthenticationSpi(), "cfg.getAuthenticationSpi()");
-        A.notNull(cfg.getSecureSessionSpi(), "cfg.getSecureSessionSpi()");
-        A.notNull(cfg.getCollisionSpi(), "cfg.getCollisionSpi()");
-        A.notNull(cfg.getFailoverSpi(), "cfg.getFailoverSpi()");
-        A.notNull(cfg.getLoadBalancingSpi(), "cfg.getLoadBalancingSpi()");
-        A.notNull(cfg.getIndexingSpi(), "cfg.getIndexingSpi()");
-
-        A.ensure(cfg.getNetworkTimeout() > 0, "cfg.getNetworkTimeout() > 0");
-        A.ensure(cfg.getNetworkSendRetryDelay() > 0, "cfg.getNetworkSendRetryDelay() > 0");
-        A.ensure(cfg.getNetworkSendRetryCount() > 0, "cfg.getNetworkSendRetryCount() > 0");
-
-        if (!F.isEmpty(cfg.getPluginConfigurations())) {
-            for (PluginConfiguration pluginCfg : cfg.getPluginConfigurations())
-                A.notNull(pluginCfg.providerClass(), "PluginConfiguration.providerClass()");
-        }
-    }
-
-    /**
-     * Checks whether physical RAM is not exceeded.
-     */
-    @SuppressWarnings("ConstantConditions")
-    private void checkPhysicalRam() {
-        long ram = ctx.discovery().localNode().attribute(ATTR_PHY_RAM);
-
-        if (ram != -1) {
-            String macs = ctx.discovery().localNode().attribute(ATTR_MACS);
-
-            long totalHeap = 0;
-
-            for (ClusterNode node : ctx.discovery().allNodes()) {
-                if (macs.equals(node.attribute(ATTR_MACS))) {
-                    long heap = node.metrics().getHeapMemoryMaximum();
-
-                    if (heap != -1)
-                        totalHeap += heap;
-                }
-            }
-
-            if (totalHeap > ram) {
-                U.quietAndWarn(log, "Attempting to start more nodes than physical RAM " +
-                    "available on current host (this can cause significant slowdown)");
-            }
-        }
-    }
-
-    /**
-     * @param ctx Context.
-     * @param cfg Configuration to check for possible performance issues.
-     */
-    private void suggestOptimizations(GridKernalContext ctx, IgniteConfiguration cfg) {
-        GridPerformanceSuggestions perf = ctx.performance();
-
-        if (ctx.collision().enabled())
-            perf.add("Disable collision resolution (remove 'collisionSpi' from configuration)");
-
-        if (ctx.checkpoint().enabled())
-            perf.add("Disable checkpoints (remove 'checkpointSpi' from configuration)");
-
-        if (cfg.isPeerClassLoadingEnabled())
-            perf.add("Disable peer class loading (set 'peerClassLoadingEnabled' to false)");
-
-        if (cfg.isMarshalLocalJobs())
-            perf.add("Disable local jobs marshalling (set 'marshalLocalJobs' to false)");
-
-        if (cfg.getIncludeEventTypes() != null && cfg.getIncludeEventTypes().length != 0)
-            perf.add("Disable grid events (remove 'includeEventTypes' from configuration)");
-
-        if (IgniteOptimizedMarshaller.available() && !(cfg.getMarshaller() instanceof IgniteOptimizedMarshaller))
-            perf.add("Enable optimized marshaller (set 'marshaller' to " +
-                IgniteOptimizedMarshaller.class.getSimpleName() + ')');
-    }
-
-    /**
-     * Warns user about unsupported features which was configured in OS edition.
-     *
-     * @param cfg Grid configuration.
-     */
-    private void warnNotSupportedFeaturesForOs(IgniteConfiguration cfg) {
-        Collection<String> msgs = new ArrayList<>();
-
-        if (!F.isEmpty(cfg.getSegmentationResolvers()))
-            msgs.add("Network segmentation detection.");
-
-        if (cfg.getSecureSessionSpi() != null && !(cfg.getSecureSessionSpi() instanceof NoopSecureSessionSpi))
-            msgs.add("Secure session SPI.");
-
-        if (cfg.getAuthenticationSpi() != null && !(cfg.getAuthenticationSpi() instanceof NoopAuthenticationSpi))
-            msgs.add("Authentication SPI.");
-
-        if (!F.isEmpty(msgs)) {
-            U.quietAndInfo(log, "The following features are not supported in open source edition, " +
-                "related configuration settings will be ignored " +
-                "(consider downloading enterprise edition from http://www.gridgain.com):");
-
-            for (String s : msgs)
-                U.quietAndInfo(log, "  ^-- " + s);
-
-            U.quietAndInfo(log, "");
-        }
-    }
-
-    /**
-     * Creates attributes map and fills it in.
-     *
-     * @param cfg Grid configuration.
-     * @param build Build string.
-     * @return Map of all node attributes.
-     * @throws IgniteCheckedException thrown if was unable to set up attribute.
-     */
-    @SuppressWarnings({"SuspiciousMethodCalls", "unchecked", "TypeMayBeWeakened"})
-    private Map<String, Object> createNodeAttributes(IgniteConfiguration cfg, String build) throws IgniteCheckedException {
-        Map<String, Object> attrs = new HashMap<>();
-
-        final String[] incProps = cfg.getIncludeProperties();
-
-        try {
-            // Stick all environment settings into node attributes.
-            attrs.putAll(F.view(System.getenv(), new P1<String>() {
-                @Override public boolean apply(String name) {
-                    return incProps == null || U.containsStringArray(incProps, name, true) ||
-                        U.isVisorNodeStartProperty(name) || U.isVisorRequiredProperty(name);
-                }
-            }));
-
-            if (log.isDebugEnabled())
-                log.debug("Added environment properties to node attributes.");
-        }
-        catch (SecurityException e) {
-            throw new IgniteCheckedException("Failed to add environment properties to node attributes due to " +
-                "security violation: " + e.getMessage());
-        }
-
-        try {
-            // Stick all system properties into node's attributes overwriting any
-            // identical names from environment properties.
-            for (Map.Entry<Object, Object> e : F.view(System.getProperties(), new P1<Object>() {
-                @Override public boolean apply(Object o) {
-                    String name = (String)o;
-
-                    return incProps == null || U.containsStringArray(incProps, name, true) ||
-                        U.isVisorRequiredProperty(name);
-                }
-            }).entrySet()) {
-                Object val = attrs.get(e.getKey());
-
-                if (val != null && !val.equals(e.getValue()))
-                    U.warn(log, "System property will override environment variable with the same name: "
-                        + e.getKey());
-
-                attrs.put((String)e.getKey(), e.getValue());
-            }
-
-            if (log.isDebugEnabled())
-                log.debug("Added system properties to node attributes.");
-        }
-        catch (SecurityException e) {
-            throw new IgniteCheckedException("Failed to add system properties to node attributes due to security " +
-                "violation: " + e.getMessage());
-        }
-
-        // Add local network IPs and MACs.
-        String ips = F.concat(U.allLocalIps(), ", "); // Exclude loopbacks.
-        String macs = F.concat(U.allLocalMACs(), ", "); // Only enabled network interfaces.
-
-        // Ack network context.
-        if (log.isInfoEnabled()) {
-            log.info("Non-loopback local IPs: " + (F.isEmpty(ips) ? "N/A" : ips));
-            log.info("Enabled local MACs: " + (F.isEmpty(macs) ? "N/A" : macs));
-        }
-
-        // Warn about loopback.
-        if (ips.isEmpty() && macs.isEmpty())
-            U.warn(log, "GridGain is starting on loopback address... Only nodes on the same physical " +
-                "computer can participate in topology.",
-                "GridGain is starting on loopback address...");
-
-        // Stick in network context into attributes.
-        add(attrs, ATTR_IPS, (ips.isEmpty() ? "" : ips));
-        add(attrs, ATTR_MACS, (macs.isEmpty() ? "" : macs));
-
-        // Stick in some system level attributes
-        add(attrs, ATTR_JIT_NAME, U.getCompilerMx() == null ? "" : U.getCompilerMx().getName());
-        add(attrs, ATTR_BUILD_VER, COMPOUND_VER);
-        add(attrs, ATTR_BUILD_DATE, build);
-        add(attrs, ATTR_COMPATIBLE_VERS, (Serializable)compatibleVersions());
-        add(attrs, ATTR_MARSHALLER, cfg.getMarshaller().getClass().getName());
-        add(attrs, ATTR_USER_NAME, System.getProperty("user.name"));
-        add(attrs, ATTR_GRID_NAME, gridName);
-
-        add(attrs, ATTR_PEER_CLASSLOADING, cfg.isPeerClassLoadingEnabled());
-        add(attrs, ATTR_DEPLOYMENT_MODE, cfg.getDeploymentMode());
-        add(attrs, ATTR_LANG_RUNTIME, getLanguage());
-
-        add(attrs, ATTR_JVM_PID, U.jvmPid());
-
-        // Build a string from JVM arguments, because parameters with spaces are split.
-        SB jvmArgs = new SB(512);
-
-        for (String arg : U.jvmArgs()) {
-            if (arg.startsWith("-"))
-                jvmArgs.a("@@@");
-            else
-                jvmArgs.a(' ');
-
-            jvmArgs.a(arg);
-        }
-        // Add it to attributes.
-        add(attrs, ATTR_JVM_ARGS, jvmArgs.toString());
-
-        // Check daemon system property and override configuration if it's set.
-        if (isDaemon())
-            add(attrs, ATTR_DAEMON, "true");
-
-        // In case of the parsing error, JMX remote disabled or port not being set
-        // node attribute won't be set.
-        if (isJmxRemoteEnabled()) {
-            String portStr = System.getProperty("com.sun.management.jmxremote.port");
-
-            if (portStr != null)
-                try {
-                    add(attrs, ATTR_JMX_PORT, Integer.parseInt(portStr));
-                }
-                catch (NumberFormatException ignore) {
-                    // No-op.
-                }
-        }
-
-        // Whether restart is enabled and stick the attribute.
-        add(attrs, ATTR_RESTART_ENABLED, Boolean.toString(isRestartEnabled()));
-
-        // Save port range, port numbers will be stored by rest processor at runtime.
-        if (cfg.getClientConnectionConfiguration() != null)
-            add(attrs, ATTR_REST_PORT_RANGE, cfg.getClientConnectionConfiguration().getRestPortRange());
-
-        try {
-            AuthenticationSpi authSpi = cfg.getAuthenticationSpi();
-
-            boolean securityEnabled = authSpi != null && !U.hasAnnotation(authSpi.getClass(), IgniteSpiNoop.class);
-
-            GridSecurityCredentialsProvider provider = cfg.getSecurityCredentialsProvider();
-
-            if (provider != null) {
-                GridSecurityCredentials cred = provider.credentials();
-
-                if (cred != null)
-                    add(attrs, ATTR_SECURITY_CREDENTIALS, cred);
-                else if (securityEnabled)
-                    throw new IgniteCheckedException("Failed to start node (authentication SPI is configured, " +
-                        "by security credentials provider returned null).");
-            }
-            else if (securityEnabled)
-                throw new IgniteCheckedException("Failed to start node (authentication SPI is configured, " +
-                    "but security credentials provider is not set. Fix the configuration and restart the node).");
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteCheckedException("Failed to create node security credentials", e);
-        }
-
-        // Stick in SPI versions and classes attributes.
-        addAttributes(attrs, cfg.getCollisionSpi());
-        addAttributes(attrs, cfg.getSwapSpaceSpi());
-        addAttributes(attrs, cfg.getDiscoverySpi());
-        addAttributes(attrs, cfg.getFailoverSpi());
-        addAttributes(attrs, cfg.getCommunicationSpi());
-        addAttributes(attrs, cfg.getEventStorageSpi());
-        addAttributes(attrs, cfg.getCheckpointSpi());
-        addAttributes(attrs, cfg.getLoadBalancingSpi());
-        addAttributes(attrs, cfg.getAuthenticationSpi());
-        addAttributes(attrs, cfg.getSecureSessionSpi());
-        addAttributes(attrs, cfg.getDeploymentSpi());
-
-        // Set user attributes for this node.
-        if (cfg.getUserAttributes() != null) {
-            for (Map.Entry<String, ?> e : cfg.getUserAttributes().entrySet()) {
-                if (attrs.containsKey(e.getKey()))
-                    U.warn(log, "User or internal attribute has the same name as environment or system " +
-                        "property and will take precedence: " + e.getKey());
-
-                attrs.put(e.getKey(), e.getValue());
-            }
-        }
-
-        return attrs;
-    }
-
-    /**
-     * Add SPI version and class attributes into node attributes.
-     *
-     * @param attrs Node attributes map to add SPI attributes to.
-     * @param spiList Collection of SPIs to get attributes from.
-     * @throws IgniteCheckedException Thrown if was unable to set up attribute.
-     */
-    private void addAttributes(Map<String, Object> attrs, IgniteSpi... spiList) throws IgniteCheckedException {
-        for (IgniteSpi spi : spiList) {
-            Class<? extends IgniteSpi> spiCls = spi.getClass();
-
-            add(attrs, U.spiAttribute(spi, ATTR_SPI_CLASS), spiCls.getName());
-        }
-    }
-
-    /** @throws IgniteCheckedException If registration failed. */
-    private void registerKernalMBean() throws IgniteCheckedException {
-        try {
-            kernalMBean = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getGridName(),
-                "Kernal",
-                getClass().getSimpleName(),
-                this,
-                IgniteMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered kernal MBean: " + kernalMBean);
-        }
-        catch (JMException e) {
-            kernalMBean = null;
-
-            throw new IgniteCheckedException("Failed to register kernal MBean.", e);
-        }
-    }
-
-    /** @throws IgniteCheckedException If registration failed. */
-    private void registerLocalNodeMBean() throws IgniteCheckedException {
-        ClusterLocalNodeMetricsMXBean mbean = new ClusterLocalNodeMetricsMXBeanImpl(ctx.discovery().localNode());
-
-        try {
-            locNodeMBean = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getGridName(),
-                "Kernal",
-                mbean.getClass().getSimpleName(),
-                mbean,
-                ClusterLocalNodeMetricsMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered local node MBean: " + locNodeMBean);
-        }
-        catch (JMException e) {
-            locNodeMBean = null;
-
-            throw new IgniteCheckedException("Failed to register local node MBean.", e);
-        }
-    }
-
-    /** @throws IgniteCheckedException If registration failed. */
-    private void registerExecutorMBeans() throws IgniteCheckedException {
-        pubExecSvcMBean = registerExecutorMBean(cfg.getExecutorService(), "GridExecutionExecutor");
-        sysExecSvcMBean = registerExecutorMBean(cfg.getSystemExecutorService(), "GridSystemExecutor");
-        mgmtExecSvcMBean = registerExecutorMBean(cfg.getManagementExecutorService(), "GridManagementExecutor");
-        p2PExecSvcMBean = registerExecutorMBean(cfg.getPeerClassLoadingExecutorService(), "GridClassLoadingExecutor");
-
-        ClientConnectionConfiguration clientCfg = cfg.getClientConnectionConfiguration();
-
-        if (clientCfg != null) {
-            restExecSvcMBean = clientCfg.getRestExecutorService() != null ?
-                registerExecutorMBean(clientCfg.getRestExecutorService(), "GridRestExecutor") : null;
-        }
-    }
-
-    /**
-     * @param exec Executor service to register.
-     * @param name Property name for executor.
-     * @return Name for created MBean.
-     * @throws IgniteCheckedException If registration failed.
-     */
-    private ObjectName registerExecutorMBean(ExecutorService exec, String name) throws IgniteCheckedException {
-        assert exec != null;
-
-        try {
-            ObjectName res = U.registerMBean(
-                cfg.getMBeanServer(),
-                cfg.getGridName(),
-                "Thread Pools",
-                name,
-                new IgniteThreadPoolMXBeanAdapter(exec),
-                IgniteThreadPoolMXBean.class);
-
-            if (log.isDebugEnabled())
-                log.debug("Registered executor service MBean: " + res);
-
-            return res;
-        }
-        catch (JMException e) {
-            throw new IgniteCheckedException("Failed to register executor service MBean [name=" + name + ", exec=" + exec + ']',
-                e);
-        }
-    }
-
-    /**
-     * Unregisters given mbean.
-     *
-     * @param mbean MBean to unregister.
-     * @return {@code True} if successfully unregistered, {@code false} otherwise.
-     */
-    private boolean unregisterMBean(@Nullable ObjectName mbean) {
-        if (mbean != null)
-            try {
-                cfg.getMBeanServer().unregisterMBean(mbean);
-
-                if (log.isDebugEnabled())
-                    log.debug("Unregistered MBean: " + mbean);
-
-                return true;
-            }
-            catch (JMException e) {
-                U.error(log, "Failed to unregister MBean.", e);
-
-                return false;
-            }
-
-        return true;
-    }
-
-    /**
-     * @param ctx Kernal context.
-     * @param mgr Manager to start.
-     * @param attrs SPI attributes to set.
-     * @throws IgniteCheckedException Throw in case of any errors.
-     */
-    private void startManager(GridKernalContextImpl ctx, GridManager mgr, Map<String, Object> attrs)
-        throws IgniteCheckedException {
-        mgr.addSpiAttributes(attrs);
-
-        // Set all node attributes into discovery manager,
-        // so they can be distributed to all nodes.
-        if (mgr instanceof GridDiscoveryManager)
-            ((GridDiscoveryManager)mgr).setNodeAttributes(attrs, ctx.product().version());
-
-        // Add manager to registry before it starts to avoid
-        // cases when manager is started but registry does not
-        // have it yet.
-        ctx.add(mgr);
-
-        try {
-            mgr.start();
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteCheckedException("Failed to start manager: " + mgr, e);
-        }
-    }
-
-    /**
-     * @param ctx Kernal context.
-     * @param proc Processor to start.
-     * @param attrs Attributes.
-     * @throws IgniteCheckedException Thrown in case of any error.
-     */
-    private void startProcessor(GridKernalContextImpl ctx, GridProcessor proc, Map<String, Object> attrs)
-        throws IgniteCheckedException {
-        ctx.add(proc);
-
-        try {
-            proc.start();
-
-            proc.addAttributes(attrs);
-        }
-        catch (IgniteCheckedException e) {
-            throw new IgniteCheckedException("Failed to start processor: " + proc, e);
-        }
-    }
-
-    /**
-     * Add helper.
-     *
-     * @param ctx Context.
-     * @param helper Helper.
-     */
-    private void addHelper(GridKernalContextImpl ctx, Object helper) {
-        ctx.addHelper(helper);
-    }
-
-    /**
-     * Gets "on" or "off" string for given boolean value.
-     *
-     * @param b Boolean value to convert.
-     * @return Result string.
-     */
-    private String onOff(boolean b) {
-        return b ? "on" : "off";
-    }
-
-    /**
-     *
-     * @return Whether or not REST is enabled.
-     */
-    private boolean isRestEnabled() {
-        assert cfg != null;
-
-        return cfg.getClientConnectionConfiguration() != null;
-    }
-
-    /**
-     * Acks remote management.
-     */
-    private void ackRemoteManagement() {
-        assert log != null;
-
-        if (!log.isInfoEnabled())
-            return;
-
-        SB sb = new SB();
-
-        sb.a("Remote Management [");
-
-        boolean on = isJmxRemoteEnabled();
-
-        sb.a("restart: ").a(onOff(isRestartEnabled())).a(", ");
-        sb.a("REST: ").a(onOff(isRestEnabled())).a(", ");
-        sb.a("JMX (");
-        sb.a("remote: ").a(onOff(on));
-
-        if (on) {
-            sb.a(", ");
-
-            sb.a("port: ").a(System.getProperty("com.sun.management.jmxremote.port", "<n/a>")).a(", ");
-            sb.a("auth: ").a(onOff(Boolean.getBoolean("com.sun.management.jmxremote.authenticate"))).a(", ");
-
-            // By default SSL is enabled, that's why additional check for null is needed.
-            // See http://docs.oracle.com/javase/6/docs/technotes/guides/management/agent.html
-            sb.a("ssl: ").a(onOff(Boolean.getBoolean("com.sun.management.jmxremote.ssl") ||
-                System.getProperty("com.sun.management.jmxremote.ssl") == null));
-        }
-
-        sb.a(")");
-
-        sb.a(']');
-
-        log.info(sb.toString());
-    }
-
-    /**
-     * Acks configuration URL.
-     */
-    private void ackConfigUrl() {
-        assert log != null;
-
-        if (log.isInfoEnabled())
-            log.info("Config URL: " + System.getProperty(GG_CONFIG_URL, "n/a"));
-    }
-
-    /**
-     * Acks Visor instructions.
-     */
-    private void ackVisor() {
-        assert log != null;
-
-        if (isDaemon())
-            return;
-
-        if (ctx.isEnterprise())
-            U.quietAndInfo(log, "To start GUI Management & Monitoring run ggvisorui.{sh|bat}");
-        else
-            U.quietAndInfo(log, "To start Console Management & Monitoring run ggvisorcmd.{sh|bat}");
-    }
-
-    /**
-     * Acks benchmarking instructions.
-     */
-    private void ackBenchmarks() {
-        if (!isDaemon())
-            U.quietAndInfo(log, "If running benchmarks, see http://bit.ly/GridGain-Benchmarking");
-    }
-
-    /**
-     * Acks ASCII-logo. Thanks to http://patorjk.com/software/taag
-     */
-    private void ackAsciiLogo() {
-        assert log != null;
-
-        String fileName = log.fileName();
-
-        if (System.getProperty(GG_NO_ASCII) == null) {
-            String ver = "ver. " + ACK_VER;
-
-            // Big thanks to: http://patorjk.com/software/taag
-            // Font name "Small Slant"
-            if (log.isQuiet()) {
-                U.quiet(false,
-                    "   __________  ________________ ",
-                    "  /  _/ ___/ |/ /  _/_  __/ __/ ",
-                    " _/ // (_ /    // /  / / / _/   ",
-                    "/___/\\___/_/|_/___/ /_/ /___/  ",
-                    " ",
-                    ver,
-                    COPYRIGHT,
-                    "",
-                    "Quiet mode.");
-
-                if (fileName != null)
-                    U.quiet(false, "  ^-- Logging to file '" +  fileName + '\'');
-
-                U.quiet(false,
-                    "  ^-- To see **FULL** console log here add -DGRIDGAIN_QUIET=false or \"-v\" to ggstart.{sh|bat}",
-                    "");
-            }
-
-            if (log.isInfoEnabled()) {
-                log.info(NL + NL +
-                    ">>>    __________  ________________  " + NL +
-                    ">>>   /  _/ ___/ |/ /  _/_  __/ __/  " + NL +
-                    ">>>  _/ // (_ /    // /  / / / _/    " + NL +
-                    ">>> /___/\\___/_/|_/___/ /_/ /___/   " + NL +
-                    ">>> " + NL +
-                    ">>> " + ver + NL +
-                    ">>> " + COPYRIGHT + NL
-                );
-            }
-        }
-    }
-
-    /**
-     * Prints start info.
-     *
-     * @param rtBean Java runtime bean.
-     */
-    private void ackStart(RuntimeMXBean rtBean) {
-        if (log.isQuiet()) {
-            U.quiet(false, "");
-            U.quiet(false, "GridGain node started OK (id=" + U.id8(localNode().id()) +
-                (F.isEmpty(gridName) ? "" : ", grid=" + gridName) + ')');
-        }
-
-        if (log.isInfoEnabled()) {
-            log.info("");
-
-            String ack = "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH;
-
-            String dash = U.dash(ack.length());
-
-            SB sb = new SB();
-
-            for (GridPortRecord rec : ctx.ports().records())
-                sb.a(rec.protocol()).a(":").a(rec.port()).a(" ");
-
-            String str =
-                NL + NL +
-                    ">>> " + dash + NL +
-                    ">>> " + ack + NL +
-                    ">>> " + dash + NL +
-                    ">>> OS name: " + U.osString() + NL +
-                    ">>> CPU(s): " + localNode().metrics().getTotalCpus() + NL +
-                    ">>> Heap: " + U.heapSize(localNode(), 2) + "GB" + NL +
-                    ">>> VM name: " + rtBean.getName() + NL +
-                    ">>> Grid name: " + gridName + NL +
-                    ">>> Local node [" +
-                    "ID=" + localNode().id().toString().toUpperCase() +
-                    ", order=" + localNode().order() +
-                    "]" + NL +
-                    ">>> Local node addresses: " + U.addressesAsString(localNode()) + NL +
-                    ">>> Local ports: " + sb + NL;
-
-            str += ">>> GridGain documentation: http://" + SITE + "/documentation" + NL;
-
-            log.info(str);
-        }
-    }
-
-    /**
-     * Logs out OS information.
-     */
-    private void ackOsInfo() {
-        assert log != null;
-
-        if (log.isInfoEnabled()) {
-            log.info("OS: " + U.osString());
-            log.info("OS user: " + System.getProperty("user.name"));
-        }
-    }
-
-    /**
-     * Logs out language runtime.
-     */
-    private void ackLanguageRuntime() {
-        assert log != null;
-
-        if (log.isInfoEnabled()) {
-            log.info("Language runtime: " + getLanguage());
-            log.info("VM information: " + U.jdkString());
-            log.info("VM total memory: " + U.heapSize(2) + "GB");
-        }
-    }
-
-    /**
-     * @return Language runtime.
-     */
-    @SuppressWarnings("ThrowableInstanceNeverThrown")
-    private String getLanguage() {
-        boolean scala = false;
-        boolean groovy = false;
-        boolean clojure = false;
-
-        for (StackTraceElement elem : Thread.currentThread().getStackTrace()) {
-            String s = elem.getClassName().toLowerCase();
-
-            if (s.contains("scala")) {
-                scala = true;
-
-                break;
-            }
-            else if (s.contains("groovy")) {
-                groovy = true;
-
-                break;
-            }
-            else if (s.contains("clojure")) {
-                clojure = true;
-
-                break;
-            }
-        }
-
-        if (scala) {
-            try (InputStream in = getClass().getResourceAsStream("/library.properties")) {
-                Properties props = new Properties();
-
-                if (in != null)
-                    props.load(in);
-
-                return "Scala ver. " + props.getProperty("version.number", "<unknown>");
-            }
-            catch (Throwable ignore) {
-                return "Scala ver. <unknown>";
-            }
-        }
-
-        // How to get Groovy and Clojure version at runtime?!?
-        return groovy ? "Groovy" : clojure ? "Clojure" : U.jdkName() + " ver. " + U.jdkVersion();
-    }
-
-    /**
-     * Stops grid instance.
-     *
-     * @param cancel Whether or not to cancel running jobs.
-     */
-    public void stop(boolean cancel) {
-        // Make sure that thread stopping grid is not interrupted.
-        boolean interrupted = Thread.interrupted();
-
-        try {
-            stop0(cancel);
-        }
-        finally {
-            if (interrupted)
-                Thread.currentThread().interrupt();
-        }
-    }
-
-    /**
-     * @param cancel Whether or not to cancel running jobs.
-     */
-    private void stop0(boolean cancel) {
-        String nid = getLocalNodeId().toString().toUpperCase();
-        String nid8 = U.id8(getLocalNodeId()).toUpperCase();
-
-        gw.compareAndSet(null, new GridKernalGatewayImpl(gridName));
-
-        GridKernalGateway gw = this.gw.get();
-
-        if (stopGuard.compareAndSet(false, true)) {
-            // Only one thread is allowed to perform stop sequence.
-            boolean firstStop = false;
-
-            GridKernalState state = gw.getState();
-
-            if (state == STARTED)
-                firstStop = true;
-            else if (state == STARTING)
-                U.warn(log, "Attempt to stop starting grid. This operation " +
-                    "cannot be guaranteed to be successful.");
-
-            if (firstStop) {
-                // Notify lifecycle beans.
-                if (log.isDebugEnabled())
-                    log.debug("Notifying lifecycle beans.");
-
-                notifyLifecycleBeansEx(LifecycleEventType.BEFORE_GRID_STOP);
-            }
-
-            IgniteEmailProcessorAdapter email = ctx.email();
-
-            List<GridComponent> comps = ctx.components();
-
-            // Callback component in reverse order while kernal is still functional
-            // if called in the same thread, at least.
-            for (ListIterator<GridComponent> it = comps.listIterator(comps.size()); it.hasPrevious();) {
-                GridComponent comp = it.previous();
-
-                try {
-                    comp.onKernalStop(cancel);
-                }
-                catch (Throwable e) {
-                    errOnStop = true;
-
-                    U.error(log, "Failed to pre-stop processor: " + comp, e);
-                }
-            }
-
-            // Cancel update notification timer.
-            if (updateNtfTimer != null)
-                updateNtfTimer.cancel();
-
-            if (starveTimer != null)
-                starveTimer.cancel();
-
-            // Cancel license timer.
-            if (licTimer != null)
-                licTimer.cancel();
-
-            // Cancel metrics log timer.
-            if (metricsLogTimer != null)
-                metricsLogTimer.cancel();
-
-            gw.writeLock();
-
-            try {
-                assert gw.getState() == STARTED || gw.getState() == STARTING;
-
-                // No more kernal calls from this point on.
-                gw.setState(STOPPING);
-
-                // Clear node local store.
-                nodeLoc.clear();
-
-                if (log.isDebugEnabled())
-                    log.debug("Grid " + (gridName == null ? "" : '\'' + gridName + "' ") + "is stopping.");
-            }
-            finally {
-                gw.writeUnlock();
-            }
-
-            // Unregister MBeans.
-            if (!(
-                unregisterMBean(pubExecSvcMBean) &
-                    unregisterMBean(sysExecSvcMBean) &
-                    unregisterMBean(mgmtExecSvcMBean) &
-                    unregisterMBean(p2PExecSvcMBean) &
-                    unregisterMBean(kernalMBean) &
-                    unregisterMBean(locNodeMBean) &
-                    unregisterMBean(restExecSvcMBean)
-            ))
-                errOnStop = false;
-
-            // Stop components in reverse order.
-            for (ListIterator<GridComponent> it = comps.listIterator(comps.size()); it.hasPrevious();) {
-                GridComponent comp = it.previous();
-
-                try {
-                    comp.stop(cancel);
-
-                    if (log.isDebugEnabled())
-                        log.debug("Component stopped: " + comp);
-                }
-                catch (Throwable e) {
-                    errOnStop = true;
-
-                    U.error(log, "Failed to stop component (ignoring): " + comp, e);
-                }
-            }
-
-            // Stops lifecycle aware components.
-            U.stopLifecycleAware(log, lifecycleAwares(cfg));
-
-            // Lifecycle notification.
-            notifyLifecycleBeansEx(LifecycleEventType.AFTER_GRID_STOP);
-
-            // Clean internal class/classloader caches to avoid stopped contexts held in memory.
-            IgniteOptimizedMarshaller.clearCache();
-            IgniteMarshallerExclusions.clearCache();
-            GridEnumCache.clear();
-
-            gw.writeLock();
-
-            try {
-                gw.setState(STOPPED);
-            }
-            finally {
-                gw.writeUnlock();
-            }
-
-            // Ack stop.
-            if (log.isQuiet()) {
-                if (!errOnStop)
-                    U.quiet(false, "GridGain node stopped OK [uptime=" +
-                        X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + ']');
-                else
-                    U.quiet(true, "GridGain node stopped wih ERRORS [uptime=" +
-                        X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + ']');
-            }
-
-            if (log.isInfoEnabled())
-                if (!errOnStop) {
-                    String ack = "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH +
-                        " stopped OK";
-
-                    String dash = U.dash(ack.length());
-
-                    log.info(NL + NL +
-                        ">>> " + dash + NL +
-                        ">>> " + ack + NL +
-                        ">>> " + dash + NL +
-                        ">>> Grid name: " + gridName + NL +
-                        ">>> Grid uptime: " + X.timeSpan2HMSM(U.currentTimeMillis() - startTime) +
-                        NL +
-                        NL);
-                }
-                else {
-                    String ack = "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH +
-                        " stopped with ERRORS";
-
-                    String dash = U.dash(ack.length());
-
-                    log.info(NL + NL +
-                        ">>> " + ack + NL +
-                        ">>> " + dash + NL +
-                        ">>> Grid name: " + gridName + NL +
-                        ">>> Grid uptime: " + X.timeSpan2HMSM(U.currentTimeMillis() - startTime) +
-                        NL +
-                        ">>> See log above for detailed error message." + NL +
-                        ">>> Note that some errors during stop can prevent grid from" + NL +
-                        ">>> maintaining correct topology since this node may have" + NL +
-                        ">>> not exited grid properly." + NL +
-                        NL);
-                }
-
-            // Send node start email notification, if enabled.
-            if (isSmtpEnabled() && isAdminEmailsSet() && cfg.isLifeCycleEmailNotification()) {
-                String errOk = errOnStop ? "with ERRORS" : "OK";
-
-                String headline = "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR +
-                    " stopped " + errOk + ":";
-                String subj = "GridGain node stopped " + errOk + ": " + nid8;
-
-                IgniteProductLicense lic = ctx.license() != null ? ctx.license().license() : null;
-
-                String body =
-                    headline + NL + NL +
-                    "----" + NL +
-                    "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH + NL +
-                    "Grid name: " + gridName + NL +
-                    "Node ID: " + nid + NL +
-                    "Node uptime: " + X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + NL;
-
-                if (lic != null) {
-                    body +=
-                        "License ID: " + lic.id().toString().toUpperCase() + NL +
-                        "Licensed to: " + lic.userOrganization() + NL;
-                }
-                else
-                    assert !ENT;
-
-                body +=
-                    "----" + NL +
-                    NL +
-                    "NOTE:" + NL +
-                    "This message is sent automatically to all configured admin emails." + NL +
-                    "To change this behavior use 'lifeCycleEmailNotify' grid configuration property.";
-
-                if (errOnStop)
-                    body +=
-                        NL + NL +
-                            "NOTE:" + NL +
-                            "See node's log for detailed error message." + NL +
-                            "Some errors during stop can prevent grid from" + NL +
-                            "maintaining correct topology since this node may " + NL +
-                            "have not exited grid properly.";
-
-                body +=
-                    NL + NL +
-                        "| " + SITE + NL +
-                        "| support@gridgain.com" + NL;
-
-                if (email != null) {
-                    try {
-                        email.sendNow(subj,
-                            body,
-                            false,
-                            Arrays.asList(cfg.getAdminEmails()));
-                    }
-                    catch (IgniteCheckedException e) {
-                        U.error(log, "Failed to send lifecycle email notification.", e);
-                    }
-                }
-            }
-
-            U.onGridStop();
-        }
-        else {
-            // Proper notification.
-            if (log.isDebugEnabled()) {
-                if (gw.getState() == STOPPED)
-                    log.debug("Grid is already stopped. Nothing to do.");
-                else
-                    log.debug("Grid is being stopped by another thread. Aborting this stop sequence " +
-                        "allowing other thread to finish.");
-            }
-        }
-    }
-
-    /**
-     * USED ONLY FOR TESTING.
-     *
-     * @param <K> Key type.
-     * @param <V> Value type.
-     * @return Internal cache instance.
-     */
-    /*@java.test.only*/
-    public <K, V> GridCacheAdapter<K, V> internalCache() {
-        return internalCache(null);
-    }
-
-    /**
-     * USED ONLY FOR TESTING.
-     *
-     * @param name Cache name.
-     * @param <K>  Key type.
-     * @param <V>  Value type.
-     * @return Internal cache instance.
-     */
-    /*@java.test.only*/
-    public <K, V> GridCacheAdapter<K, V> internalCache(@Nullable String name) {
-        return ctx.cache().internalCache(name);
-    }
-
-    /**
-     * It's intended for use by internal marshalling implementation only.
-     *
-     * @return Kernal context.
-     */
-    public GridKernalContext context() {
-        return ctx;
-    }
-
-    /**
-     * Prints all system properties in debug mode.
-     */
-    private void ackSystemProperties() {
-        assert log != null;
-
-        if (log.isDebugEnabled())
-            for (Object key : U.asIterable(System.getProperties().keys()))
-                log.debug("System property [" + key + '=' + System.getProperty((String) key) + ']');
-    }
-
-    /**
-     * Prints all user attributes in info mode.
-     */
-    private void logNodeUserAttributes() {
-        assert log != null;
-
-        if (log.isInfoEnabled())
-            for (Map.Entry<?, ?> attr : cfg.getUserAttributes().entrySet())
-                log.info("Local node user attribute [" + attr.getKey() + '=' + attr.getValue() + ']');
-    }
-
-    /**
-     * Prints all environment variables in debug mode.
-     */
-    private void ackEnvironmentVariables() {
-        assert log != null;
-
-        if (log.isDebugEnabled())
-            for (Map.Entry<?, ?> envVar : System.getenv().entrySet())
-                log.debug("Environment variable [" + envVar.getKey() + '=' + envVar.getValue() + ']');
-    }
-
-    /**
-     * Acks daemon mode status.
-     */
-    private void ackDaemon() {
-        assert log != null;
-
-        if (log.isInfoEnabled())
-            log.info("Daemon mode: " + (isDaemon() ? "on" : "off"));
-    }
-
-    /**
-     *
-     * @return {@code True} is this node is daemon.
-     */
-    private boolean isDaemon() {
-        assert cfg != null;
-
-        return cfg.isDaemon() || "true".equalsIgnoreCase(System.getProperty(GG_DAEMON));
-    }
-
-    /**
-     * Whether or not remote JMX management is enabled for this node. Remote JMX management is
-     * enabled when the following system property is set:
-     * <ul>
-     *     <li>{@code com.sun.management.jmxremote}</li>
-     * </ul>
-     *
-     * @return {@code True} if remote JMX management is enabled - {@code false} otherwise.
-     */
-    @Override public boolean isJmxRemoteEnabled() {
-        return System.getProperty("com.sun.management.jmxremote") != null;
-    }
-
-    /**
-     * Whether or not node restart is enabled. Node restart us supported when this node was started
-     * with {@code bin/ggstart.{sh|bat}} script using {@code -r} argument. Node can be
-     * programmatically restarted using {@link org.apache.ignite.Ignition#restart(boolean)}} method.
-     *
-     * @return {@code True} if restart mode is enabled, {@code false} otherwise.
-     * @see org.apache.ignite.Ignition#restart(boolean)
-     */
-    @Override public boolean isRestartEnabled() {
-        return System.getProperty(GG_SUCCESS_FILE) != null;
-    }
-
-    /**
-     * Whether or not SMTP is configured. Note that SMTP is considered configured if
-     * SMTP host is provided in configuration (see {@link org.apache.ignite.configuration.IgniteConfiguration#getSmtpHost()}.
-     * <p>
-     * If SMTP is not configured all emails notifications will be disabled.
-     *
-     * @return {@code True} if SMTP is configured - {@code false} otherwise.
-     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpFromEmail()
-     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpHost()
-     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpPassword()
-     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpPort()
-     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpUsername()
-     * @see org.apache.ignite.configuration.IgniteConfiguration#isSmtpSsl()
-     * @see org.apache.ignite.configuration.IgniteConfiguration#isSmtpStartTls()
-     * @see #sendAdminEmailAsync(String, String, boolean)
-     */
-    @Override public boolean isSmtpEnabled() {
-        assert cfg != null;
-
-        return cfg.getSmtpHost() != null;
-    }
-
-    /**
-     * Prints all configuration properties in info mode and SPIs in debug mode.
-     */
-    private void ackSpis() {
-        assert log != null;
-
-        if (log.isDebugEnabled()) {
-            log.debug("+-------------+");
-            log.debug("START SPI LIST:");
-            log.debug("+-------------+");
-            log.debug("Grid checkpoint SPI     : " + Arrays.toString(cfg.getCheckpointSpi()));
-            log.debug("Grid collision SPI      : " + cfg.getCollisionSpi());
-            log.debug("Grid communication SPI  : " + cfg.getCommunicationSpi());
-            log.debug("Grid deployment SPI     : " + cfg.getDeploymentSpi());
-            log.debug("Grid discovery SPI      : " + cfg.getDiscoverySpi());
-            log.debug("Grid event storage SPI  : " + cfg.getEventStorageSpi());
-            log.debug("Grid failover SPI       : " + Arrays.toString(cfg.getFailoverSpi()));
-            log.debug("Grid load balancing SPI : " + Arrays.toString(cfg.getLoadBalancingSpi()));
-            log.debug("Grid authentication SPI : " + cfg.getAuthenticationSpi());
-            log.debug("Grid secure session SPI : " + cfg.getSecureSessionSpi());
-            log.debug("Grid swap space SPI     : " + cfg.getSwapSpaceSpi());
-        }
-    }
-
-    /**
-     *
-     */
-    private void ackCacheConfiguration() {
-        CacheConfiguration[] cacheCfgs = cfg.getCacheConfiguration();
-
-        if (cacheCfgs == null || cacheCfgs.length == 0)
-            U.warn(log, "Cache is not configured - in-memory data grid is off.");
-        else {
-            SB sb = new SB();
-
-            for (CacheConfiguration c : cacheCfgs) {
-                String name = c.getName();
-
-                if (name == null)
-                    name = "<default>";
-
-                sb.a("'").a(name).a("', ");
-            }
-
-            String names = sb.toString();
-
-            U.log(log, "Configured caches [" + names.substring(0, names.length() - 2) + ']');
-        }
-    }
-
-    /**
-     *
-     */
-    private void ackP2pConfiguration() {
-        assert cfg != null;
-
-        if (cfg.isPeerClassLoadingEnabled())
-            U.warn(
-                log,
-                "Peer class loading is enabled (disable it in production for performance and " +
-                    "deployment consistency reasons)",
-                "Peer class loading is enabled (disable it for better performance)"
-            );
-    }
-
-    /**
-     * Prints security status.
-     *
-     * @param ctx Kernal context.
-     */
-    private void ackSecurity(GridKernalContext ctx) {
-        assert log != null;
-
-        if (log.isInfoEnabled())
-            log.info("Security status [authentication=" + onOff(ctx.security().enabled()) + ", " +
-                "secure-session=" + onOff(ctx.secureSession().enabled()) + ']');
-    }
-
-    /**
-     * Prints out SMTP configuration.
-     */
-    private void ackSmtpConfiguration() {
-        assert log != null;
-
-        String host = cfg.getSmtpHost();
-
-        boolean ssl = cfg.isSmtpSsl();
-        int port = cfg.getSmtpPort();
-
-        if (host == null) {
-            U.warn(log, "SMTP is not configured - email notifications are off.");
-
-            return;
-        }
-
-        String from = cfg.getSmtpFromEmail();
-
-        if (log.isQuiet())
-            U.quiet(false, "SMTP enabled [host=" + host + ":" + port + ", ssl=" + (ssl ? "on" : "off") + ", from=" +
-                from + ']');
-
-        if (log.isInfoEnabled()) {
-            String[] adminEmails = cfg.getAdminEmails();
-
-            log.info("SMTP enabled [host=" + host + ", port=" + port + ", ssl=" + ssl + ", from=" + from + ']');
-            log.info("Admin emails: " + (!isAdminEmailsSet() ? "N/A" : Arrays.toString(adminEmails)));
-        }
-
-        if (!isAdminEmailsSet

<TRUNCATED>

[39/50] [abbrv] incubator-ignite git commit: GridEx -> IgniteEx GridKernal -> IgniteKernal

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index a7dd3aa..17f3ec8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -138,7 +138,7 @@ public interface GridKernalContext extends Iterable<GridComponent> {
      *
      * @return Grid instance.
      */
-    public GridEx grid();
+    public IgniteEx grid();
 
     /**
      * Gets grid configuration.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 8bb0085..776cda6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -266,7 +266,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     private List<GridComponent> comps = new LinkedList<>();
 
     /** */
-    private GridEx grid;
+    private IgniteEx grid;
 
     /** */
     private ExecutorService utilityCachePool;
@@ -320,7 +320,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
      */
     @SuppressWarnings("TypeMayBeWeakened")
     protected GridKernalContextImpl(GridLoggerProxy log,
-        GridEx grid,
+        IgniteEx grid,
         IgniteConfiguration cfg,
         GridKernalGateway gw,
         ExecutorService utilityCachePool,
@@ -496,7 +496,7 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
-    @Override public GridEx grid() {
+    @Override public IgniteEx grid() {
         return grid;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java b/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
index 9720a60..4bfeb5c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridUpdateNotifier.java
@@ -219,7 +219,7 @@ class GridUpdateNotifier {
                     throttle(log, false, "Your version is up to date.");
             }
             else
-                throttle(log, true, "New version is available at " + GridKernal.SITE + ": " + latestVer);
+                throttle(log, true, "New version is available at " + IgniteKernal.SITE + ": " + latestVer);
         else
             if (!reportOnlyNew)
                 throttle(log, false, "Update status is not available.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
index a02c931..ae4b541 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteClusterAsyncImpl.java
@@ -31,12 +31,12 @@ import java.util.*;
  */
 public class IgniteClusterAsyncImpl extends IgniteAsyncSupportAdapter<IgniteCluster> implements IgniteCluster {
     /** */
-    private final GridKernal grid;
+    private final IgniteKernal grid;
 
     /**
      * @param grid Grid.
      */
-    public IgniteClusterAsyncImpl(GridKernal grid) {
+    public IgniteClusterAsyncImpl(IgniteKernal grid) {
         super(true);
 
         this.grid = grid;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
new file mode 100644
index 0000000..6aa8bf5
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -0,0 +1,143 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.processors.interop.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+/**
+ * Extended Grid interface which provides some additional methods required for kernal and Visor.
+ */
+public interface IgniteEx extends Ignite, ClusterGroupEx, IgniteCluster {
+    /**
+     * Gets utility cache.
+     *
+     * @param keyCls Key class.
+     * @param valCls Value class.
+     * @return Utility cache.
+     */
+    public <K extends GridCacheUtilityKey, V> GridCacheProjectionEx<K, V> utilityCache(Class<K> keyCls, Class<V> valCls);
+
+    /**
+     * Gets the cache instance for the given name if one is configured or
+     * <tt>null</tt> otherwise returning even non-public caches.
+     *
+     * @param <K> Key type.
+     * @param <V> Value type.
+     * @param name Cache name.
+     * @return Cache instance for given name or <tt>null</tt> if one does not exist.
+     */
+    @Nullable public <K, V> GridCache<K, V> cachex(@Nullable String name);
+
+    /**
+     * Gets default cache instance if one is configured or <tt>null</tt> otherwise returning even non-public caches.
+     * The {@link org.apache.ignite.cache.GridCache#name()} method on default instance returns <tt>null</tt>.
+     *
+     * @param <K> Key type.
+     * @param <V> Value type.
+     * @return Default cache instance.
+     */
+    @Nullable public <K, V> GridCache<K, V> cachex();
+
+    /**
+     * Gets configured cache instance that satisfy all provided predicates including non-public caches. If no
+     * predicates provided - all configured caches will be returned.
+     *
+     * @param p Predicates. If none provided - all configured caches will be returned.
+     * @return Configured cache instances that satisfy all provided predicates.
+     */
+    public Collection<GridCache<?, ?>> cachesx(@Nullable IgnitePredicate<? super GridCache<?, ?>>... p);
+
+    /**
+     * Checks if the event type is user-recordable.
+     *
+     * @param type Event type to check.
+     * @return {@code true} if passed event should be recorded, {@code false} - otherwise.
+     */
+    public boolean eventUserRecordable(int type);
+
+    /**
+     * Checks whether all provided events are user-recordable.
+     * <p>
+     * Note that this method supports only predefined GridGain events.
+     *
+     * @param types Event types.
+     * @return Whether all events are recordable.
+     * @throws IllegalArgumentException If {@code types} contains user event type.
+     */
+    public boolean allEventsUserRecordable(int[] types);
+
+    /**
+     * Gets list of compatible versions.
+     *
+     * @return Compatible versions.
+     */
+    public Collection<String> compatibleVersions();
+
+    /**
+     * @return Grace period left in minutes if bursting or {@code -1} otherwise.
+     */
+    public long licenseGracePeriodLeft();
+
+    /**
+     * Whether or not remote JMX management is enabled for this node.
+     *
+     * @return {@code True} if remote JMX management is enabled - {@code false} otherwise.
+     */
+    public boolean isJmxRemoteEnabled();
+
+    /**
+     * Whether or not node restart is enabled.
+     *
+     * @return {@code True} if restart mode is enabled, {@code false} otherwise.
+     */
+    public boolean isRestartEnabled();
+
+    /**
+     * Whether or not SMTP is configured.
+     *
+     * @return {@code True} if SMTP is configured - {@code false} otherwise.
+     */
+    public boolean isSmtpEnabled();
+
+    /**
+     * Schedule sending of given email to all configured admin emails.
+     */
+    IgniteInternalFuture<Boolean> sendAdminEmailAsync(String subj, String body, boolean html);
+
+    /**
+     * Get GGFS instance returning null if it doesn't exist.
+     *
+     * @param name GGFS name.
+     * @return GGFS.
+     */
+    @Nullable public IgniteFs ggfsx(@Nullable String name);
+
+    /**
+     * Gets interop processor.
+     *
+     * @return Interop processor.
+     */
+    public GridInteropProcessor interop();
+}


[43/50] [abbrv] incubator-ignite git commit: GridNamedInstance -> IgniteNamedInstance GridgainEx -> IgnitionEx

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
new file mode 100644
index 0000000..acab6d0
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -0,0 +1,2399 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.rendezvous.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.fs.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.logger.*;
+import org.apache.ignite.logger.java.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.marshaller.jdk.*;
+import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.mxbean.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.authentication.*;
+import org.apache.ignite.spi.authentication.noop.*;
+import org.apache.ignite.spi.indexing.*;
+import org.apache.ignite.streamer.*;
+import org.apache.ignite.thread.*;
+import org.apache.ignite.internal.processors.resource.*;
+import org.apache.ignite.internal.processors.spring.*;
+import org.apache.ignite.plugin.segmentation.*;
+import org.apache.ignite.spi.checkpoint.*;
+import org.apache.ignite.spi.checkpoint.noop.*;
+import org.apache.ignite.spi.collision.*;
+import org.apache.ignite.spi.collision.noop.*;
+import org.apache.ignite.spi.communication.*;
+import org.apache.ignite.spi.communication.tcp.*;
+import org.apache.ignite.spi.deployment.*;
+import org.apache.ignite.spi.deployment.local.*;
+import org.apache.ignite.spi.discovery.*;
+import org.apache.ignite.spi.discovery.tcp.*;
+import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
+import org.apache.ignite.spi.eventstorage.*;
+import org.apache.ignite.spi.eventstorage.memory.*;
+import org.apache.ignite.spi.failover.*;
+import org.apache.ignite.spi.failover.always.*;
+import org.apache.ignite.spi.loadbalancing.*;
+import org.apache.ignite.spi.loadbalancing.roundrobin.*;
+import org.apache.ignite.spi.securesession.*;
+import org.apache.ignite.spi.securesession.noop.*;
+import org.apache.ignite.spi.swapspace.*;
+import org.apache.ignite.spi.swapspace.file.*;
+import org.apache.ignite.spi.swapspace.noop.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jdk8.backport.*;
+import org.jetbrains.annotations.*;
+
+import javax.management.*;
+import java.io.*;
+import java.lang.management.*;
+import java.lang.reflect.*;
+import java.net.*;
+import java.util.*;
+import java.util.Map.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+import java.util.logging.*;
+
+import static org.apache.ignite.configuration.IgniteConfiguration.*;
+import static org.apache.ignite.IgniteState.*;
+import static org.apache.ignite.IgniteSystemProperties.*;
+import static org.apache.ignite.cache.CacheAtomicityMode.*;
+import static org.apache.ignite.cache.CacheDistributionMode.*;
+import static org.apache.ignite.cache.CacheMode.*;
+import static org.apache.ignite.cache.CachePreloadMode.*;
+import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
+import static org.apache.ignite.internal.IgniteComponentType.*;
+import static org.apache.ignite.plugin.segmentation.GridSegmentationPolicy.*;
+
+/**
+ * This class defines a factory for the main GridGain API. It controls Grid life cycle
+ * and allows listening for grid events.
+ * <h1 class="header">Grid Loaders</h1>
+ * Although user can apply grid factory directly to start and stop grid, grid is
+ * often started and stopped by grid loaders. Grid loaders can be found in
+ * {@link org.apache.ignite.startup} package, for example:
+ * <ul>
+ * <li>{@code GridCommandLineStartup}</li>
+ * <li>{@code GridServletStartup}</li>
+ * </ul>
+ * <h1 class="header">Examples</h1>
+ * Use {@link #start()} method to start grid with default configuration. You can also use
+ * {@link org.apache.ignite.configuration.IgniteConfiguration} to override some default configuration. Below is an
+ * example on how to start grid with <strong>URI deployment</strong>.
+ * <pre name="code" class="java">
+ * GridConfiguration cfg = new GridConfiguration();
+ */
+public class IgnitionEx {
+    /** Default configuration path relative to GridGain home. */
+    public static final String DFLT_CFG = "config/default-config.xml";
+
+    /** Map of named grids. */
+    private static final ConcurrentMap<Object, IgniteNamedInstance> grids = new ConcurrentHashMap8<>();
+
+    /** Map of grid states ever started in this JVM. */
+    private static final Map<Object, IgniteState> gridStates = new ConcurrentHashMap8<>();
+
+    /** Mutex to synchronize updates of default grid reference. */
+    private static final Object dfltGridMux = new Object();
+
+    /** Default grid. */
+    private static volatile IgniteNamedInstance dfltGrid;
+
+    /** Default grid state. */
+    private static volatile IgniteState dfltGridState;
+
+    /** List of state listeners. */
+    private static final Collection<IgniteListener> lsnrs = new GridConcurrentHashSet<>(4);
+
+    /** */
+    private static volatile boolean daemon;
+
+    /**
+     * Checks runtime version to be 1.7.x or 1.8.x.
+     * This will load pretty much first so we must do these checks here.
+     */
+    static {
+        // Check 1.8 just in case for forward compatibility.
+        if (!U.jdkVersion().contains("1.7") &&
+            !U.jdkVersion().contains("1.8"))
+            throw new IllegalStateException("GridGain requires Java 7 or above. Current Java version " +
+                "is not supported: " + U.jdkVersion());
+
+        // To avoid nasty race condition in UUID.randomUUID() in JDK prior to 6u34.
+        // For details please see:
+        // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=7071826
+        // http://www.oracle.com/technetwork/java/javase/2col/6u34-bugfixes-1733379.html
+        // http://hg.openjdk.java.net/jdk6/jdk6/jdk/rev/563d392b3e5c
+        UUID.randomUUID();
+    }
+
+    /**
+     * Enforces singleton.
+     */
+    private IgnitionEx() {
+        // No-op.
+    }
+
+    /**
+     * Sets daemon flag.
+     * <p>
+     * If daemon flag is set then all grid instances created by the factory will be
+     * daemon, i.e. the local node for these instances will be a daemon node. Note that
+     * if daemon flag is set - it will override the same settings in {@link org.apache.ignite.configuration.IgniteConfiguration#isDaemon()}.
+     * Note that you can set on and off daemon flag at will.
+     *
+     * @param daemon Daemon flag to set.
+     */
+    public static void setDaemon(boolean daemon) {
+        IgnitionEx.daemon = daemon;
+    }
+
+    /**
+     * Gets daemon flag.
+     * <p>
+     * If daemon flag it set then all grid instances created by the factory will be
+     * daemon, i.e. the local node for these instances will be a daemon node. Note that
+     * if daemon flag is set - it will override the same settings in {@link org.apache.ignite.configuration.IgniteConfiguration#isDaemon()}.
+     * Note that you can set on and off daemon flag at will.
+     *
+     * @return Daemon flag.
+     */
+    public static boolean isDaemon() {
+        return daemon;
+    }
+
+    /**
+     * Gets state of grid default grid.
+     *
+     * @return Default grid state.
+     */
+    public static IgniteState state() {
+        return state(null);
+    }
+
+    /**
+     * Gets states of named grid. If name is {@code null}, then state of
+     * default no-name grid is returned.
+     *
+     * @param name Grid name. If name is {@code null}, then state of
+     *      default no-name grid is returned.
+     * @return Grid state.
+     */
+    public static IgniteState state(@Nullable String name) {
+        IgniteNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
+
+        if (grid == null) {
+            IgniteState state = name != null ? gridStates.get(name) : dfltGridState;
+
+            return state != null ? state : STOPPED;
+        }
+
+        return grid.state();
+    }
+
+    /**
+     * Stops default grid. This method is identical to {@code G.stop(null, cancel)} apply.
+     * Note that method does not wait for all tasks to be completed.
+     *
+     * @param cancel If {@code true} then all jobs currently executing on
+     *      default grid will be cancelled by calling {@link org.apache.ignite.compute.ComputeJob#cancel()}
+     *      method. Note that just like with {@link Thread#interrupt()}, it is
+     *      up to the actual job to exit from execution
+     * @return {@code true} if default grid instance was indeed stopped,
+     *      {@code false} otherwise (if it was not started).
+     */
+    public static boolean stop(boolean cancel) {
+        return stop(null, cancel);
+    }
+
+    /**
+     * Stops named grid. If {@code cancel} flag is set to {@code true} then
+     * all jobs currently executing on local node will be interrupted. If
+     * grid name is {@code null}, then default no-name grid will be stopped.
+     * If wait parameter is set to {@code true} then grid will wait for all
+     * tasks to be finished.
+     *
+     * @param name Grid name. If {@code null}, then default no-name grid will
+     *      be stopped.
+     * @param cancel If {@code true} then all jobs currently will be cancelled
+     *      by calling {@link org.apache.ignite.compute.ComputeJob#cancel()} method. Note that just like with
+     *      {@link Thread#interrupt()}, it is up to the actual job to exit from
+     *      execution. If {@code false}, then jobs currently running will not be
+     *      canceled. In either case, grid node will wait for completion of all
+     *      jobs running on it before stopping.
+     * @return {@code true} if named grid instance was indeed found and stopped,
+     *      {@code false} otherwise (the instance with given {@code name} was
+     *      not found).
+     */
+    public static boolean stop(@Nullable String name, boolean cancel) {
+        IgniteNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
+
+        if (grid != null && grid.state() == STARTED) {
+            grid.stop(cancel);
+
+            boolean fireEvt;
+
+            if (name != null)
+                fireEvt = grids.remove(name, grid);
+            else {
+                synchronized (dfltGridMux) {
+                    fireEvt = dfltGrid == grid;
+
+                    if (fireEvt)
+                        dfltGrid = null;
+                }
+            }
+
+            if (fireEvt)
+                notifyStateChange(grid.getName(), grid.state());
+
+            return true;
+        }
+
+        // We don't have log at this point...
+        U.warn(null, "Ignoring stopping grid instance that was already stopped or never started: " + name);
+
+        return false;
+    }
+
+    /**
+     * Stops <b>all</b> started grids. If {@code cancel} flag is set to {@code true} then
+     * all jobs currently executing on local node will be interrupted.
+     * If wait parameter is set to {@code true} then grid will wait for all
+     * tasks to be finished.
+     * <p>
+     * <b>Note:</b> it is usually safer and more appropriate to stop grid instances individually
+     * instead of blanket operation. In most cases, the party that started the grid instance
+     * should be responsible for stopping it.
+     *
+     * @param cancel If {@code true} then all jobs currently executing on
+     *      all grids will be cancelled by calling {@link org.apache.ignite.compute.ComputeJob#cancel()}
+     *      method. Note that just like with {@link Thread#interrupt()}, it is
+     *      up to the actual job to exit from execution
+     */
+    public static void stopAll(boolean cancel) {
+        IgniteNamedInstance dfltGrid0 = dfltGrid;
+
+        if (dfltGrid0 != null) {
+            dfltGrid0.stop(cancel);
+
+            boolean fireEvt;
+
+            synchronized (dfltGridMux) {
+                fireEvt = dfltGrid == dfltGrid0;
+
+                if (fireEvt)
+                    dfltGrid = null;
+            }
+
+            if (fireEvt)
+                notifyStateChange(dfltGrid0.getName(), dfltGrid0.state());
+        }
+
+        // Stop the rest and clear grids map.
+        for (IgniteNamedInstance grid : grids.values()) {
+            grid.stop(cancel);
+
+            boolean fireEvt = grids.remove(grid.getName(), grid);
+
+            if (fireEvt)
+                notifyStateChange(grid.getName(), grid.state());
+        }
+    }
+
+    /**
+     * Restarts <b>all</b> started grids. If {@code cancel} flag is set to {@code true} then
+     * all jobs currently executing on the local node will be interrupted.
+     * If {@code wait} parameter is set to {@code true} then grid will wait for all
+     * tasks to be finished.
+     * <p>
+     * <b>Note:</b> it is usually safer and more appropriate to stop grid instances individually
+     * instead of blanket operation. In most cases, the party that started the grid instance
+     * should be responsible for stopping it.
+     * <p>
+     * Note also that restarting functionality only works with the tools that specifically
+     * support GridGain's protocol for restarting. Currently only standard <tt>ggstart.{sh|bat}</tt>
+     * scripts support restarting of JVM GridGain's process.
+     *
+     * @param cancel If {@code true} then all jobs currently executing on
+     *      all grids will be cancelled by calling {@link org.apache.ignite.compute.ComputeJob#cancel()}
+     *      method. Note that just like with {@link Thread#interrupt()}, it is
+     *      up to the actual job to exit from execution.
+     * @see org.apache.ignite.Ignition#RESTART_EXIT_CODE
+     */
+    public static void restart(boolean cancel) {
+        String file = System.getProperty(GG_SUCCESS_FILE);
+
+        if (file == null)
+            U.warn(null, "Cannot restart node when restart not enabled.");
+        else {
+            try {
+                new File(file).createNewFile();
+            }
+            catch (IOException e) {
+                U.error(null, "Failed to create restart marker file (restart aborted): " + e.getMessage());
+
+                return;
+            }
+
+            U.log(null, "Restarting node. Will exit (" + Ignition.RESTART_EXIT_CODE + ").");
+
+            // Set the exit code so that shell process can recognize it and loop
+            // the start up sequence again.
+            System.setProperty(GG_RESTART_CODE, Integer.toString(Ignition.RESTART_EXIT_CODE));
+
+            stopAll(cancel);
+
+            // This basically leaves loaders hang - we accept it.
+            System.exit(Ignition.RESTART_EXIT_CODE);
+        }
+    }
+
+    /**
+     * Stops <b>all</b> started grids. If {@code cancel} flag is set to {@code true} then
+     * all jobs currently executing on the local node will be interrupted.
+     * If {@code wait} parameter is set to {@code true} then grid will wait for all
+     * tasks to be finished.
+     * <p>
+     * <b>Note:</b> it is usually safer and more appropriate to stop grid instances individually
+     * instead of blanket operation. In most cases, the party that started the grid instance
+     * should be responsible for stopping it.
+     * <p>
+     * Note that upon completion of this method, the JVM with forcefully exist with
+     * exit code {@link org.apache.ignite.Ignition#KILL_EXIT_CODE}.
+     *
+     * @param cancel If {@code true} then all jobs currently executing on
+     *      all grids will be cancelled by calling {@link org.apache.ignite.compute.ComputeJob#cancel()}
+     *      method. Note that just like with {@link Thread#interrupt()}, it is
+     *      up to the actual job to exit from execution.
+     * @see org.apache.ignite.Ignition#KILL_EXIT_CODE
+     */
+    public static void kill(boolean cancel) {
+        stopAll(cancel);
+
+        // This basically leaves loaders hang - we accept it.
+        System.exit(Ignition.KILL_EXIT_CODE);
+    }
+
+    /**
+     * Starts grid with default configuration. By default this method will
+     * use grid configuration defined in {@code GRIDGAIN_HOME/config/default-config.xml}
+     * configuration file. If such file is not found, then all system defaults will be used.
+     *
+     * @return Started grid.
+     * @throws IgniteCheckedException If default grid could not be started. This exception will be thrown
+     *      also if default grid has already been started.
+     */
+    public static Ignite start() throws IgniteCheckedException {
+        return start((GridSpringResourceContext)null);
+    }
+
+    /**
+     * Starts grid with default configuration. By default this method will
+     * use grid configuration defined in {@code GRIDGAIN_HOME/config/default-config.xml}
+     * configuration file. If such file is not found, then all system defaults will be used.
+     *
+     * @param springCtx Optional Spring application context, possibly {@code null}.
+     *      Spring bean definitions for bean injection are taken from this context.
+     *      If provided, this context can be injected into grid tasks and grid jobs using
+     *      {@link org.apache.ignite.resources.IgniteSpringApplicationContextResource @IgniteSpringApplicationContextResource} annotation.
+     * @return Started grid.
+     * @throws IgniteCheckedException If default grid could not be started. This exception will be thrown
+     *      also if default grid has already been started.
+     */
+    public static Ignite start(@Nullable GridSpringResourceContext springCtx) throws IgniteCheckedException {
+        URL url = U.resolveGridGainUrl(DFLT_CFG);
+
+        if (url != null)
+            return start(DFLT_CFG, null, springCtx);
+
+        U.warn(null, "Default Spring XML file not found (is GRIDGAIN_HOME set?): " + DFLT_CFG);
+
+        return start0(new GridStartContext(new IgniteConfiguration(), null, springCtx)).grid();
+    }
+
+    /**
+     * Starts grid with given configuration. Note that this method is no-op if grid with the name
+     * provided in given configuration is already started.
+     *
+     * @param cfg Grid configuration. This cannot be {@code null}.
+     * @return Started grid.
+     * @throws IgniteCheckedException If grid could not be started. This exception will be thrown
+     *      also if named grid has already been started.
+     */
+    public static Ignite start(IgniteConfiguration cfg) throws IgniteCheckedException {
+        return start(cfg, null);
+    }
+
+    /**
+     * Starts grid with given configuration. Note that this method is no-op if grid with the name
+     * provided in given configuration is already started.
+     *
+     * @param cfg Grid configuration. This cannot be {@code null}.
+     * @param springCtx Optional Spring application context, possibly {@code null}.
+     *      Spring bean definitions for bean injection are taken from this context.
+     *      If provided, this context can be injected into grid tasks and grid jobs using
+     *      {@link org.apache.ignite.resources.IgniteSpringApplicationContextResource @IgniteSpringApplicationContextResource} annotation.
+     * @return Started grid.
+     * @throws IgniteCheckedException If grid could not be started. This exception will be thrown
+     *      also if named grid has already been started.
+     */
+    public static Ignite start(IgniteConfiguration cfg, @Nullable GridSpringResourceContext springCtx) throws IgniteCheckedException {
+        A.notNull(cfg, "cfg");
+
+        return start0(new GridStartContext(cfg, null, springCtx)).grid();
+    }
+
+    /**
+     * Starts all grids specified within given Spring XML configuration file. If grid with given name
+     * is already started, then exception is thrown. In this case all instances that may
+     * have been started so far will be stopped too.
+     * <p>
+     * Usually Spring XML configuration file will contain only one Grid definition. Note that
+     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
+     * the Grid configuration bean is ignored.
+     *
+     * @param springCfgPath Spring XML configuration file path or URL.
+     * @return Started grid. If Spring configuration contains multiple grid instances,
+     *      then the 1st found instance is returned.
+     * @throws IgniteCheckedException If grid could not be started or configuration
+     *      read. This exception will be thrown also if grid with given name has already
+     *      been started or Spring XML configuration file is invalid.
+     */
+    public static Ignite start(@Nullable String springCfgPath) throws IgniteCheckedException {
+        return springCfgPath == null ? start() : start(springCfgPath, null);
+    }
+
+    /**
+     * Starts all grids specified within given Spring XML configuration file. If grid with given name
+     * is already started, then exception is thrown. In this case all instances that may
+     * have been started so far will be stopped too.
+     * <p>
+     * Usually Spring XML configuration file will contain only one Grid definition. Note that
+     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
+     * the Grid configuration bean is ignored.
+     *
+     * @param springCfgPath Spring XML configuration file path or URL.
+     * @param gridName Grid name that will override default.
+     * @return Started grid. If Spring configuration contains multiple grid instances,
+     *      then the 1st found instance is returned.
+     * @throws IgniteCheckedException If grid could not be started or configuration
+     *      read. This exception will be thrown also if grid with given name has already
+     *      been started or Spring XML configuration file is invalid.
+     */
+    public static Ignite start(@Nullable String springCfgPath, @Nullable String gridName) throws IgniteCheckedException {
+        if (springCfgPath == null) {
+            IgniteConfiguration cfg = new IgniteConfiguration();
+
+            if (cfg.getGridName() == null && !F.isEmpty(gridName))
+                cfg.setGridName(gridName);
+
+            return start(cfg);
+        }
+        else
+            return start(springCfgPath, gridName, null);
+    }
+
+    /**
+     * Start Grid for interop scenario.
+     *
+     * @param springCfgPath Spring config path.
+     * @param gridName Grid name.
+     * @param cfgClo Configuration closure.
+     * @return Started Grid.
+     * @throws IgniteCheckedException If failed.
+     */
+    public static Ignite startInterop(@Nullable String springCfgPath, @Nullable String gridName,
+        IgniteClosure<IgniteConfiguration, IgniteConfiguration> cfgClo) throws IgniteCheckedException {
+        URL url = resolveSpringUrl(springCfgPath);
+
+        return start(url, gridName, null, cfgClo);
+    }
+
+    /**
+     * Loads all grid configurations specified within given Spring XML configuration file.
+     * <p>
+     * Usually Spring XML configuration file will contain only one Grid definition. Note that
+     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
+     * the Grid configuration bean is ignored.
+     *
+     * @param springCfgUrl Spring XML configuration file path or URL. This cannot be {@code null}.
+     * @return Tuple containing all loaded configurations and Spring context used to load them.
+     * @throws IgniteCheckedException If grid could not be started or configuration
+     *      read. This exception will be thrown also if grid with given name has already
+     *      been started or Spring XML configuration file is invalid.
+     */
+    public static IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> loadConfigurations(
+        URL springCfgUrl) throws IgniteCheckedException {
+        IgniteSpringProcessor spring = SPRING.create(false);
+
+        return spring.loadConfigurations(springCfgUrl);
+    }
+
+    /**
+     * Loads all grid configurations specified within given Spring XML configuration file.
+     * <p>
+     * Usually Spring XML configuration file will contain only one Grid definition. Note that
+     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
+     * the Grid configuration bean is ignored.
+     *
+     * @param springCfgPath Spring XML configuration file path. This cannot be {@code null}.
+     * @return Tuple containing all loaded configurations and Spring context used to load them.
+     * @throws IgniteCheckedException If grid could not be started or configuration
+     *      read. This exception will be thrown also if grid with given name has already
+     *      been started or Spring XML configuration file is invalid.
+     */
+    public static IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> loadConfigurations(
+        String springCfgPath) throws IgniteCheckedException {
+        A.notNull(springCfgPath, "springCfgPath");
+
+        URL url;
+
+        try {
+            url = new URL(springCfgPath);
+        }
+        catch (MalformedURLException e) {
+            url = U.resolveGridGainUrl(springCfgPath);
+
+            if (url == null)
+                throw new IgniteCheckedException("Spring XML configuration path is invalid: " + springCfgPath +
+                    ". Note that this path should be either absolute or a relative local file system path, " +
+                    "relative to META-INF in classpath or valid URL to GRIDGAIN_HOME.", e);
+        }
+
+        return loadConfigurations(url);
+    }
+
+    /**
+     * Loads first found grid configuration specified within given Spring XML configuration file.
+     * <p>
+     * Usually Spring XML configuration file will contain only one Grid definition. Note that
+     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
+     * the Grid configuration bean is ignored.
+     *
+     * @param springCfgUrl Spring XML configuration file path or URL. This cannot be {@code null}.
+     * @return First found configuration and Spring context used to load it.
+     * @throws IgniteCheckedException If grid could not be started or configuration
+     *      read. This exception will be thrown also if grid with given name has already
+     *      been started or Spring XML configuration file is invalid.
+     */
+    public static IgniteBiTuple<IgniteConfiguration, GridSpringResourceContext> loadConfiguration(URL springCfgUrl)
+        throws IgniteCheckedException {
+        IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> t = loadConfigurations(springCfgUrl);
+
+        return F.t(F.first(t.get1()), t.get2());
+    }
+
+    /**
+     * Loads first found grid configuration specified within given Spring XML configuration file.
+     * <p>
+     * Usually Spring XML configuration file will contain only one Grid definition. Note that
+     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
+     * the Grid configuration bean is ignored.
+     *
+     * @param springCfgPath Spring XML configuration file path. This cannot be {@code null}.
+     * @return First found configuration and Spring context used to load it.
+     * @throws IgniteCheckedException If grid could not be started or configuration
+     *      read. This exception will be thrown also if grid with given name has already
+     *      been started or Spring XML configuration file is invalid.
+     */
+    public static IgniteBiTuple<IgniteConfiguration, GridSpringResourceContext> loadConfiguration(String springCfgPath)
+        throws IgniteCheckedException {
+        IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> t =
+            loadConfigurations(springCfgPath);
+
+        return F.t(F.first(t.get1()), t.get2());
+    }
+
+    /**
+     * Starts all grids specified within given Spring XML configuration file. If grid with given name
+     * is already started, then exception is thrown. In this case all instances that may
+     * have been started so far will be stopped too.
+     * <p>
+     * Usually Spring XML configuration file will contain only one Grid definition. Note that
+     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
+     * the Grid configuration bean is ignored.
+     *
+     * @param springCfgPath Spring XML configuration file path or URL. This cannot be {@code null}.
+     * @param gridName Grid name that will override default.
+     * @param springCtx Optional Spring application context, possibly {@code null}.
+     *      Spring bean definitions for bean injection are taken from this context.
+     *      If provided, this context can be injected into grid tasks and grid jobs using
+     *      {@link org.apache.ignite.resources.IgniteSpringApplicationContextResource @IgniteSpringApplicationContextResource} annotation.
+     * @return Started grid. If Spring configuration contains multiple grid instances,
+     *      then the 1st found instance is returned.
+     * @throws IgniteCheckedException If grid could not be started or configuration
+     *      read. This exception will be thrown also if grid with given name has already
+     *      been started or Spring XML configuration file is invalid.
+     */
+    public static Ignite start(String springCfgPath, @Nullable String gridName,
+        @Nullable GridSpringResourceContext springCtx) throws IgniteCheckedException {
+        URL url = resolveSpringUrl(springCfgPath);
+
+        return start(url, gridName, springCtx);
+    }
+
+    /**
+     * Starts all grids specified within given Spring XML configuration file URL. If grid with given name
+     * is already started, then exception is thrown. In this case all instances that may
+     * have been started so far will be stopped too.
+     * <p>
+     * Usually Spring XML configuration file will contain only one Grid definition. Note that
+     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
+     * the Grid configuration bean is ignored.
+     *
+     * @param springCfgUrl Spring XML configuration file URL. This cannot be {@code null}.
+     * @return Started grid. If Spring configuration contains multiple grid instances,
+     *      then the 1st found instance is returned.
+     * @throws IgniteCheckedException If grid could not be started or configuration
+     *      read. This exception will be thrown also if grid with given name has already
+     *      been started or Spring XML configuration file is invalid.
+     */
+    public static Ignite start(URL springCfgUrl) throws IgniteCheckedException {
+        return start(springCfgUrl, null, null);
+    }
+
+    /**
+     * Starts all grids specified within given Spring XML configuration file URL. If grid with given name
+     * is already started, then exception is thrown. In this case all instances that may
+     * have been started so far will be stopped too.
+     * <p>
+     * Usually Spring XML configuration file will contain only one Grid definition. Note that
+     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
+     * the Grid configuration bean is ignored.
+     *
+     * @param springCfgUrl Spring XML configuration file URL. This cannot be {@code null}.
+     * @param gridName Grid name that will override default.
+     * @param springCtx Optional Spring application context, possibly {@code null}.
+     *      Spring bean definitions for bean injection are taken from this context.
+     *      If provided, this context can be injected into grid tasks and grid jobs using
+     *      {@link org.apache.ignite.resources.IgniteSpringApplicationContextResource @IgniteSpringApplicationContextResource} annotation.
+     * @return Started grid. If Spring configuration contains multiple grid instances,
+     *      then the 1st found instance is returned.
+     * @throws IgniteCheckedException If grid could not be started or configuration
+     *      read. This exception will be thrown also if grid with given name has already
+     *      been started or Spring XML configuration file is invalid.
+     */
+    public static Ignite start(URL springCfgUrl, @Nullable String gridName,
+        @Nullable GridSpringResourceContext springCtx) throws IgniteCheckedException {
+        return start(springCfgUrl, gridName, springCtx, null);
+    }
+
+    /**
+     * Internal Spring-based start routine.
+     *
+     * @param springCfgUrl Spring XML configuration file URL. This cannot be {@code null}.
+     * @param gridName Grid name that will override default.
+     * @param springCtx Optional Spring application context.
+     * @param cfgClo Optional closure to change configuration before it is used to start the grid.
+     * @return Started grid.
+     * @throws IgniteCheckedException If failed.
+     */
+    private static Ignite start(URL springCfgUrl, @Nullable String gridName,
+        @Nullable GridSpringResourceContext springCtx,
+        @Nullable IgniteClosure<IgniteConfiguration, IgniteConfiguration> cfgClo)
+        throws IgniteCheckedException {
+        A.notNull(springCfgUrl, "springCfgUrl");
+
+        boolean isLog4jUsed = U.gridClassLoader().getResource("org/apache/log4j/Appender.class") != null;
+
+        IgniteBiTuple<Object, Object> t = null;
+
+        Collection<Handler> savedHnds = null;
+
+        if (isLog4jUsed)
+            t = U.addLog4jNoOpLogger();
+        else
+            savedHnds = U.addJavaNoOpLogger();
+
+        IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> cfgMap;
+
+        try {
+            cfgMap = loadConfigurations(springCfgUrl);
+        }
+        finally {
+            if (isLog4jUsed && t != null)
+                U.removeLog4jNoOpLogger(t);
+
+            if (!isLog4jUsed)
+                U.removeJavaNoOpLogger(savedHnds);
+        }
+
+        List<IgniteNamedInstance> grids = new ArrayList<>(cfgMap.size());
+
+        try {
+            for (IgniteConfiguration cfg : cfgMap.get1()) {
+                assert cfg != null;
+
+                if (cfg.getGridName() == null && !F.isEmpty(gridName))
+                    cfg.setGridName(gridName);
+
+                if (cfgClo != null) {
+                    cfg = cfgClo.apply(cfg);
+
+                    assert cfg != null;
+                }
+
+                // Use either user defined context or our one.
+                IgniteNamedInstance grid = start0(
+                    new GridStartContext(cfg, springCfgUrl, springCtx == null ? cfgMap.get2() : springCtx));
+
+                // Add it if it was not stopped during startup.
+                if (grid != null)
+                    grids.add(grid);
+            }
+        }
+        catch (IgniteCheckedException e) {
+            // Stop all instances started so far.
+            for (IgniteNamedInstance grid : grids) {
+                try {
+                    grid.stop(true);
+                }
+                catch (Exception e1) {
+                    U.error(grid.log, "Error when stopping grid: " + grid, e1);
+                }
+            }
+
+            throw e;
+        }
+
+        // Return the first grid started.
+        IgniteNamedInstance res = !grids.isEmpty() ? grids.get(0) : null;
+
+        return res != null ? res.grid() : null;
+    }
+
+    /**
+     * Resolve Spring configuration URL.
+     *
+     * @param springCfgPath Spring XML configuration file path or URL. This cannot be {@code null}.
+     * @return URL.
+     * @throws IgniteCheckedException If failed.
+     */
+    private static URL resolveSpringUrl(String springCfgPath) throws IgniteCheckedException {
+        A.notNull(springCfgPath, "springCfgPath");
+
+        URL url;
+
+        try {
+            url = new URL(springCfgPath);
+        }
+        catch (MalformedURLException e) {
+            url = U.resolveGridGainUrl(springCfgPath);
+
+            if (url == null)
+                throw new IgniteCheckedException("Spring XML configuration path is invalid: " + springCfgPath +
+                    ". Note that this path should be either absolute or a relative local file system path, " +
+                    "relative to META-INF in classpath or valid URL to GRIDGAIN_HOME.", e);
+        }
+
+        return url;
+    }
+
+    /**
+     * Starts grid with given configuration.
+     *
+     * @param startCtx Start context.
+     * @return Started grid.
+     * @throws IgniteCheckedException If grid could not be started.
+     */
+    private static IgniteNamedInstance start0(GridStartContext startCtx) throws IgniteCheckedException {
+        assert startCtx != null;
+
+        String name = startCtx.config().getGridName();
+
+        if (name != null && name.isEmpty())
+            throw new IgniteCheckedException("Non default grid instances cannot have empty string name.");
+
+        IgniteNamedInstance grid = new IgniteNamedInstance(name);
+
+        IgniteNamedInstance old;
+
+        if (name != null)
+            old = grids.putIfAbsent(name, grid);
+        else {
+            synchronized (dfltGridMux) {
+                old = dfltGrid;
+
+                if (old == null)
+                    dfltGrid = grid;
+            }
+        }
+
+        if (old != null) {
+            if (name == null)
+                throw new IgniteCheckedException("Default grid instance has already been started.");
+            else
+                throw new IgniteCheckedException("Grid instance with this name has already been started: " + name);
+        }
+
+        if (startCtx.config().getWarmupClosure() != null)
+            startCtx.config().getWarmupClosure().apply(startCtx.config());
+
+        startCtx.single(grids.size() == 1);
+
+        boolean success = false;
+
+        try {
+            grid.start(startCtx);
+
+            notifyStateChange(name, STARTED);
+
+            success = true;
+        }
+        finally {
+            if (!success) {
+                if (name != null)
+                    grids.remove(name, grid);
+                else {
+                    synchronized (dfltGridMux) {
+                        if (dfltGrid == grid)
+                            dfltGrid = null;
+                    }
+                }
+
+                grid = null;
+            }
+        }
+
+        if (grid == null)
+            throw new IgniteCheckedException("Failed to start grid with provided configuration.");
+
+        return grid;
+    }
+
+    /**
+     * Gets an instance of default no-name grid. Note that
+     * caller of this method should not assume that it will return the same
+     * instance every time.
+     * <p>
+     * This method is identical to {@code G.grid(null)} apply.
+     *
+     * @return An instance of default no-name grid. This method never returns
+     *      {@code null}.
+     * @throws org.apache.ignite.IgniteIllegalStateException Thrown if default grid was not properly
+     *      initialized or grid instance was stopped or was not started.
+     */
+    public static Ignite grid() throws IgniteIllegalStateException {
+        return grid((String)null);
+    }
+
+    /**
+     * Gets a list of all grids started so far.
+     *
+     * @return List of all grids started so far.
+     */
+    public static List<Ignite> allGrids() {
+        List<Ignite> allIgnites = new ArrayList<>(grids.size() + 1);
+
+        for (IgniteNamedInstance grid : grids.values()) {
+            Ignite g = grid.grid();
+
+            if (g != null)
+                allIgnites.add(g);
+        }
+
+        IgniteNamedInstance dfltGrid0 = dfltGrid;
+
+        if (dfltGrid0 != null) {
+            IgniteKernal g = dfltGrid0.grid();
+
+            if (g != null)
+                allIgnites.add(g);
+        }
+
+        return allIgnites;
+    }
+
+    /**
+     * Gets a grid instance for given local node ID. Note that grid instance and local node have
+     * one-to-one relationship where node has ID and instance has name of the grid to which
+     * both grid instance and its node belong. Note also that caller of this method
+     * should not assume that it will return the same instance every time.
+     *
+     * @param locNodeId ID of local node the requested grid instance is managing.
+     * @return An instance of named grid. This method never returns
+     *      {@code null}.
+     * @throws org.apache.ignite.IgniteIllegalStateException Thrown if grid was not properly
+     *      initialized or grid instance was stopped or was not started.
+     */
+    public static Ignite grid(UUID locNodeId) throws IgniteIllegalStateException {
+        A.notNull(locNodeId, "locNodeId");
+
+        IgniteNamedInstance dfltGrid0 = dfltGrid;
+
+        if (dfltGrid0 != null) {
+            IgniteKernal g = dfltGrid0.grid();
+
+            if (g != null && g.getLocalNodeId().equals(locNodeId))
+                return g;
+        }
+
+        for (IgniteNamedInstance grid : grids.values()) {
+            IgniteKernal g = grid.grid();
+
+            if (g != null && g.getLocalNodeId().equals(locNodeId))
+                return g;
+        }
+
+        throw new IgniteIllegalStateException("Grid instance with given local node ID was not properly " +
+            "started or was stopped: " + locNodeId);
+    }
+
+    /**
+     * Gets an named grid instance. If grid name is {@code null} or empty string,
+     * then default no-name grid will be returned. Note that caller of this method
+     * should not assume that it will return the same instance every time.
+     * <p>
+     * Note that Java VM can run multiple grid instances and every grid instance (and its
+     * node) can belong to a different grid. Grid name defines what grid a particular grid
+     * instance (and correspondingly its node) belongs to.
+     *
+     * @param name Grid name to which requested grid instance belongs to. If {@code null},
+     *      then grid instance belonging to a default no-name grid will be returned.
+     * @return An instance of named grid. This method never returns
+     *      {@code null}.
+     * @throws org.apache.ignite.IgniteIllegalStateException Thrown if default grid was not properly
+     *      initialized or grid instance was stopped or was not started.
+     */
+    public static Ignite grid(@Nullable String name) throws IgniteIllegalStateException {
+        IgniteNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
+
+        Ignite res;
+
+        if (grid == null || (res = grid.grid()) == null)
+            throw new IgniteIllegalStateException("Grid instance was not properly started " +
+                "or was already stopped: " + name);
+
+        return res;
+    }
+
+    /**
+     * Gets grid instance without waiting its initialization.
+     *
+     * @param name Grid name.
+     * @return Grid instance.
+     */
+    public static IgniteKernal gridx(@Nullable String name) {
+        IgniteNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
+
+        IgniteKernal res;
+
+        if (grid == null || (res = grid.gridx()) == null)
+            throw new IllegalStateException("Grid instance was not properly started or was already stopped: " + name);
+
+        return res;
+    }
+
+    /**
+     * Adds a lsnr for grid life cycle events.
+     * <p>
+     * Note that unlike other listeners in GridGain this listener will be
+     * notified from the same thread that triggers the state change. Because of
+     * that it is the responsibility of the user to make sure that listener logic
+     * is light-weight and properly handles (catches) any runtime exceptions, if any
+     * are expected.
+     *
+     * @param lsnr Listener for grid life cycle events. If this listener was already added
+     *      this method is no-op.
+     */
+    public static void addListener(IgniteListener lsnr) {
+        A.notNull(lsnr, "lsnr");
+
+        lsnrs.add(lsnr);
+    }
+
+    /**
+     * Removes lsnr added by {@link #addListener(org.apache.ignite.lifecycle.IgniteListener)} method.
+     *
+     * @param lsnr Listener to remove.
+     * @return {@code true} if lsnr was added before, {@code false} otherwise.
+     */
+    public static boolean removeListener(IgniteListener lsnr) {
+        A.notNull(lsnr, "lsnr");
+
+        return lsnrs.remove(lsnr);
+    }
+
+    /**
+     * @param gridName Grid instance name.
+     * @param state Factory state.
+     */
+    private static void notifyStateChange(@Nullable String gridName, IgniteState state) {
+        if (gridName != null)
+            gridStates.put(gridName, state);
+        else
+            dfltGridState = state;
+
+        for (IgniteListener lsnr : lsnrs)
+            lsnr.onStateChange(gridName, state);
+    }
+
+    /**
+     * Start context encapsulates all starting parameters.
+     */
+    private static final class GridStartContext {
+        /** User-defined configuration. */
+        private IgniteConfiguration cfg;
+
+        /** Optional configuration path. */
+        private URL cfgUrl;
+
+        /** Optional Spring application context. */
+        private GridSpringResourceContext springCtx;
+
+        /** Whether or not this is a single grid instance in current VM. */
+        private boolean single;
+
+        /**
+         *
+         * @param cfg User-defined configuration.
+         * @param cfgUrl Optional configuration path.
+         * @param springCtx Optional Spring application context.
+         */
+        GridStartContext(IgniteConfiguration cfg, @Nullable URL cfgUrl, @Nullable GridSpringResourceContext springCtx) {
+            assert(cfg != null);
+
+            this.cfg = cfg;
+            this.cfgUrl = cfgUrl;
+            this.springCtx = springCtx;
+        }
+
+        /**
+         * @return Whether or not this is a single grid instance in current VM.
+         */
+        public boolean single() {
+            return single;
+        }
+
+        /**
+         * @param single Whether or not this is a single grid instance in current VM.
+         */
+        public void single(boolean single) {
+            this.single = single;
+        }
+
+        /**
+         * @return User-defined configuration.
+         */
+        IgniteConfiguration config() {
+            return cfg;
+        }
+
+        /**
+         * @param cfg User-defined configuration.
+         */
+        void config(IgniteConfiguration cfg) {
+            this.cfg = cfg;
+        }
+
+        /**
+         * @return Optional configuration path.
+         */
+        URL configUrl() {
+            return cfgUrl;
+        }
+
+        /**
+         * @param cfgUrl Optional configuration path.
+         */
+        void configUrl(URL cfgUrl) {
+            this.cfgUrl = cfgUrl;
+        }
+
+        /**
+         * @return Optional Spring application context.
+         */
+        public GridSpringResourceContext springContext() {
+            return springCtx;
+        }
+    }
+
+    /**
+     * Grid data container.
+     */
+    private static final class IgniteNamedInstance {
+        /** Map of registered MBeans. */
+        private static final Map<MBeanServer, GridMBeanServerData> mbeans =
+            new HashMap<>();
+
+        /** */
+        private static final String[] EMPTY_STR_ARR = new String[0];
+
+        /** Empty array of caches. */
+        private static final CacheConfiguration[] EMPTY_CACHE_CONFIGS = new CacheConfiguration[0];
+
+        /** Grid name. */
+        private final String name;
+
+        /** Grid instance. */
+        private volatile IgniteKernal grid;
+
+        /** Executor service. */
+        private ExecutorService execSvc;
+
+        /** Auto executor service flag. */
+        private boolean isAutoExecSvc;
+
+        /** Executor service shutdown flag. */
+        private boolean execSvcShutdown;
+
+        /** System executor service. */
+        private ExecutorService sysExecSvc;
+
+        /** Auto system service flag. */
+        private boolean isAutoSysSvc;
+
+        /** System executor service shutdown flag. */
+        private boolean sysSvcShutdown;
+
+        /** Management executor service. */
+        private ExecutorService mgmtExecSvc;
+
+        /** Auto management service flag. */
+        private boolean isAutoMgmtSvc;
+
+        /** Management executor service shutdown flag. */
+        private boolean mgmtSvcShutdown;
+
+        /** P2P executor service. */
+        private ExecutorService p2pExecSvc;
+
+        /** Auto P2P service flag. */
+        private boolean isAutoP2PSvc;
+
+        /** P2P executor service shutdown flag. */
+        private boolean p2pSvcShutdown;
+
+        /** GGFS executor service. */
+        private ExecutorService ggfsExecSvc;
+
+        /** Auto GGFS service flag. */
+        private boolean isAutoGgfsSvc;
+
+        /** GGFS executor service shutdown flag. */
+        private boolean ggfsSvcShutdown;
+
+        /** REST requests executor service. */
+        private ExecutorService restExecSvc;
+
+        /** Auto REST service flag. */
+        private boolean isAutoRestSvc;
+
+        /** REST executor service shutdown flag. */
+        private boolean restSvcShutdown;
+
+        /** Utility cache executor service. */
+        private ExecutorService utilityCacheExecSvc;
+
+        /** Grid state. */
+        private volatile IgniteState state = STOPPED;
+
+        /** Shutdown hook. */
+        private Thread shutdownHook;
+
+        /** Grid log. */
+        private IgniteLogger log;
+
+        /** Start guard. */
+        private final AtomicBoolean startGuard = new AtomicBoolean();
+
+        /** Start latch. */
+        private final CountDownLatch startLatch = new CountDownLatch(1);
+
+        /**
+         * Thread that starts this named instance. This field can be non-volatile since
+         * it makes sense only for thread where it was originally initialized.
+         */
+        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
+        private Thread starterThread;
+
+        /**
+         * Creates un-started named instance.
+         *
+         * @param name Grid name (possibly {@code null} for default grid).
+         */
+        IgniteNamedInstance(@Nullable String name) {
+            this.name = name;
+        }
+
+        /**
+         * Gets grid name.
+         *
+         * @return Grid name.
+         */
+        String getName() {
+            return name;
+        }
+
+        /**
+         * Gets grid instance.
+         *
+         * @return Grid instance.
+         */
+        IgniteKernal grid() {
+            if (starterThread != Thread.currentThread())
+                U.awaitQuiet(startLatch);
+
+            return grid;
+        }
+
+        /**
+         * Gets grid instance without waiting for its initialization.
+         *
+         * @return Grid instance.
+         */
+        public IgniteKernal gridx() {
+            return grid;
+        }
+
+        /**
+         * Gets grid state.
+         *
+         * @return Grid state.
+         */
+        IgniteState state() {
+            if (starterThread != Thread.currentThread())
+                U.awaitQuiet(startLatch);
+
+            return state;
+        }
+
+        /**
+         * @param spi SPI implementation.
+         * @throws IgniteCheckedException Thrown in case if multi-instance is not supported.
+         */
+        private void ensureMultiInstanceSupport(IgniteSpi spi) throws IgniteCheckedException {
+            IgniteSpiMultipleInstancesSupport ann = U.getAnnotation(spi.getClass(),
+                IgniteSpiMultipleInstancesSupport.class);
+
+            if (ann == null || !ann.value())
+                throw new IgniteCheckedException("SPI implementation doesn't support multiple grid instances in " +
+                    "the same VM: " + spi);
+        }
+
+        /**
+         * @param spis SPI implementations.
+         * @throws IgniteCheckedException Thrown in case if multi-instance is not supported.
+         */
+        private void ensureMultiInstanceSupport(IgniteSpi[] spis) throws IgniteCheckedException {
+            for (IgniteSpi spi : spis)
+                ensureMultiInstanceSupport(spi);
+        }
+
+        /**
+         * Starts grid with given configuration.
+         *
+         * @param startCtx Starting context.
+         * @throws IgniteCheckedException If start failed.
+         */
+        synchronized void start(GridStartContext startCtx) throws IgniteCheckedException {
+            if (startGuard.compareAndSet(false, true)) {
+                try {
+                    starterThread = Thread.currentThread();
+
+                    start0(startCtx);
+                }
+                catch (Exception e) {
+                    if (log != null)
+                        stopExecutors(log);
+
+                    throw e;
+                }
+                finally {
+                    startLatch.countDown();
+                }
+            }
+            else
+                U.awaitQuiet(startLatch);
+        }
+
+        /**
+         * @param startCtx Starting context.
+         * @throws IgniteCheckedException If start failed.
+         */
+        @SuppressWarnings({"unchecked", "TooBroadScope"})
+        private void start0(GridStartContext startCtx) throws IgniteCheckedException {
+            assert grid == null : "Grid is already started: " + name;
+
+            IgniteConfiguration cfg = startCtx.config();
+
+            if (cfg == null)
+                cfg = new IgniteConfiguration();
+
+            IgniteConfiguration myCfg = new IgniteConfiguration();
+
+            String ggHome = cfg.getGridGainHome();
+
+            // Set GridGain home.
+            if (ggHome == null)
+                ggHome = U.getGridGainHome();
+            else
+                // If user provided GRIDGAIN_HOME - set it as a system property.
+                U.setGridGainHome(ggHome);
+
+            U.setWorkDirectory(cfg.getWorkDirectory(), ggHome);
+
+            /*
+             * Set up all defaults and perform all checks.
+             */
+
+            // Ensure invariant.
+            // It's a bit dirty - but this is a result of late refactoring
+            // and I don't want to reshuffle a lot of code.
+            assert F.eq(name, cfg.getGridName());
+
+            // Set configuration URL, if any, into system property.
+            if (startCtx.configUrl() != null)
+                System.setProperty(GG_CONFIG_URL, startCtx.configUrl().toString());
+
+            myCfg.setGridName(cfg.getGridName());
+
+            UUID nodeId = cfg.getNodeId();
+
+            if (nodeId == null)
+                nodeId = UUID.randomUUID();
+
+            IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId);
+
+            assert cfgLog != null;
+
+            cfgLog = new GridLoggerProxy(cfgLog, null, name, U.id8(nodeId));
+
+            // Initialize factory's log.
+            log = cfgLog.getLogger(G.class);
+
+            // Check GridGain home folder (after log is available).
+            if (ggHome != null) {
+                File ggHomeFile = new File(ggHome);
+
+                if (!ggHomeFile.exists() || !ggHomeFile.isDirectory())
+                    throw new IgniteCheckedException("Invalid GridGain installation home folder: " + ggHome);
+            }
+
+            myCfg.setGridGainHome(ggHome);
+
+            // Copy values that don't need extra processing.
+            myCfg.setLicenseUrl(cfg.getLicenseUrl());
+            myCfg.setPeerClassLoadingEnabled(cfg.isPeerClassLoadingEnabled());
+            myCfg.setDeploymentMode(cfg.getDeploymentMode());
+            myCfg.setNetworkTimeout(cfg.getNetworkTimeout());
+            myCfg.setClockSyncSamples(cfg.getClockSyncSamples());
+            myCfg.setClockSyncFrequency(cfg.getClockSyncFrequency());
+            myCfg.setDiscoveryStartupDelay(cfg.getDiscoveryStartupDelay());
+            myCfg.setMetricsHistorySize(cfg.getMetricsHistorySize());
+            myCfg.setMetricsExpireTime(cfg.getMetricsExpireTime());
+            myCfg.setMetricsUpdateFrequency(cfg.getMetricsUpdateFrequency());
+            myCfg.setLifecycleBeans(cfg.getLifecycleBeans());
+            myCfg.setLocalEventListeners(cfg.getLocalEventListeners());
+            myCfg.setPeerClassLoadingMissedResourcesCacheSize(cfg.getPeerClassLoadingMissedResourcesCacheSize());
+            myCfg.setIncludeEventTypes(cfg.getIncludeEventTypes());
+            myCfg.setDaemon(cfg.isDaemon());
+            myCfg.setIncludeProperties(cfg.getIncludeProperties());
+            myCfg.setLifeCycleEmailNotification(cfg.isLifeCycleEmailNotification());
+            myCfg.setMetricsLogFrequency(cfg.getMetricsLogFrequency());
+            myCfg.setNetworkSendRetryDelay(cfg.getNetworkSendRetryDelay());
+            myCfg.setNetworkSendRetryCount(cfg.getNetworkSendRetryCount());
+            myCfg.setSecurityCredentialsProvider(cfg.getSecurityCredentialsProvider());
+            myCfg.setServiceConfiguration(cfg.getServiceConfiguration());
+            myCfg.setWarmupClosure(cfg.getWarmupClosure());
+            myCfg.setInteropConfiguration(cfg.getInteropConfiguration());
+            myCfg.setPluginConfigurations(cfg.getPluginConfigurations());
+            myCfg.setTransactionsConfiguration(new TransactionsConfiguration(cfg.getTransactionsConfiguration()));
+            myCfg.setQueryConfiguration(cfg.getQueryConfiguration());
+
+            ClientConnectionConfiguration clientCfg = cfg.getClientConnectionConfiguration();
+
+            if (clientCfg == null) {
+                // If client config is not provided then create config copying values from GridConfiguration.
+                if (cfg.isRestEnabled()) {
+                    clientCfg = new ClientConnectionConfiguration();
+
+                    clientCfg.setClientMessageInterceptor(cfg.getClientMessageInterceptor());
+                    clientCfg.setRestAccessibleFolders(cfg.getRestAccessibleFolders());
+                    clientCfg.setRestExecutorService(cfg.getRestExecutorService());
+                    clientCfg.setRestExecutorServiceShutdown(cfg.getRestExecutorServiceShutdown());
+                    clientCfg.setRestIdleTimeout(cfg.getRestIdleTimeout());
+                    clientCfg.setRestJettyPath(cfg.getRestJettyPath());
+                    clientCfg.setRestPortRange(cfg.getRestPortRange());
+                    clientCfg.setRestSecretKey(cfg.getRestSecretKey());
+                    clientCfg.setRestTcpDirectBuffer(cfg.isRestTcpDirectBuffer());
+                    clientCfg.setRestTcpHost(cfg.getRestTcpHost());
+                    clientCfg.setRestTcpNoDelay(cfg.isRestTcpNoDelay());
+                    clientCfg.setRestTcpPort(cfg.getRestTcpPort());
+                    clientCfg.setRestTcpReceiveBufferSize(cfg.getRestTcpReceiveBufferSize());
+                    clientCfg.setRestTcpSelectorCount(cfg.getRestTcpSelectorCount());
+                    clientCfg.setRestTcpSendBufferSize(cfg.getRestTcpSendBufferSize());
+                    clientCfg.setRestTcpSendQueueLimit(cfg.getRestTcpSendQueueLimit());
+                    clientCfg.setRestTcpSslClientAuth(cfg.isRestTcpSslClientAuth());
+                    clientCfg.setRestTcpSslContextFactory(cfg.getRestTcpSslContextFactory());
+                    clientCfg.setRestTcpSslEnabled(cfg.isRestTcpSslEnabled());
+                }
+            }
+            else
+                clientCfg = new ClientConnectionConfiguration(clientCfg);
+
+
+            String ntfStr = IgniteSystemProperties.getString(GG_LIFECYCLE_EMAIL_NOTIFY);
+
+            if (ntfStr != null)
+                myCfg.setLifeCycleEmailNotification(Boolean.parseBoolean(ntfStr));
+
+            // Local host.
+            String locHost = IgniteSystemProperties.getString(GG_LOCAL_HOST);
+
+            myCfg.setLocalHost(F.isEmpty(locHost) ? cfg.getLocalHost() : locHost);
+
+            // Override daemon flag if it was set on the factory.
+            if (daemon)
+                myCfg.setDaemon(true);
+
+            // Check for deployment mode override.
+            String depModeName = IgniteSystemProperties.getString(GG_DEP_MODE_OVERRIDE);
+
+            if (!F.isEmpty(depModeName)) {
+                if (!F.isEmpty(cfg.getCacheConfiguration())) {
+                    U.quietAndInfo(log, "Skipping deployment mode override for caches (custom closure " +
+                        "execution may not work for console Visor)");
+                }
+                else {
+                    try {
+                        IgniteDeploymentMode depMode = IgniteDeploymentMode.valueOf(depModeName);
+
+                        if (myCfg.getDeploymentMode() != depMode)
+                            myCfg.setDeploymentMode(depMode);
+                    }
+                    catch (IllegalArgumentException e) {
+                        throw new IgniteCheckedException("Failed to override deployment mode using system property " +
+                            "(are there any misspellings?)" +
+                            "[name=" + GG_DEP_MODE_OVERRIDE + ", value=" + depModeName + ']', e);
+                    }
+                }
+            }
+
+            Map<String, ?> attrs = cfg.getUserAttributes();
+
+            if (attrs == null)
+                attrs = Collections.emptyMap();
+
+            MBeanServer mbSrv = cfg.getMBeanServer();
+
+            IgniteMarshaller marsh = cfg.getMarshaller();
+
+            String[] p2pExclude = cfg.getPeerClassLoadingLocalClassPathExclude();
+
+            CommunicationSpi commSpi = cfg.getCommunicationSpi();
+            DiscoverySpi discoSpi = cfg.getDiscoverySpi();
+            EventStorageSpi evtSpi = cfg.getEventStorageSpi();
+            CollisionSpi colSpi = cfg.getCollisionSpi();
+            AuthenticationSpi authSpi = cfg.getAuthenticationSpi();
+            SecureSessionSpi sesSpi = cfg.getSecureSessionSpi();
+            DeploymentSpi deploySpi = cfg.getDeploymentSpi();
+            CheckpointSpi[] cpSpi = cfg.getCheckpointSpi();
+            FailoverSpi[] failSpi = cfg.getFailoverSpi();
+            LoadBalancingSpi[] loadBalancingSpi = cfg.getLoadBalancingSpi();
+            SwapSpaceSpi swapspaceSpi = cfg.getSwapSpaceSpi();
+            GridIndexingSpi indexingSpi = cfg.getIndexingSpi();
+
+            execSvc = cfg.getExecutorService();
+            sysExecSvc = cfg.getSystemExecutorService();
+            p2pExecSvc = cfg.getPeerClassLoadingExecutorService();
+            mgmtExecSvc = cfg.getManagementExecutorService();
+            ggfsExecSvc = cfg.getGgfsExecutorService();
+
+            if (execSvc == null) {
+                isAutoExecSvc = true;
+
+                execSvc = new IgniteThreadPoolExecutor(
+                    "pub-" + cfg.getGridName(),
+                    DFLT_PUBLIC_CORE_THREAD_CNT,
+                    DFLT_PUBLIC_MAX_THREAD_CNT,
+                    DFLT_PUBLIC_KEEP_ALIVE_TIME,
+                    new LinkedBlockingQueue<Runnable>(DFLT_PUBLIC_THREADPOOL_QUEUE_CAP));
+
+                // Pre-start all threads as they are guaranteed to be needed.
+                ((ThreadPoolExecutor)execSvc).prestartAllCoreThreads();
+            }
+
+            if (sysExecSvc == null) {
+                isAutoSysSvc = true;
+
+                // Note that since we use 'LinkedBlockingQueue', number of
+                // maximum threads has no effect.
+                sysExecSvc = new IgniteThreadPoolExecutor(
+                    "sys-" + cfg.getGridName(),
+                    DFLT_SYSTEM_CORE_THREAD_CNT,
+                    DFLT_SYSTEM_MAX_THREAD_CNT,
+                    DFLT_SYSTEM_KEEP_ALIVE_TIME,
+                    new LinkedBlockingQueue<Runnable>(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP));
+
+                // Pre-start all threads as they are guaranteed to be needed.
+                ((ThreadPoolExecutor)sysExecSvc).prestartAllCoreThreads();
+            }
+
+            if (mgmtExecSvc == null) {
+                isAutoMgmtSvc = true;
+
+                // Note that since we use 'LinkedBlockingQueue', number of
+                // maximum threads has no effect.
+                // Note, that we do not pre-start threads here as management pool may
+                // not be needed.
+                mgmtExecSvc = new IgniteThreadPoolExecutor(
+                    "mgmt-" + cfg.getGridName(),
+                    DFLT_MGMT_THREAD_CNT,
+                    DFLT_MGMT_THREAD_CNT,
+                    0,
+                    new LinkedBlockingQueue<Runnable>());
+            }
+
+            if (p2pExecSvc == null) {
+                isAutoP2PSvc = true;
+
+                // Note that since we use 'LinkedBlockingQueue', number of
+                // maximum threads has no effect.
+                // Note, that we do not pre-start threads here as class loading pool may
+                // not be needed.
+                p2pExecSvc = new IgniteThreadPoolExecutor(
+                    "p2p-" + cfg.getGridName(),
+                    DFLT_P2P_THREAD_CNT,
+                    DFLT_P2P_THREAD_CNT,
+                    0,
+                    new LinkedBlockingQueue<Runnable>());
+            }
+
+            if (ggfsExecSvc == null) {
+                isAutoGgfsSvc = true;
+
+                int procCnt = Runtime.getRuntime().availableProcessors();
+
+                // Note that we do not pre-start threads here as ggfs pool may not be needed.
+                ggfsExecSvc = new IgniteThreadPoolExecutor(
+                    "ggfs-" + cfg.getGridName(),
+                    procCnt,
+                    procCnt,
+                    0,
+                    new LinkedBlockingQueue<Runnable>());
+            }
+
+            restExecSvc = clientCfg != null ? clientCfg.getRestExecutorService() : null;
+
+            if (restExecSvc != null && !cfg.isRestEnabled()) {
+                U.warn(log, "REST executor service is configured, but REST is disabled in configuration " +
+                    "(safely ignoring).");
+            }
+            else if (restExecSvc == null && clientCfg != null) {
+                isAutoRestSvc = true;
+
+                restExecSvc = new IgniteThreadPoolExecutor(
+                    "rest-" + cfg.getGridName(),
+                    DFLT_REST_CORE_THREAD_CNT,
+                    DFLT_REST_MAX_THREAD_CNT,
+                    DFLT_REST_KEEP_ALIVE_TIME,
+                    new LinkedBlockingQueue<Runnable>(DFLT_REST_THREADPOOL_QUEUE_CAP)
+                );
+
+                clientCfg.setRestExecutorService(restExecSvc);
+            }
+
+            utilityCacheExecSvc = new IgniteThreadPoolExecutor(
+                "utility-" + cfg.getGridName(),
+                DFLT_SYSTEM_CORE_THREAD_CNT,
+                DFLT_SYSTEM_MAX_THREAD_CNT,
+                DFLT_SYSTEM_KEEP_ALIVE_TIME,
+                new LinkedBlockingQueue<Runnable>(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP));
+
+            execSvcShutdown = cfg.getExecutorServiceShutdown();
+            sysSvcShutdown = cfg.getSystemExecutorServiceShutdown();
+            mgmtSvcShutdown = cfg.getManagementExecutorServiceShutdown();
+            p2pSvcShutdown = cfg.getPeerClassLoadingExecutorServiceShutdown();
+            ggfsSvcShutdown = cfg.getGgfsExecutorServiceShutdown();
+            restSvcShutdown = clientCfg != null && clientCfg.isRestExecutorServiceShutdown();
+
+            if (marsh == null) {
+                if (!U.isHotSpot()) {
+                    U.warn(log, "GridOptimizedMarshaller is not supported on this JVM " +
+                        "(only Java HotSpot VMs are supported). Switching to standard JDK marshalling - " +
+                        "object serialization performance will be significantly slower.",
+                        "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release.");
+
+                    marsh = new IgniteJdkMarshaller();
+                }
+                else if (!IgniteOptimizedMarshaller.available()) {
+                    U.warn(log, "GridOptimizedMarshaller is not supported on this JVM " +
+                        "(only recent 1.6 and 1.7 versions HotSpot VMs are supported). " +
+                        "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release. " +
+                        "Switching to standard JDK marshalling - " +
+                        "object serialization performance will be significantly slower.",
+                        "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release.");
+
+                    marsh = new IgniteJdkMarshaller();
+                }
+                else
+                    marsh = new IgniteOptimizedMarshaller();
+            }
+            else if (marsh instanceof IgniteOptimizedMarshaller && !U.isHotSpot()) {
+                U.warn(log, "Using GridOptimizedMarshaller on untested JVM (only Java HotSpot VMs were tested) - " +
+                    "object serialization behavior could yield unexpected results.",
+                    "Using GridOptimizedMarshaller on untested JVM.");
+            }
+
+            myCfg.setUserAttributes(attrs);
+            myCfg.setMBeanServer(mbSrv == null ? ManagementFactory.getPlatformMBeanServer() : mbSrv);
+            myCfg.setGridLogger(cfgLog);
+            myCfg.setMarshaller(marsh);
+            myCfg.setMarshalLocalJobs(cfg.isMarshalLocalJobs());
+            myCfg.setExecutorService(execSvc);
+            myCfg.setSystemExecutorService(sysExecSvc);
+            myCfg.setManagementExecutorService(mgmtExecSvc);
+            myCfg.setPeerClassLoadingExecutorService(p2pExecSvc);
+            myCfg.setGgfsExecutorService(ggfsExecSvc);
+            myCfg.setExecutorServiceShutdown(execSvcShutdown);
+            myCfg.setSystemExecutorServiceShutdown(sysSvcShutdown);
+            myCfg.setManagementExecutorServiceShutdown(mgmtSvcShutdown);
+            myCfg.setPeerClassLoadingExecutorServiceShutdown(p2pSvcShutdown);
+            myCfg.setGgfsExecutorServiceShutdown(ggfsSvcShutdown);
+            myCfg.setNodeId(nodeId);
+
+            IgniteFsConfiguration[] ggfsCfgs = cfg.getGgfsConfiguration();
+
+            if (ggfsCfgs != null) {
+                IgniteFsConfiguration[] clone = ggfsCfgs.clone();
+
+                for (int i = 0; i < ggfsCfgs.length; i++)
+                    clone[i] = new IgniteFsConfiguration(ggfsCfgs[i]);
+
+                myCfg.setGgfsConfiguration(clone);
+            }
+
+            StreamerConfiguration[] streamerCfgs = cfg.getStreamerConfiguration();
+
+            if (streamerCfgs != null) {
+                StreamerConfiguration[] clone = streamerCfgs.clone();
+
+                for (int i = 0; i < streamerCfgs.length; i++)
+                    clone[i] = new StreamerConfiguration(streamerCfgs[i]);
+
+                myCfg.setStreamerConfiguration(clone);
+            }
+
+            if (p2pExclude == null)
+                p2pExclude = EMPTY_STR_ARR;
+
+            myCfg.setPeerClassLoadingLocalClassPathExclude(p2pExclude);
+
+            /*
+             * Initialize default SPI implementations.
+             */
+
+            if (commSpi == null)
+                commSpi = new TcpCommunicationSpi();
+
+            if (discoSpi == null)
+                discoSpi = new TcpDiscoverySpi();
+
+            if (discoSpi instanceof TcpDiscoverySpi) {
+                TcpDiscoverySpi tcpDisco = (TcpDiscoverySpi)discoSpi;
+
+                if (tcpDisco.getIpFinder() == null)
+                    tcpDisco.setIpFinder(new TcpDiscoveryMulticastIpFinder());
+            }
+
+            if (evtSpi == null)
+                evtSpi = new MemoryEventStorageSpi();
+
+            if (colSpi == null)
+                colSpi = new NoopCollisionSpi();
+
+            if (authSpi == null)
+                authSpi = new NoopAuthenticationSpi();
+
+            if (sesSpi == null)
+                sesSpi = new NoopSecureSessionSpi();
+
+            if (deploySpi == null)
+                deploySpi = new LocalDeploymentSpi();
+
+            if (cpSpi == null)
+                cpSpi = new CheckpointSpi[] {new NoopCheckpointSpi()};
+
+            if (failSpi == null)
+                failSpi = new FailoverSpi[] {new AlwaysFailoverSpi()};
+
+            if (loadBalancingSpi == null)
+                loadBalancingSpi = new LoadBalancingSpi[] {new RoundRobinLoadBalancingSpi()};
+
+            if (swapspaceSpi == null) {
+                boolean needSwap = false;
+
+                CacheConfiguration[] caches = cfg.getCacheConfiguration();
+
+                if (caches != null) {
+                    for (CacheConfiguration c : caches) {
+                        if (c.isSwapEnabled()) {
+                            needSwap = true;
+
+                            break;
+                        }
+                    }
+                }
+
+                swapspaceSpi = needSwap ? new FileSwapSpaceSpi() : new NoopSwapSpaceSpi();
+            }
+
+            if (indexingSpi == null)
+                indexingSpi = new GridNoopIndexingSpi();
+
+            myCfg.setCommunicationSpi(commSpi);
+            myCfg.setDiscoverySpi(discoSpi);
+            myCfg.setCheckpointSpi(cpSpi);
+            myCfg.setEventStorageSpi(evtSpi);
+            myCfg.setAuthenticationSpi(authSpi);
+            myCfg.setSecureSessionSpi(sesSpi);
+            myCfg.setDeploymentSpi(deploySpi);
+            myCfg.setFailoverSpi(failSpi);
+            myCfg.setCollisionSpi(colSpi);
+            myCfg.setLoadBalancingSpi(loadBalancingSpi);
+            myCfg.setSwapSpaceSpi(swapspaceSpi);
+            myCfg.setIndexingSpi(indexingSpi);
+
+            myCfg.setAddressResolver(cfg.getAddressResolver());
+
+            // Set SMTP configuration.
+            myCfg.setSmtpFromEmail(cfg.getSmtpFromEmail());
+            myCfg.setSmtpHost(cfg.getSmtpHost());
+            myCfg.setSmtpPort(cfg.getSmtpPort());
+            myCfg.setSmtpSsl(cfg.isSmtpSsl());
+            myCfg.setSmtpUsername(cfg.getSmtpUsername());
+            myCfg.setSmtpPassword(cfg.getSmtpPassword());
+            myCfg.setAdminEmails(cfg.getAdminEmails());
+
+            // REST configuration.
+            myCfg.setClientConnectionConfiguration(clientCfg);
+
+            // Portable configuration.
+            myCfg.setPortableConfiguration(cfg.getPortableConfiguration());
+
+            // Hadoop configuration.
+            myCfg.setHadoopConfiguration(cfg.getHadoopConfiguration());
+
+            // Validate segmentation configuration.
+            GridSegmentationPolicy segPlc = cfg.getSegmentationPolicy();
+
+            // 1. Warn on potential configuration problem: grid is not configured to wait
+            // for correct segment after segmentation happens.
+            if (!F.isEmpty(cfg.getSegmentationResolvers()) && segPlc == RESTART_JVM && !cfg.isWaitForSegmentOnStart()) {
+                U.warn(log, "Found potential configuration problem (forgot to enable waiting for segment" +
+                    "on start?) [segPlc=" + segPlc + ", wait=false]");
+            }
+
+            myCfg.setSegmentationResolvers(cfg.getSegmentationResolvers());
+            myCfg.setSegmentationPolicy(segPlc);
+            myCfg.setSegmentCheckFrequency(cfg.getSegmentCheckFrequency());
+            myCfg.setWaitForSegmentOnStart(cfg.isWaitForSegmentOnStart());
+            myCfg.setAllSegmentationResolversPassRequired(cfg.isAllSegmentationResolversPassRequired());
+
+            // Override SMTP configuration from system properties
+            // and environment variables, if specified.
+            String fromEmail = IgniteSystemProperties.getString(GG_SMTP_FROM);
+
+            if (fromEmail != null)
+                myCfg.setSmtpFromEmail(fromEmail);
+
+            String smtpHost = IgniteSystemProperties.getString(GG_SMTP_HOST);
+
+            if (smtpHost != null)
+                myCfg.setSmtpHost(smtpHost);
+
+            String smtpUsername = IgniteSystemProperties.getString(GG_SMTP_USERNAME);
+
+            if (smtpUsername != null)
+                myCfg.setSmtpUsername(smtpUsername);
+
+            String smtpPwd = IgniteSystemProperties.getString(GG_SMTP_PWD);
+
+            if (smtpPwd != null)
+                myCfg.setSmtpPassword(smtpPwd);
+
+            int smtpPort = IgniteSystemProperties.getInteger(GG_SMTP_PORT, -1);
+
+            if(smtpPort != -1)
+                myCfg.setSmtpPort(smtpPort);
+
+            myCfg.setSmtpSsl(IgniteSystemProperties.getBoolean(GG_SMTP_SSL));
+
+            String adminEmails = IgniteSystemProperties.getString(GG_ADMIN_EMAILS);
+
+            if (adminEmails != null)
+                myCfg.setAdminEmails(adminEmails.split(","));
+
+            CacheConfiguration[] cacheCfgs = cfg.getCacheConfiguration();
+
+            boolean hasHadoop = IgniteComponentType.HADOOP.inClassPath();
+
+            CacheConfiguration[] copies;
+
+            if (cacheCfgs != null && cacheCfgs.length > 0) {
+                if (!U.discoOrdered(discoSpi) && !U.relaxDiscoveryOrdered())
+                    throw new IgniteCheckedException("Discovery SPI implementation does not support node ordering and " +
+                        "cannot be used with cache (use SPI with @GridDiscoverySpiOrderSupport annotation, " +
+                        "like GridTcpDiscoverySpi)");
+
+                for (CacheConfiguration ccfg : cacheCfgs) {
+                    if (CU.isHadoopSystemCache(ccfg.getName()))
+                        throw new IgniteCheckedException("Cache name cannot be \"" + CU.SYS_CACHE_HADOOP_MR +
+                            "\" because it is reserved for internal purposes.");
+
+                    if (CU.isUtilityCache(ccfg.getName()))
+                        throw new IgniteCheckedException("Cache name cannot start with \"" + CU.UTILITY_CACHE_NAME +
+                            "\" because this prefix is reserved for internal purposes.");
+                }
+
+                copies = new CacheConfiguration[cacheCfgs.length + (hasHadoop ? 2 : 1)];
+
+                int cloneIdx = 1;
+
+                if (hasHadoop)
+                    copies[cloneIdx++] = CU.hadoopSystemCache();
+
+                for (CacheConfiguration ccfg : cacheCfgs)
+                    copies[cloneIdx++] = new CacheConfiguration(ccfg);
+            }
+            else if (hasHadoop) {
+                // Populate system caches
+                copies = new CacheConfiguration[hasHadoop ? 2 : 1];
+
+                copies[1] = CU.hadoopSystemCache();
+            }
+            else
+                copies = new CacheConfiguration[1];
+
+            // Always add utility cache.
+            copies[0] = utilitySystemCache(discoSpi instanceof TcpClientDiscoverySpi);
+
+            myCfg.setCacheConfiguration(copies);
+
+            myCfg.setCacheSanityCheckEnabled(cfg.isCacheSanityCheckEnabled());
+
+            try {
+                // Use reflection to avoid loading undesired classes.
+                Class helperCls = Class.forName("org.gridgain.grid.util.GridConfigurationHelper");
+
+                helperCls.getMethod("overrideConfiguration", IgniteConfiguration.class, Properties.class,
+                    String.class, IgniteLogger.class).invoke(helperCls, myCfg, System.getProperties(), name, log);
+            }
+            catch (Exception ignored) {
+                // No-op.
+            }
+
+            // Ensure that SPIs support multiple grid instances, if required.
+            if (!startCtx.single()) {
+                ensureMultiInstanceSupport(deploySpi);
+                ensureMultiInstanceSupport(commSpi);
+                ensureMultiInstanceSupport(discoSpi);
+                ensureMultiInstanceSupport(cpSpi);
+                ensureMultiInstanceSupport(evtSpi);
+                ensureMultiInstanceSupport(colSpi);
+                ensureMultiInstanceSupport(failSpi);
+                ensureMultiInstanceSupport(authSpi);
+                ensureMultiInstanceSupport(sesSpi);
+                ensureMultiInstanceSupport(loadBalancingSpi);
+                ensureMultiInstanceSupport(swapspaceSpi);
+            }
+
+            // Register GridGain MBean for current grid instance.
+            registerFactoryMbean(myCfg.getMBeanServer());
+
+            boolean started = false;
+
+            try {
+                IgniteKernal grid0 = new IgniteKernal(startCtx.springContext());
+
+                // Init here to make grid available to lifecycle listeners.
+                grid = grid0;
+
+                grid0.start(myCfg, utilityCacheExecSvc, new CA() {
+                    @Override public void apply() {
+                        startLatch.countDown();
+                    }
+                });
+
+                state = STARTED;
+
+                if (log.isDebugEnabled())
+                    log.debug("Grid factory started ok: " + name);
+
+                started = true;
+            }
+            catch (IgniteCheckedException e) {
+                unregisterFactoryMBean();
+
+                throw e;
+            }
+            // Catch Throwable to protect against any possible failure.
+            catch (Throwable e) {
+                unregisterFactoryMBean();
+
+                throw new IgniteCheckedException("Unexpected exception when starting grid.", e);
+            }
+            finally {
+                if (!started)
+                    // Grid was not started.
+                    grid = null;
+            }
+
+            // Do NOT set it up only if GRIDGAIN_NO_SHUTDOWN_HOOK=TRUE is provided.
+            if (!IgniteSystemProperties.getBoolean(GG_NO_SHUTDOWN_HOOK, false)) {
+                try {
+                    Runtime.getRuntime().addShutdownHook(shutdownHook = new Thread() {
+                        @Override public void run() {
+                            if (log.isInfoEnabled())
+                                log.info("Invoking shutdown hook...");
+
+                            IgniteNamedInstance.this.stop(true);
+                        }
+                    });
+
+                    if (log.isDebugEnabled())
+                        log.debug("Shutdown hook is installed.");
+                }
+                catch (IllegalStateException e) {
+                    stop(true);
+
+                    throw new IgniteCheckedException("Failed to install shutdown hook.", e);
+                }
+            }
+            else {
+                if (log.isDebugEnabled())
+                    log.debug("Shutdown hook has not been installed because environment " +
+                        "or system property " + GG_NO_SHUTDOWN_HOOK + " is set.");
+            }
+        }
+
+        /**
+         * @param cfgLog Configured logger.
+         * @param nodeId Local node ID.
+         * @return Initialized logger.
+         * @throws IgniteCheckedException If failed.
+         */
+        private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId) throws IgniteCheckedException {
+            try {
+                if (cfgLog == null) {
+                    Class<?> log4jCls;
+
+                    try {
+                        log4jCls = Class.forName("org.gridgain.grid.logger.log4j.GridLog4jLogger");
+                    }
+                    catch (ClassNotFoundException | NoClassDefFoundError ignored) {
+                        log4jCls = null;
+                    }
+
+                    if (log4jCls != null) {
+                        URL url = U.resolveGridGainUrl("config/gridgain-log4j.xml");
+
+                        if (url == null) {
+                            File cfgFile = new File("config/gridgain-log4j.xml");
+
+                            if (!cfgFile.exists())
+                                cfgFile = new File("../config/gridgain-log4j.xml");
+
+                            if (cfgFile.exists()) {
+                                try {
+                                    url = cfgFile.toURI().toURL();
+                                }
+                                catch (MalformedURLException ignore) {
+                                    // No-op.
+                                }
+                            }
+                        }
+
+                        if (url != null) {
+                            boolean configured = (Boolean)log4jCls.getMethod("isConfigured").invoke(null);
+
+                            if (configured)
+                                url = null;
+                        }
+
+                        if (url != null) {
+                            Constructor<?> ctor = log4jCls.getConstructor(URL.class);
+
+                            cfgLog = (IgniteLogger)ctor.newInstance(url);
+                        }
+                        else
+                            cfgLog = (IgniteLogger)log4jCls.newInstance();
+                    }
+                    else
+                        cfgLog = new IgniteJavaLogger();
+                }
+
+                // Set node IDs for all file appenders.
+                if (cfgLog instanceof IgniteLoggerNodeIdAware)
+                    ((IgniteLoggerNodeIdAware)cfgLog).setNodeId(nodeId);
+
+                return cfgLog;
+            }
+            catch (Exception e) {
+                throw new IgniteCheckedException("Failed to create logger.", e);
+            }
+        }
+
+        /**
+         * Creates utility system cache configuration.
+         *
+         * @param client If {@code true} creates client-only cache configuration.
+         * @return Utility system cache configuration.
+         */
+        private CacheConfiguration utilitySystemCache(boolean client) {
+            CacheConfiguration cache = new CacheConfiguration();
+
+            cache.setName(CU.UTILITY_CACHE_NAME);
+            cache.setCacheMode(REPLICATED);
+            cache.setAtomicityMode(TRANSACTIONAL);
+            cache.setSwapEnabled(false);
+            cache.setQueryIndexEnabled(false);
+            cache.setPreloadMode(SYNC);
+            cache.setWriteSynchronizationMode(FULL_SYNC);
+            cache.setAffinity(new CacheRendezvousAffinityFunction(false, 100));
+
+            if (client)
+                cache.setDistributionMode(CLIENT_ONLY);
+
+            return cache;
+        }
+
+        /**
+         * Stops grid.
+         *
+         * @param cancel Flag indicating whether all currently running jobs
+         *      should be cancelled.
+         */
+        void stop(boolean cancel) {
+            // Stop cannot be called prior to start from public API,
+            // since it checks for STARTED state. So, we can assert here.
+            assert startGuard.get();
+
+            stop0(cancel);
+        }
+
+        /**
+         * @param cancel Flag indicating whether all currently running jobs
+         *      should be cancelled.
+         */
+        private synchronized void stop0(boolean cancel) {
+            IgniteKernal grid0 = grid;
+
+            // Double check.
+            if (grid0 == null) {
+                if (log != null)
+                    U.warn(log, "Attempting to stop an already stopped grid instance (ignore): " + name);
+
+                return;
+            }
+
+            if (shutdownHook != null)
+                try {
+                    Runtime.getRuntime().removeShutdownHook(shutdownHook);
+
+                    shutdownHook = null;
+
+                    if (log.isDebugEnabled())
+                        log.debug("Shutdown hook is removed.");
+                }
+                catch (IllegalStateException e) {
+                    // Shutdown is in progress...
+                    if (log.isDebugEnabled())
+                        log.debug("Shutdown is in progress (ignoring): " + e.getMessage());
+                }
+
+            // Unregister GridGain MBean.
+            unregisterFactoryMBean();
+
+            try {
+                grid0.stop(cancel);
+
+                if (log.isDebugEnabled())
+       

<TRUNCATED>

[47/50] [abbrv] incubator-ignite git commit: minor

Posted by vo...@apache.org.
minor


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

Branch: refs/heads/ignite-16
Commit: 535ca6f3b32f80df24325b3fcf6fb722b525e201
Parents: c5247ab
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jan 29 12:46:12 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jan 29 12:46:12 2015 +0300

----------------------------------------------------------------------
 .../core/src/main/java/org/apache/ignite/internal/IgnitionEx.java | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/535ca6f3/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index acab6d0..d962de0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1144,9 +1144,6 @@ public class IgnitionEx {
         /** */
         private static final String[] EMPTY_STR_ARR = new String[0];
 
-        /** Empty array of caches. */
-        private static final CacheConfiguration[] EMPTY_CACHE_CONFIGS = new CacheConfiguration[0];
-
         /** Grid name. */
         private final String name;
 


[19/50] [abbrv] incubator-ignite git commit: # sprint-1 Updated readme files under docs folder.

Posted by vo...@apache.org.
# sprint-1 Updated readme files under docs folder.


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

Branch: refs/heads/ignite-16
Commit: d19c73e431f04aa9d7eb54032bdfd07a6678fd42
Parents: f5d64b2
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Jan 29 06:32:04 2015 +0000
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Jan 29 06:32:04 2015 +0000

----------------------------------------------------------------------
 docs/ignite_readme.md  | 100 ++++++++++++++++++++++++++++++++++++++++++++
 docs/ignite_readme.pdf | Bin 0 -> 76810 bytes
 2 files changed, 100 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d19c73e4/docs/ignite_readme.md
----------------------------------------------------------------------
diff --git a/docs/ignite_readme.md b/docs/ignite_readme.md
new file mode 100644
index 0000000..297b8b3
--- /dev/null
+++ b/docs/ignite_readme.md
@@ -0,0 +1,100 @@
+<center>
+![Ignite Logo](https://ignite.incubator.apache.org/images/logo3.png "Ignite Logo")
+</center>
+
+<div style="height: 5px"></div>
+
+## 1. Apache Ignite In-Memory Data Fabric
+
+Ignite In-Memory Data Fabric is designed to deliver uncompromised performance for the widest array of in-memory computing use cases.
+
+Following main components are included in the fabric:
+* `Advanced Clustering` - support for auto-discovery of cluster nodes in any environment, including, public clouds (e.g. AWS), private clouds, or hybrid clouds.
+* `Distributed Messaging` - support for topic-based and point-to-point message exchange between cluster nodes.
+* `Distributed Events` - support for cluster-wide event notifications.
+* `Managed Services` - support for managed user-defined services, like cluster singletons, node-singletons, and services with custom deployment topology.
+* `Distributed Data Structures` - support for common distributed data structures, like maps, sets, queues, atomics, etc.
+* `In-Memory Compute Grid` - includes distributed clustering, messaging, events, and computational features.
+* `In-Memory Data Grid` - partitioned in-memory key-value store with support for ACID transactions, off-heap memory, SQL, and more.
+* `In-Memory Streaming` - supports event workflow, rolling data windows and indexing, continuous querying, and more.
+* `In-Memory Filesystem` - distributed Hadoop-compliant in-memory file system.
+
+## 2. Apache Ignite Installation
+Ignite distribution comes in a ZIP file that simply needs to be unzipped, and `IGNITE_HOME` environment variable can optionally be set to point to it.
+
+There are no additional steps required for Ignite installation in such multi machine setup.
+
+Installation requirements:
+
+1. Windows, Linux, or MacOS environment.
+2. Java 7 or 8 (latest update is advisable).
+3. Point `JAVA_HOME` environment variable to your JDK or JRE installation.
+4. Optional: point `IGNITE_HOME` environment variable to the Ignite installation folder.
+
+### 2.1 Check Ignite Installation
+
+To verify Ignite installation, you can execute the Ignite startup script.
+
+The following command will startup Ignite with default configuration using Multicast node discovery.
+
+    bin/ggstart.{sh|bat}
+
+The following command will startup Ignite with example configuration.
+
+    bin/ggstart.{sh|bat} examples/config/example-compute.xml
+
+If Ignite was installed successfully, the output from above commands should produce no exceptions or errors.
+Note that you may see some warnings during startup, but this is OK as they are meant to inform that certain functionality is turned on or off by default.
+
+You can execute the above commands multiple times on the same machine and make sure that nodes discover each other.
+Here is an example of log printout when 2 nodes join topology:
+
+    ... Topology snapshot [nodes=2, CPUs=8, hash=0xD551B245]
+
+You can also start Ignite Management Console, called Visor, and observe started nodes. To startup Visor, you should execute the following script:
+
+    /bin/ggvisorcmd.{sh|bat} 
+
+### 2.2 Running Ignite Examples
+
+Ignite comes with many well documented examples. All examples have documentation about how they should be started and what the expected outcome should be.
+
+> Use provided pom.xml to import examples into IDE of your choice.
+
+## 3. Maven
+Apache Ignite hosts its Maven artifacts in Apache maven repository as well as in Maven Central.
+
+### 3.1 Maven Artifacts
+You can use maven to add Ignite artifacts to your project. Ignite has one main artifact called `ignite-fabric`. You can also import individual maven artifacts a al carte to bring in more optional dependencies, like `ignite-aws` for AWS integration, for example. All optional maven dependencies are also available in the Ignite  installation under `libs/optional` folder.
+
+### 3.2 Maven Example
+
+    <dependency>
+        <groupId>org.apache.ignite</groupId>
+        <artifactId>ignite-fabric</artifactId>
+        <version>${ignite.version}</version>
+        <type>pom</type>
+    </dependency>
+
+
+## 4. Starting Ignite Nodes
+Ignite nodes can be started by executing `bin/ggstart.{sh|bat}` script and passing a relative path to Ignite configuration file. If no file is passed, then grid nodes are started with default configuration using Multicast discovery protocol.
+
+Here is an example of how to start Ignite node with non-default configuration:
+
+    `bin/ggstart.sh examples/config/example-cache.xml`
+
+## 5. Management & Monitoring with Visor
+Ignite comes with CLI (command) based DevOps Managements Console, called Visor, delivering advance set of management and monitoring capabilities. 
+
+To start Visor in console mode you should execute the following command:
+
+    `bin/ggvisorcmd.sh`
+
+On Windows, run the same commands with `.bat` extension.
+
+## 6. Scala Integration
+Ignite provides a very nice and easy to use DSL for Scala users called `Scalar`. If you like Scala, take a look at Scalar examples located under `examples/src/main/scala` folder.
+
+## 7. Javadoc & Scaladoc
+All documentation is shipped with it and you can find it under `docs/javadoc` and `docs/scaladoc` sub-folder respectively.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d19c73e4/docs/ignite_readme.pdf
----------------------------------------------------------------------
diff --git a/docs/ignite_readme.pdf b/docs/ignite_readme.pdf
new file mode 100644
index 0000000..c270d15
Binary files /dev/null and b/docs/ignite_readme.pdf differ


[02/50] [abbrv] incubator-ignite git commit: #Tests: fix GridUtils.isGrid

Posted by vo...@apache.org.
#Tests: fix GridUtils.isGrid


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

Branch: refs/heads/ignite-16
Commit: 79ba5af86bdcda62b36a2d1287fcd13a1811c455
Parents: 9996140
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jan 28 18:03:04 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jan 28 18:03:04 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/util/GridUtils.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/79ba5af8/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java
index 6ebc7e7..82ae077 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java
@@ -5607,7 +5607,7 @@ public abstract class GridUtils {
      * @return {@code True} if given class is of {@code Grid} type.
      */
     public static boolean isGrid(Class<?> cls) {
-        return cls.getName().startsWith("org.gridgain.grid");
+        return cls.getName().startsWith("org.apache.ignite.internal");
     }
 
     /**


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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
index ab2d4bf..275af80 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractDataStructuresFailoverSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.datastructures;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.datastructures.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
@@ -122,7 +123,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
         try {
             CacheAtomicLong s = cache().dataStructures().atomicLong(STRUCTURE_NAME, 1, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -172,7 +173,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
         try {
             CacheAtomicLong s = cache().dataStructures().atomicLong(STRUCTURE_NAME, 1, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -251,7 +252,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
         try {
             CacheAtomicReference<Integer> s = cache().dataStructures().atomicReference(STRUCTURE_NAME, 1, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -300,7 +301,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
         try {
             CacheAtomicReference<Integer> s = cache().dataStructures().atomicReference(STRUCTURE_NAME, 1, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -386,7 +387,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
         try {
             CacheAtomicStamped<Integer, Integer> s = cache().dataStructures().atomicStamped(STRUCTURE_NAME, 1, 1, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -449,7 +450,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
         try {
             CacheAtomicStamped<Integer, Integer> s = cache().dataStructures().atomicStamped(STRUCTURE_NAME, 1, 1, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -544,7 +545,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
             CacheCountDownLatch s = cache().dataStructures().countDownLatch(STRUCTURE_NAME, Integer.MAX_VALUE,
                 false, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -599,7 +600,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
             CacheCountDownLatch s = cache().dataStructures()
                 .countDownLatch(STRUCTURE_NAME, Integer.MAX_VALUE, false, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -684,7 +685,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
 
             s.put(1);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -734,7 +735,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
 
             s.put(1);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         for (int i = 0; i < TOP_CHANGE_CNT; i++) {
@@ -809,7 +810,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
         try {
             CacheAtomicSequence s = cache().dataStructures().atomicSequence(STRUCTURE_NAME, 1, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
                 @Override public void apply() {
                     try {
                         String name = UUID.randomUUID().toString();
@@ -860,7 +861,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
 
         final AtomicInteger idx = new AtomicInteger(gridCount());
 
-        IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
             @Override public void apply() {
                 int id = idx.getAndIncrement();
 
@@ -911,7 +912,7 @@ public abstract class GridCacheAbstractDataStructuresFailoverSelfTest extends Gr
         try {
             CacheAtomicSequence s = cache().dataStructures().atomicSequence(STRUCTURE_NAME, 1, true);
 
-            IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(new CA() {
+            IgniteInternalFuture<?> 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/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java
index 95db99e..a29d461 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
 
@@ -173,7 +172,7 @@ public abstract class GridCacheAbstractQueueFailoverDataConsistencySelfTest exte
 
         final AtomicBoolean stop = new AtomicBoolean();
 
-        IgniteFuture<?> fut = startNodeKiller(stop, new AtomicInteger(), killIdxs);
+        IgniteInternalFuture<?> fut = startNodeKiller(stop, new AtomicInteger(), killIdxs);
 
         final int ITEMS = (atomicityMode() == ATOMIC) ? 10_000 : 3000;
 
@@ -279,7 +278,7 @@ public abstract class GridCacheAbstractQueueFailoverDataConsistencySelfTest exte
 
         final AtomicInteger stopCnt = new AtomicInteger();
 
-        IgniteFuture<?> fut = startNodeKiller(stop, stopCnt, killIdxs);
+        IgniteInternalFuture<?> fut = startNodeKiller(stop, stopCnt, killIdxs);
 
         int err = 0;
 
@@ -332,7 +331,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 IgniteInternalFuture<?> 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/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java
index 3860d1a..763135d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java
@@ -152,7 +152,7 @@ public class GridCacheCountDownLatchSelfTest extends GridCommonAbstractTest impl
 
         assert latch.count() == 2;
 
-        IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     CacheCountDownLatch latch = grid(0).cache("local").dataStructures()
@@ -219,7 +219,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(
+                IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(
                     new Callable<Object>() {
                         @Nullable @Override public Object call() throws Exception {
                             CacheCountDownLatch latch = ignite.cache(cacheName).dataStructures()
@@ -246,7 +246,7 @@ public class GridCacheCountDownLatchSelfTest extends GridCommonAbstractTest impl
             }
         });
 
-        IgniteFuture<Object> fut = comp.future();
+        IgniteInternalFuture<Object> fut = comp.future();
 
         Thread.sleep(3000);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java
index b123402..131e7b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java
@@ -23,7 +23,6 @@ import org.apache.ignite.cache.datastructures.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
 
@@ -208,7 +207,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 IgniteInternalFuture<?> startAddPollThread(final Ignite ignite, final AtomicBoolean stop, final String queueName) {
         return GridTestUtils.runAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 CacheQueue<Integer> queue = ignite.cache(null).dataStructures().queue(queueName, 0, false, true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java
index 2268044..935dcec 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueJoinedNodeSelfAbstractTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.datastructures;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.datastructures.*;
 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.discovery.tcp.*;
@@ -89,9 +90,9 @@ public abstract class GridCacheQueueJoinedNodeSelfAbstractTest extends GridCommo
 
         comp.run(putJob);
 
-        IgniteFuture<?> fut = comp.future();
+        IgniteInternalFuture<?> fut = comp.future();
 
-        Collection<IgniteFuture<?>> futs = new ArrayList<>(GRID_CNT - 1);
+        Collection<IgniteInternalFuture<?>> futs = new ArrayList<>(GRID_CNT - 1);
 
         Collection<TakeJob> jobs = new ArrayList<>(GRID_CNT - 1);
 
@@ -134,7 +135,7 @@ public abstract class GridCacheQueueJoinedNodeSelfAbstractTest extends GridCommo
 
         info(">>> Joined node polled " + polled);
 
-        for (IgniteFuture<?> f : futs)
+        for (IgniteInternalFuture<?> f : futs)
             f.cancel();
 
         putJob.stop(true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
index a480880..a3b78df 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.datastructures.*;
 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.spi.discovery.tcp.*;
@@ -177,7 +178,7 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon
 
             final Ignite g = startGrid(GRID_CNT + 1);
 
-            IgniteFuture<Object> fut1 = GridTestUtils.runAsync(new Callable<Object>() {
+            IgniteInternalFuture<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=" +
@@ -208,7 +209,7 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon
 
             final Ignite g1 = startGrid(GRID_CNT + 2);
 
-            IgniteFuture<Object> fut2 = GridTestUtils.runAsync(new Callable<Object>() {
+            IgniteInternalFuture<Object> fut2 = GridTestUtils.runAsync(new Callable<Object>() {
                 @SuppressWarnings("BusyWait")
                 @Override public Object call() throws Exception {
                     try {
@@ -362,7 +363,7 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon
         info("Queue name: " + queueName + ", collocated: " + collocated);
 
         try {
-            Collection<IgniteFuture> futs = new ArrayList<>();
+            Collection<IgniteInternalFuture> futs = new ArrayList<>();
 
             final int THREADS_PER_NODE = 3;
             final int ITEMS_PER_THREAD = 1000;
@@ -384,7 +385,7 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon
                 }, THREADS_PER_NODE, "testPutMultiNode"));
             }
 
-            for (IgniteFuture fut : futs)
+            for (IgniteInternalFuture fut : futs)
                 fut.get();
 
             GridCache cache = grid(0).cache(null);
@@ -427,8 +428,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<IgniteInternalFuture> putFuts = new ArrayList<>();
+            Collection<IgniteInternalFuture> pollFuts = new ArrayList<>();
 
             final int PUT_THREADS_PER_NODE = 3;
             final int POLL_THREADS_PER_NODE = 2;
@@ -479,12 +480,12 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon
                 }
             }
 
-            for (IgniteFuture fut : putFuts)
+            for (IgniteInternalFuture fut : putFuts)
                 fut.get();
 
             stopPoll.set(true);
 
-            for (IgniteFuture fut : pollFuts)
+            for (IgniteInternalFuture fut : pollFuts)
                 fut.get();
 
             GridCache cache = grid(0).cache(null);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
index f92a963..47d4daa 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.cache.datastructures.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.cache.query.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
@@ -668,7 +667,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf
 
         assertNotNull(set0);
 
-        Collection<IgniteFuture> futs = new ArrayList<>();
+        Collection<IgniteInternalFuture> futs = new ArrayList<>();
 
         final int THREADS_PER_NODE = 5;
         final int KEY_RANGE = 10_000;
@@ -723,7 +722,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf
             }, THREADS_PER_NODE, "testSetMultithreaded"));
         }
 
-        for (IgniteFuture fut : futs)
+        for (IgniteInternalFuture fut : futs)
             fut.get();
     }
 
@@ -775,7 +774,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf
 
         final AtomicInteger val = new AtomicInteger(10_000);
 
-        IgniteFuture<?> fut;
+        IgniteInternalFuture<?> fut;
 
         try {
             fut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
index 67d8300..efcc227 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
@@ -113,7 +113,7 @@ public class GridCacheSetFailoverAbstractSelfTest extends GridCacheAbstractSelfT
 
         AtomicBoolean stop = new AtomicBoolean();
 
-        IgniteFuture<?> killFut = startNodeKiller(stop);
+        IgniteInternalFuture<?> killFut = startNodeKiller(stop);
 
         long stopTime = System.currentTimeMillis() + TEST_DURATION;
 
@@ -219,7 +219,7 @@ public class GridCacheSetFailoverAbstractSelfTest extends GridCacheAbstractSelfT
      * @param stop Stop flag.
      * @return Future completing when thread finishes.
      */
-    private IgniteFuture<?> startNodeKiller(final AtomicBoolean stop) {
+    private IgniteInternalFuture<?> 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/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java
index 63aaf60..111980f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueCreateMultiNodeSelfTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.datastructures.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -87,7 +87,7 @@ public class GridCachePartitionedQueueCreateMultiNodeSelfTest extends GridCommon
     public void testQueueCreation() throws Exception {
         final AtomicInteger idx = new AtomicInteger();
 
-        IgniteFuture<?> fut = multithreadedAsync(
+        IgniteInternalFuture<?> fut = multithreadedAsync(
             new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     Ignite ignite = startGrid(idx.getAndIncrement());
@@ -139,7 +139,7 @@ public class GridCachePartitionedQueueCreateMultiNodeSelfTest extends GridCommon
 
         final CountDownLatch latch = new CountDownLatch(threadCnt);
 
-        IgniteFuture<?> fut = multithreadedAsync(
+        IgniteInternalFuture<?> 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/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
index 2401115..0a499b4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.datastructures.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -100,7 +100,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>() {
+            IgniteInternalFuture<?> fut1 = GridTestUtils.runAsync(new Callable<Void>() {
                 @Override public Void call() throws IgniteCheckedException {
                     Ignite ignite = grid(0);
 
@@ -140,7 +140,7 @@ public class GridCachePartitionedQueueEntryMoveSelfTest extends GridCommonAbstra
 
             //latch2.countDown();
 
-            IgniteFuture<?> fut2 = GridTestUtils.runAsync(new Callable<Void>() {
+            IgniteInternalFuture<?> 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/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java
index 9878cb5..172e914 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAbstractJobExecutionTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed;
 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.resources.*;
 import org.apache.ignite.transactions.*;
@@ -121,7 +121,7 @@ public abstract class GridCacheAbstractJobExecutionTest extends GridCommonAbstra
 
         Ignite ignite = grid(0);
 
-        Collection<IgniteFuture<?>> futs = new LinkedList<>();
+        Collection<IgniteInternalFuture<?>> futs = new LinkedList<>();
 
         IgniteCompute comp = ignite.compute().withAsync();
 
@@ -158,7 +158,7 @@ public abstract class GridCacheAbstractJobExecutionTest extends GridCommonAbstra
             futs.add(comp.future());
         }
 
-        for (IgniteFuture<?> fut : futs)
+        for (IgniteInternalFuture<?> 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/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java
index 6f15f64..4bf8d42 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheAtomicTimeoutSelfTest.java
@@ -23,7 +23,6 @@ import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.atomic.*;
@@ -121,7 +120,7 @@ public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest {
 
         commSpi.skipNearRequest = true;
 
-        IgniteFuture<Object> fut = cache.putAsync(key, 1);
+        IgniteInternalFuture<Object> fut = cache.putAsync(key, 1);
 
         Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients");
 
@@ -155,7 +154,7 @@ public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest {
 
         commSpi.skipNearResponse = true;
 
-        IgniteFuture<Object> fut = cache.putAsync(key, 1);
+        IgniteInternalFuture<Object> fut = cache.putAsync(key, 1);
 
         Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients");
 
@@ -189,7 +188,7 @@ public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest {
 
         commSpi.skipDhtRequest = true;
 
-        IgniteFuture<Object> fut = cache.putAsync(key, 1);
+        IgniteInternalFuture<Object> fut = cache.putAsync(key, 1);
 
         Map<UUID, GridCommunicationClient> clients = U.field(commSpi, "clients");
 
@@ -224,7 +223,7 @@ public class GridCacheAtomicTimeoutSelfTest extends GridCommonAbstractTest {
 
         commSpi.skipDhtResponse = true;
 
-        IgniteFuture<Object> fut = cache.putAsync(key, 1);
+        IgniteInternalFuture<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/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
index d904d99..462b84f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheBasicOpAbstractTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -183,7 +184,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");
+            IgniteInternalFuture<String> f1 = cache1.getAsync("async1");
 
             assert f1.get() == null;
 
@@ -200,8 +201,8 @@ public abstract class GridCacheBasicOpAbstractTest extends GridCommonAbstractTes
 
             assert latch.await(5, SECONDS);
 
-            IgniteFuture<String> f2 = cache2.getAsync("async1");
-            IgniteFuture<String> f3 = cache3.getAsync("async1");
+            IgniteInternalFuture<String> f2 = cache2.getAsync("async1");
+            IgniteInternalFuture<String> f3 = cache3.getAsync("async1");
 
             String v2 = f2.get();
             String v3 = f3.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
index c6686cf..f83e151 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheLockAbstractTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed;
 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.*;
@@ -393,7 +393,7 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
         final CountDownLatch l1 = new CountDownLatch(1);
         final CountDownLatch l2 = new CountDownLatch(1);
 
-        IgniteFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     info("Before lock for keys.");
 
@@ -438,7 +438,7 @@ public abstract class GridCacheLockAbstractTest extends GridCommonAbstractTest {
                 }
             }, 1, "TEST-THREAD-1");
 
-        IgniteFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     info("Waiting for latch1...");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
index 6d3351b..1610e4a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultiNodeAbstractTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
+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.*;
@@ -232,7 +233,7 @@ public abstract class GridCacheMultiNodeAbstractTest extends GridCommonAbstractT
         addListener(ignite2, unlockLsnr, EVT_CACHE_OBJECT_UNLOCKED);
         addListener(ignite3, unlockLsnr, EVT_CACHE_OBJECT_UNLOCKED);
 
-        IgniteFuture<Boolean> f1 = cache1.lockAsync(1, 0L);
+        IgniteInternalFuture<Boolean> f1 = cache1.lockAsync(1, 0L);
 
         assert f1.get(10000);
 
@@ -274,9 +275,9 @@ public abstract class GridCacheMultiNodeAbstractTest extends GridCommonAbstractT
         addListener(ignite2, unlockLsnr, EVT_CACHE_OBJECT_UNLOCKED);
         addListener(ignite3, unlockLsnr, EVT_CACHE_OBJECT_UNLOCKED);
 
-        IgniteFuture<Boolean> f1 = cache1.lockAsync(1, 0L);
-        IgniteFuture<Boolean> f2 = cache2.lockAsync(1, 0L);
-        IgniteFuture<Boolean> f3 = cache3.lockAsync(1, 0L);
+        IgniteInternalFuture<Boolean> f1 = cache1.lockAsync(1, 0L);
+        IgniteInternalFuture<Boolean> f2 = cache2.lockAsync(1, 0L);
+        IgniteInternalFuture<Boolean> f3 = cache3.lockAsync(1, 0L);
 
         boolean l1 = false;
         boolean l2 = false;
@@ -361,9 +362,9 @@ public abstract class GridCacheMultiNodeAbstractTest extends GridCommonAbstractT
         addListener(ignite2, lsnr);
         addListener(ignite3, lsnr);
 
-        IgniteFuture<String> f1 = cache1.putAsync(2, "val1");
-        IgniteFuture<String> f2 = cache2.putAsync(2, "val2");
-        IgniteFuture<String> f3 = cache3.putAsync(2, "val3");
+        IgniteInternalFuture<String> f1 = cache1.putAsync(2, "val1");
+        IgniteInternalFuture<String> f2 = cache2.putAsync(2, "val2");
+        IgniteInternalFuture<String> f3 = cache3.putAsync(2, "val3");
 
         String v1 = f1.get(20000);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java
index e57434d..73a9bbe 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheMultithreadedFailoverAbstractTest.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.cache.distributed;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
@@ -357,7 +357,7 @@ public class GridCacheMultithreadedFailoverAbstractTest extends GridCommonAbstra
             putThreads[i] = thread;
         }
 
-        IgniteFuture<?> killNodeFut = null;
+        IgniteInternalFuture<?> killNodeFut = null;
 
         if (nodeKillProbability() > 0) {
             killNodeFut = GridTestUtils.runAsync(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/distributed/GridCacheNodeFailureAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
index 49611c3..9466f5b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheNodeFailureAbstractTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -174,7 +174,7 @@ public abstract class GridCacheNodeFailureAbstractTest extends GridCommonAbstrac
 
             info("Check grid index: " + checkIdx);
 
-            IgniteFuture<?> f = waitForLocalEvent(grid(checkIdx).events(), new P1<IgniteEvent>() {
+            IgniteInternalFuture<?> f = waitForLocalEvent(grid(checkIdx).events(), new P1<IgniteEvent>() {
                 @Override public boolean apply(IgniteEvent e) {
                     info("Received grid event: " + e);
 
@@ -259,7 +259,7 @@ public abstract class GridCacheNodeFailureAbstractTest extends GridCommonAbstrac
 
         assert e.isLocked() : "Entry is not locked for grid [idx=" + checkIdx + ", entry=" + e + ']';
 
-        IgniteFuture<?> f = waitForLocalEvent(grid(checkIdx).events(), new P1<IgniteEvent>() {
+        IgniteInternalFuture<?> f = waitForLocalEvent(grid(checkIdx).events(), new P1<IgniteEvent>() {
             @Override public boolean apply(IgniteEvent e) {
                 info("Received grid event: " + e);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
index c6b3adc..2240552 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest.java
@@ -190,7 +190,7 @@ public abstract class IgniteTxPessimisticOriginatingNodeFailureAbstractSelfTest
 
                     tx.commit();
 
-                    IgniteFuture<IgniteTx> fut = tx.future();
+                    IgniteInternalFuture<IgniteTx> fut = tx.future();
 
                     info("Got future for commitAsync().");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java
index 7f62202..4fffa17 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/IgniteTxPreloadAbstractTest.java
@@ -19,8 +19,8 @@ package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.testframework.*;
 import org.jetbrains.annotations.*;
@@ -76,7 +76,7 @@ public abstract class IgniteTxPreloadAbstractTest extends GridCacheAbstractSelfT
 
         final AtomicInteger gridIdx = new AtomicInteger(1);
 
-        IgniteFuture<?> fut = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     int idx = gridIdx.getAndIncrement();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
index 3435991..433bda5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheColocatedDebugTest.java
@@ -26,7 +26,6 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
@@ -282,7 +281,7 @@ public class GridCacheColocatedDebugTest extends GridCommonAbstractTest {
 
             final int keysCnt = 10;
 
-            IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
                 @Override public void run() {
                     // Make thread-local copy to shuffle keys.
                     List<Integer> threadKeys = new ArrayList<>(keys);
@@ -378,7 +377,7 @@ public class GridCacheColocatedDebugTest extends GridCommonAbstractTest {
 
             final Lock lock = g0.jcache(null).lock(key);
 
-            IgniteFuture<?> unlockFut = multithreadedAsync(new Runnable() {
+            IgniteInternalFuture<?> unlockFut = multithreadedAsync(new Runnable() {
                 @Override public void run() {
                     try {
                         lock.lock();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
index fee7183..b3fa9ee 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionNearReadersSelfTest.java
@@ -255,13 +255,13 @@ public class GridCacheDhtEvictionNearReadersSelfTest extends GridCommonAbstractT
         assert nearOther.peekExx(key) == null;
         assert dhtOther.peekExx(key) == null;
 
-        IgniteFuture<IgniteEvent> futOther =
+        IgniteInternalFuture<IgniteEvent> futOther =
             waitForLocalEvent(grid(other).events(), nodeEvent(other.id()), EVT_CACHE_ENTRY_EVICTED);
 
-        IgniteFuture<IgniteEvent> futBackup =
+        IgniteInternalFuture<IgniteEvent> futBackup =
             waitForLocalEvent(grid(backup).events(), nodeEvent(backup.id()), EVT_CACHE_ENTRY_EVICTED);
 
-        IgniteFuture<IgniteEvent> futPrimary =
+        IgniteInternalFuture<IgniteEvent> futPrimary =
             waitForLocalEvent(grid(primary).events(), nodeEvent(primary.id()), EVT_CACHE_ENTRY_EVICTED);
 
         // Get value on other node, it should be loaded to near cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
index fab65a9..b518b96 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtEvictionSelfTest.java
@@ -243,10 +243,10 @@ public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest {
         assertTrue(entryPrimary.readers().isEmpty());
         assertTrue(entryBackup.readers().isEmpty());
 
-        IgniteFuture<IgniteEvent> futBackup =
+        IgniteInternalFuture<IgniteEvent> futBackup =
             waitForLocalEvent(grid(backup).events(), nodeEvent(backup.id()), EVT_CACHE_ENTRY_EVICTED);
 
-        IgniteFuture<IgniteEvent> futPrimary =
+        IgniteInternalFuture<IgniteEvent> futPrimary =
             waitForLocalEvent(grid(primary).events(), nodeEvent(primary.id()), EVT_CACHE_ENTRY_EVICTED);
 
         // Evict on primary node.
@@ -308,7 +308,7 @@ public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest {
 
         final AtomicInteger cntBackup = new AtomicInteger();
 
-        IgniteFuture<IgniteEvent> futBackup = waitForLocalEvent(backupIgnite.events(), new P1<IgniteEvent>() {
+        IgniteInternalFuture<IgniteEvent> futBackup = waitForLocalEvent(backupIgnite.events(), new P1<IgniteEvent>() {
             @Override public boolean apply(IgniteEvent e) {
                 return e.node().id().equals(backupIgnite.cluster().localNode().id()) &&
                     cntBackup.incrementAndGet() == keyCnt;
@@ -317,7 +317,7 @@ public class GridCacheDhtEvictionSelfTest extends GridCommonAbstractTest {
 
         final AtomicInteger cntPrimary = new AtomicInteger();
 
-        IgniteFuture<IgniteEvent> futPrimary = waitForLocalEvent(primaryIgnite.events(), new P1<IgniteEvent>() {
+        IgniteInternalFuture<IgniteEvent> futPrimary = waitForLocalEvent(primaryIgnite.events(), new P1<IgniteEvent>() {
             @Override public boolean apply(IgniteEvent e) {
                 return e.node().id().equals(primaryIgnite.cluster().localNode().id()) &&
                     cntPrimary.incrementAndGet() == keyCnt;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
index 9a0224f..c48e353 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadPutGetSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 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.internal.processors.cache.distributed.dht.preloader.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -186,7 +186,7 @@ public class GridCacheDhtPreloadPutGetSelfTest extends GridCommonAbstractTest {
 
             final AtomicBoolean done = new AtomicBoolean();
 
-            IgniteFuture fut1 = GridTestUtils.runMultiThreadedAsync(
+            IgniteInternalFuture fut1 = GridTestUtils.runMultiThreadedAsync(
                 new Callable<Object>() {
                     @Nullable @Override public Object call() throws Exception {
                         Ignite g2 = startGrid(2);
@@ -222,7 +222,7 @@ public class GridCacheDhtPreloadPutGetSelfTest extends GridCommonAbstractTest {
                 "reader"
             );
 
-            IgniteFuture fut2 = GridTestUtils.runMultiThreadedAsync(
+            IgniteInternalFuture fut2 = GridTestUtils.runMultiThreadedAsync(
                 new Callable<Object>() {
                     @Nullable @Override public Object call() throws Exception {
                         writeLatch.await();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
index d13f647..1d77b81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cluster.*;
 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.processors.cache.distributed.dht.preloader.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -257,7 +258,7 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
             info(">>> Finished checking nodes [keyCnt=" + keyCnt + ", nodeCnt=" + nodeCnt + ", grids=" +
                 U.grids2names(ignites) + ']');
 
-            Collection<IgniteFuture<?>> futs = new LinkedList<>();
+            Collection<IgniteInternalFuture<?>> futs = new LinkedList<>();
 
             Ignite last = F.last(ignites);
 
@@ -536,7 +537,7 @@ public class GridCacheDhtPreloadSelfTest extends GridCommonAbstractTest {
 
                 it.remove();
 
-                Collection<IgniteFuture<?>> futs = new LinkedList<>();
+                Collection<IgniteInternalFuture<?>> futs = new LinkedList<>();
 
                 for (Ignite gg : ignites)
                     futs.add(waitForLocalEvent(gg.events(), new P1<IgniteEvent>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
index bce6cf8..e81c3ec 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheDhtPreloadStartStopSelfTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -216,7 +215,7 @@ public class GridCacheDhtPreloadStartStopSelfTest extends GridCommonAbstractTest
                 = ((GridKernal)g1).context().cache().context().exchange();
 
             // Wait for exchanges to complete.
-            for (IgniteFuture<?> fut : exchMgr.exchangeFutures())
+            for (IgniteInternalFuture<?> fut : exchMgr.exchangeFutures())
                 fut.get();
 
             CacheAffinity<Integer> aff = affinity(c1);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
index b71c7da..7865567 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCachePartitionedTopologyChangeSelfTest.java
@@ -136,7 +136,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
 
             GridKernal[] nodes = new GridKernal[] {(GridKernal)grid(0), (GridKernal)grid(1)};
 
-            Collection<IgniteFuture> futs = new ArrayList<>();
+            Collection<IgniteInternalFuture> futs = new ArrayList<>();
 
             final CountDownLatch startLatch = new CountDownLatch(1);
 
@@ -180,7 +180,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
                 }
             }
 
-            IgniteFuture<?> startFut = multithreadedAsync(new Runnable() {
+            IgniteInternalFuture<?> startFut = multithreadedAsync(new Runnable() {
                 @Override public void run() {
                     try {
                         startGrid(2);
@@ -201,7 +201,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
 
             startLatch.countDown();
 
-            for (IgniteFuture fut : futs)
+            for (IgniteInternalFuture fut : futs)
                 fut.get(1000);
 
             startFut.get();
@@ -229,7 +229,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
 
             final CountDownLatch commitLatch = new CountDownLatch(1);
 
-            Collection<IgniteFuture> futs = new ArrayList<>();
+            Collection<IgniteInternalFuture> futs = new ArrayList<>();
 
             for (final GridKernal node : nodes) {
                 printDistribution(node);
@@ -292,7 +292,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
 
             // Now start new node. We do it in a separate thread since startGrid
             // should block until partition exchange completes.
-            IgniteFuture startFut = multithreadedAsync(new Runnable() {
+            IgniteInternalFuture startFut = multithreadedAsync(new Runnable() {
                 @Override public void run() {
                     try {
                         Ignite g3 = startGrid(3);
@@ -312,7 +312,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
             assertFalse("Node was able to join the grid while there exist pending transactions.", startFut.isDone());
 
             // Now check that new transactions will wait for new topology version to become available.
-            Collection<IgniteFuture> txFuts = new ArrayList<>(nodes.length);
+            Collection<IgniteInternalFuture> txFuts = new ArrayList<>(nodes.length);
 
             for (final Ignite g : nodes) {
                 txFuts.add(multithreadedAsync(new Runnable() {
@@ -340,19 +340,19 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
 
             Thread.sleep(500);
 
-            for (IgniteFuture txFut : txFuts)
+            for (IgniteInternalFuture txFut : txFuts)
                 assertFalse("New transaction was completed before new node joined topology", txFut.isDone());
 
             info(">>> Committing pending transactions.");
 
             commitLatch.countDown();
 
-            for (IgniteFuture fut : futs)
+            for (IgniteInternalFuture fut : futs)
                 fut.get(1000);
 
             startFut.get(1000);
 
-            for (IgniteFuture txFut : txFuts)
+            for (IgniteInternalFuture txFut : txFuts)
                 txFut.get(1000);
         }
         finally {
@@ -381,7 +381,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
             info(">>> Started nodes [g0=" + g0.localNode().id() + ", g1=" + g1.localNode().id() + ", g2=" +
                 g2.localNode().id() + ", g3=" + g3.localNode().id() + ']');
 
-            Collection<IgniteFuture> futs = new ArrayList<>();
+            Collection<IgniteInternalFuture> futs = new ArrayList<>();
 
             printDistribution(g3);
 
@@ -448,7 +448,7 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
             leaveLatch.await();
 
             // Now check that new transactions will wait for new topology version to become available.
-            Collection<IgniteFuture> txFuts = new ArrayList<>(nodes.length);
+            Collection<IgniteInternalFuture> txFuts = new ArrayList<>(nodes.length);
 
             for (final Ignite g : nodes) {
                 txFuts.add(multithreadedAsync(new Runnable() {
@@ -474,17 +474,17 @@ public class GridCachePartitionedTopologyChangeSelfTest extends GridCommonAbstra
 
             Thread.sleep(500);
 
-            for (IgniteFuture txFut : txFuts)
+            for (IgniteInternalFuture txFut : txFuts)
                 assertFalse("New transaction was completed before old transactions were committed", txFut.isDone());
 
             info(">>> Committing pending transactions.");
 
             commitLatch.countDown();
 
-            for (IgniteFuture fut : futs)
+            for (IgniteInternalFuture fut : futs)
                 fut.get(1000);
 
-            for (IgniteFuture txFut : txFuts)
+            for (IgniteInternalFuture txFut : txFuts)
                 txFut.get(1000);
 
             for (int i = 0; i < 3; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
index b60a86e..12b2aba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridCacheAtomicInvalidPartitionHandlingSelfTest.java
@@ -24,7 +24,6 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.internal.managers.communication.*;
 import org.apache.ignite.spi.communication.tcp.*;
@@ -177,7 +176,7 @@ public class GridCacheAtomicInvalidPartitionHandlingSelfTest extends GridCommonA
             System.err.println("FINISHED PUTS");
 
             // Start put threads.
-            IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     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/distributed/near/GridCachePartitionedEntryLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEntryLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEntryLockSelfTest.java
index cf93827..2d0c63e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEntryLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedEntryLockSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 
@@ -61,7 +62,7 @@ public class GridCachePartitionedEntryLockSelfTest extends GridCacheAbstractSelf
 
                 comp.call(new Callable<Boolean>() {
                     @Override public Boolean call() throws Exception {
-                        IgniteFuture<Boolean> f = e.lockAsync(1000);
+                        IgniteInternalFuture<Boolean> f = e.lockAsync(1000);
 
                         try {
                             f.get(100);
@@ -83,7 +84,7 @@ public class GridCachePartitionedEntryLockSelfTest extends GridCacheAbstractSelf
                     }
                 });
 
-                IgniteFuture<Boolean> f = comp.future();
+                IgniteInternalFuture<Boolean> f = comp.future();
 
                 // Let another thread start.
                 Thread.sleep(300);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
index cbf1a7e..da3bab4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedExplicitLockNodeFailureSelfTest.java
@@ -21,7 +21,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
-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.*;
@@ -114,7 +114,7 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom
             assert e.isLocked() : "Entry is not locked for grid [idx=" + checkIdx + ", entry=" + e + ']';
         }
 
-        Collection<IgniteFuture<?>> futs = new LinkedList<>();
+        Collection<IgniteInternalFuture<?>> futs = new LinkedList<>();
 
         for (int i = 1; i < GRID_CNT; i++) {
             futs.add(
@@ -129,7 +129,7 @@ public class GridCachePartitionedExplicitLockNodeFailureSelfTest extends GridCom
 
         stopGrid(idx);
 
-        for (IgniteFuture<?> fut : futs)
+        for (IgniteInternalFuture<?> fut : futs)
             fut.get();
 
         for (int i = 0; i < 3; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
index 86ba60b..e18421f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedMultiNodeFullApiSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -452,7 +453,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
                     @Override public Boolean call() throws Exception {
                         syncLatch.countDown();
 
-                        IgniteFuture<Boolean> f = e.lockAsync(15000);
+                        IgniteInternalFuture<Boolean> f = e.lockAsync(15000);
 
                         try {
                             f.get(100);
@@ -477,7 +478,7 @@ public class GridCachePartitionedMultiNodeFullApiSelfTest extends GridCacheParti
                     }
                 });
 
-            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/distributed/near/GridCachePartitionedQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedQueryMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedQueryMultiThreadedSelfTest.java
index 1ff2317..d2820ae 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedQueryMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedQueryMultiThreadedSelfTest.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.*;
@@ -144,7 +144,7 @@ public class GridCachePartitionedQueryMultiThreadedSelfTest extends GridCommonAb
         final AtomicLong luceneCnt = new AtomicLong();
 
         // Start lucene query threads.
-        IgniteFuture<?> futLucene = GridTestUtils.runMultiThreadedAsync(new CAX() {
+        IgniteInternalFuture<?> futLucene = GridTestUtils.runMultiThreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 while (!done.get()) {
                     CacheQuery<Map.Entry<UUID, Person>> masters = cache0.queries().createFullTextQuery(
@@ -165,7 +165,7 @@ public class GridCachePartitionedQueryMultiThreadedSelfTest extends GridCommonAb
         final AtomicLong sqlCnt = new AtomicLong();
 
         // Start sql query threads.
-        IgniteFuture<?> futSql = GridTestUtils.runMultiThreadedAsync(new CAX() {
+        IgniteInternalFuture<?> futSql = GridTestUtils.runMultiThreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 while (!done.get()) {
                     CacheQuery<Map.Entry<UUID, Person>> bachelors =

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
index db674fc..50eefad 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCachePartitionedTxSalvageSelfTest.java
@@ -23,7 +23,6 @@ import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -196,7 +195,7 @@ public class GridCachePartitionedTxSalvageSelfTest extends GridCommonAbstractTes
 
         final Collection<Integer> keys = nearKeys(ignite);
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 GridCache<Object, Object> c = cache(0);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedEvictionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedEvictionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedEvictionSelfTest.java
index 5a23839..ac89ea0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedEvictionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedEvictionSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.replicated;
 
 import org.apache.ignite.cache.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -78,7 +79,7 @@ public class GridCacheReplicatedEvictionSelfTest extends GridCacheAbstractSelfTe
                 assertNotNull(cache(g).peek(String.valueOf(i)));
         }
 
-        Collection<IgniteFuture<IgniteEvent>> futs = new ArrayList<>();
+        Collection<IgniteInternalFuture<IgniteEvent>> futs = new ArrayList<>();
 
         for (int g = 0 ; g < gridCount(); g++)
             futs.add(waitForLocalEvent(grid(g).events(), nodeEvent(grid(g).localNode().id()), EVT_CACHE_ENTRY_EVICTED));
@@ -90,7 +91,7 @@ public class GridCacheReplicatedEvictionSelfTest extends GridCacheAbstractSelfTe
             }
         }
 
-        for (IgniteFuture<IgniteEvent> fut : futs)
+        for (IgniteInternalFuture<IgniteEvent> fut : futs)
             fut.get(3000);
 
         boolean evicted = GridTestUtils.waitForCondition(new PA() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
index f2d07ec..f6ffaa5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheBatchEvictUnswapSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.GridCache;
 import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
@@ -124,7 +125,7 @@ public class GridCacheBatchEvictUnswapSelfTest extends GridCacheAbstractSelfTest
 
         info("Finished load cache.");
 
-        IgniteFuture<?> evictFut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> evictFut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 Collection<Long> keys = new ArrayList<>(batchSize);
 
@@ -151,7 +152,7 @@ public class GridCacheBatchEvictUnswapSelfTest extends GridCacheAbstractSelfTest
 
         final AtomicInteger unswappedKeys = new AtomicInteger();
 
-        IgniteFuture<?> unswapFut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> unswapFut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     Collection<Long> keys = new ArrayList<>(batchSize);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
index a5a4fba..d4be3a6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionConsistencySelfTest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.cache.eviction.*;
 import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.cache.eviction.lru.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -176,7 +176,7 @@ public class GridCacheConcurrentEvictionConsistencySelfTest extends GridCommonAb
 
             long start = System.currentTimeMillis();
 
-            IgniteFuture<?> fut = multithreadedAsync(
+            IgniteInternalFuture<?> fut = multithreadedAsync(
                 new Callable<Object>() {
                     @Override
                     public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
index 5726c11..ffc80a8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheConcurrentEvictionsSelfTest.java
@@ -23,7 +23,7 @@ import org.apache.ignite.cache.eviction.*;
 import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.cache.eviction.lru.*;
 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.*;
@@ -152,7 +152,7 @@ public class GridCacheConcurrentEvictionsSelfTest extends GridCommonAbstractTest
 
             long start = System.currentTimeMillis();
 
-            IgniteFuture<?> fut = multithreadedAsync(
+            IgniteInternalFuture<?> fut = multithreadedAsync(
                 new Callable<Object>() {
                     @Override public Object call() throws Exception {
                         for (int i = 0; i < iterCnt; i++) {


[09/50] [abbrv] incubator-ignite git commit: Merge branch 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into sprint-1

Posted by vo...@apache.org.
Merge branch 'sprint-1' of https://git-wip-us.apache.org/repos/asf/incubator-ignite into sprint-1


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

Branch: refs/heads/ignite-16
Commit: 6cab420e72bada8a9658ce893faec00710e08cd4
Parents: 8643e94 ddbd18e
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jan 28 19:20:46 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jan 28 19:20:46 2015 +0300

----------------------------------------------------------------------
 .../datagrid/CacheDataLoaderExample.java        |  15 +-
 .../org/apache/ignite/internal/GridKernal.java  |  30 +--
 .../processors/cache/GridCacheContext.java      |  43 +++++
 .../processors/cache/GridCacheEntryEx.java      |   8 +
 .../processors/cache/GridCacheMapEntry.java     | 116 +++++++++---
 .../cache/GridCacheUpdateAtomicResult.java      |   6 +-
 .../processors/cache/GridDrResolveResult.java   |  63 -------
 .../GridDistributedTxRemoteAdapter.java         |  47 +++--
 .../dht/atomic/GridDhtAtomicCache.java          |  10 +-
 .../processors/cache/dr/GridCacheDrManager.java |  57 ++----
 .../cache/dr/os/GridOsCacheDrManager.java       |  35 +---
 .../cache/transactions/IgniteTxAdapter.java     |  45 +++++
 .../transactions/IgniteTxLocalAdapter.java      |  52 +++--
 ...ridCacheVersionAbstractConflictResolver.java |  56 ++++++
 .../GridCacheVersionConflictContext.java        |  73 +++++++
 .../GridCacheVersionConflictContextImpl.java    | 188 +++++++++++++++++++
 .../GridCacheVersionConflictResolver.java       |  59 ++++++
 .../version/GridCacheVersionedEntryEx.java      |   2 +-
 .../dataload/IgniteDataLoaderImpl.java          |  29 ++-
 .../processors/cache/GridCacheTestEntryEx.java  |   5 +
 ...ClusterMetricsSnapshotSerializeSelfTest.java |  52 ++++-
 21 files changed, 760 insertions(+), 231 deletions(-)
----------------------------------------------------------------------



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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index 17ca50c..d113cda 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -100,8 +100,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     for (Map.Entry<Long, GridFutureAdapter<QueryResult<K, V>>> entry : futs.entrySet()) {
                         final Object recipient = recipient(nodeId, entry.getKey());
 
-                        entry.getValue().listenAsync(new CIX1<IgniteFuture<QueryResult<K, V>>>() {
-                            @Override public void applyx(IgniteFuture<QueryResult<K, V>> f) throws IgniteCheckedException {
+                        entry.getValue().listenAsync(new CIX1<IgniteInternalFuture<QueryResult<K, V>>>() {
+                            @Override public void applyx(IgniteInternalFuture<QueryResult<K, V>> f) throws IgniteCheckedException {
                                 f.get().closeIfNotShared(recipient);
                             }
                         });
@@ -114,8 +114,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     for (Map.Entry<Long, GridFutureAdapter<FieldsResult>> entry : fieldsFuts.entrySet()) {
                         final Object recipient = recipient(nodeId, entry.getKey());
 
-                        entry.getValue().listenAsync(new CIX1<IgniteFuture<FieldsResult>>() {
-                            @Override public void applyx(IgniteFuture<FieldsResult> f)
+                        entry.getValue().listenAsync(new CIX1<IgniteInternalFuture<FieldsResult>>() {
+                            @Override public void applyx(IgniteInternalFuture<FieldsResult> f)
                                 throws IgniteCheckedException {
                                 f.get().closeIfNotShared(recipient);
                             }
@@ -186,7 +186,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param valType Value type.
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
-    public IgniteFuture<?> rebuildIndexes(Class<?> valType) {
+    public IgniteInternalFuture<?> rebuildIndexes(Class<?> valType) {
         return rebuildIndexes(valType.getName());
     }
 
@@ -196,7 +196,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param typeName Value type name.
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
-    public IgniteFuture<?> rebuildIndexes(String typeName) {
+    public IgniteInternalFuture<?> rebuildIndexes(String typeName) {
         if (!enterBusy())
             throw new IllegalStateException("Failed to rebuild indexes (grid is stopping).");
 
@@ -213,7 +213,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      *
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
-    public IgniteFuture<?> rebuildAllIndexes() {
+    public IgniteInternalFuture<?> rebuildAllIndexes() {
         if (!enterBusy())
             throw new IllegalStateException("Failed to rebuild indexes (grid is stopping).");
 
@@ -1481,7 +1481,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         Map<Long, GridFutureAdapter<QueryResult<K, V>>> futs = qryIters.get(sndId);
 
         if (futs != null) {
-            IgniteFuture<QueryResult<K, V>> fut;
+            IgniteInternalFuture<QueryResult<K, V>> fut;
 
             synchronized (futs) {
                 fut = futs.remove(reqId);
@@ -1607,7 +1607,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         Map<Long, GridFutureAdapter<FieldsResult>> futs = fieldsQryRes.get(sndId);
 
         if (futs != null) {
-            IgniteFuture<FieldsResult> fut;
+            IgniteInternalFuture<FieldsResult> fut;
 
             synchronized (futs) {
                 fut = futs.remove(reqId);
@@ -1701,14 +1701,14 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
             Collection<Collection<CacheSqlMetadata>> res = new ArrayList<>(nodes.size() + 1);
 
-            IgniteFuture<Collection<Collection<CacheSqlMetadata>>> rmtFut = null;
+            IgniteInternalFuture<Collection<Collection<CacheSqlMetadata>>> rmtFut = null;
 
             // Get metadata from remote nodes.
             if (!nodes.isEmpty())
                 rmtFut = cctx.closures().callAsyncNoFailover(BROADCAST, F.asSet(job), nodes, true);
 
             // Get local metadata.
-            IgniteFuture<Collection<CacheSqlMetadata>> locFut = cctx.closures().callLocalSafe(job, true);
+            IgniteInternalFuture<Collection<CacheSqlMetadata>> locFut = cctx.closures().callLocalSafe(job, true);
 
             if (rmtFut != null)
                 res.addAll(rmtFut.get());
@@ -2610,7 +2610,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<V> reloadAsync() {
+        @Override public IgniteInternalFuture<V> reloadAsync() {
             throw new UnsupportedOperationException();
         }
 
@@ -2652,7 +2652,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<V> getAsync() {
+        @Override public IgniteInternalFuture<V> getAsync() {
             return new GridFinishedFuture<V>(cctx.kernalContext(), getValue());
         }
 
@@ -2667,7 +2667,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<V> setAsync(V val, IgnitePredicate<CacheEntry<K, V>>... filter) {
+        @Override public IgniteInternalFuture<V> setAsync(V val, IgnitePredicate<CacheEntry<K, V>>... filter) {
             throw new UnsupportedOperationException();
         }
 
@@ -2677,7 +2677,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<V> setIfAbsentAsync(V val) {
+        @Override public IgniteInternalFuture<V> setIfAbsentAsync(V val) {
             throw new UnsupportedOperationException();
         }
 
@@ -2687,7 +2687,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<Boolean> setxAsync(V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+        @Override public IgniteInternalFuture<Boolean> setxAsync(V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
             throw new UnsupportedOperationException();
         }
 
@@ -2697,7 +2697,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<Boolean> setxIfAbsentAsync(V val) {
+        @Override public IgniteInternalFuture<Boolean> setxIfAbsentAsync(V val) {
             throw new UnsupportedOperationException();
         }
 
@@ -2707,7 +2707,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<V> replaceAsync(V val) {
+        @Override public IgniteInternalFuture<V> replaceAsync(V val) {
             throw new UnsupportedOperationException();
         }
 
@@ -2717,7 +2717,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<Boolean> replacexAsync(V val) {
+        @Override public IgniteInternalFuture<Boolean> replacexAsync(V val) {
             throw new UnsupportedOperationException();
         }
 
@@ -2727,7 +2727,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<Boolean> replaceAsync(V oldVal, V newVal) {
+        @Override public IgniteInternalFuture<Boolean> replaceAsync(V oldVal, V newVal) {
             throw new UnsupportedOperationException();
         }
 
@@ -2737,7 +2737,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<V> removeAsync(IgnitePredicate<CacheEntry<K, V>>... filter) {
+        @Override public IgniteInternalFuture<V> removeAsync(IgnitePredicate<CacheEntry<K, V>>... filter) {
             throw new UnsupportedOperationException();
         }
 
@@ -2747,7 +2747,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<Boolean> removexAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+        @Override public IgniteInternalFuture<Boolean> removexAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
             throw new UnsupportedOperationException();
         }
 
@@ -2757,7 +2757,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<Boolean> removeAsync(V val) {
+        @Override public IgniteInternalFuture<Boolean> removeAsync(V val) {
             throw new UnsupportedOperationException();
         }
 
@@ -2782,7 +2782,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<Boolean> lockAsync(long timeout,
+        @Override public IgniteInternalFuture<Boolean> lockAsync(long timeout,
             @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
             throw new UnsupportedOperationException();
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetadataAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetadataAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetadataAware.java
index f6cc15d..e47f845 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetadataAware.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryMetadataAware.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.query.*;
-import org.apache.ignite.lang.*;
 
 import java.util.*;
 
@@ -29,5 +29,5 @@ public interface GridCacheQueryMetadataAware {
     /**
      * @return Future to retrieve metadata.
      */
-    public IgniteFuture<List<GridQueryFieldMetadata>> metadata();
+    public IgniteInternalFuture<List<GridQueryFieldMetadata>> metadata();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java
index f4084a2..5355cb1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/continuous/GridCacheContinuousQueryEntry.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.query.continuous;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
@@ -290,7 +291,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getAsync() {
+    @Override public IgniteInternalFuture<V> getAsync() {
         assert impl != null;
 
         ctx.denyOnFlag(LOCAL);
@@ -308,7 +309,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> reloadAsync() {
+    @Override public IgniteInternalFuture<V> reloadAsync() {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -390,7 +391,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> setAsync(V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<V> setAsync(V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -408,7 +409,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> setIfAbsentAsync(V val) {
+    @Override public IgniteInternalFuture<V> setIfAbsentAsync(V val) {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -427,7 +428,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> setxAsync(V val,
+    @Override public IgniteInternalFuture<Boolean> setxAsync(V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         assert impl != null;
 
@@ -446,7 +447,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> setxIfAbsentAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> setxIfAbsentAsync(V val) {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -464,7 +465,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(V val) {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -482,7 +483,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(V val) {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -500,7 +501,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(V oldVal, V newVal) {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -519,7 +520,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<V> removeAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -537,7 +538,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<Boolean> removexAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -555,7 +556,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(V val) {
         assert impl != null;
 
         ctx.denyOnFlag(READ);
@@ -600,7 +601,7 @@ public class GridCacheContinuousQueryEntry<K, V> implements CacheEntry<K, V>, Gr
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAsync(long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAsync(long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         assert impl != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
index 32e11ea..de7416e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTransactionsImpl.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -221,7 +220,7 @@ public class IgniteTransactionsImpl<K, V> implements IgniteTransactionsEx {
         if (ctx.hasFlag(SYNC_COMMIT))
             tx0.syncCommit(true);
 
-        IgniteFuture<?> lockFut = tx0.groupLockAsync(ctx, (Collection)F.asList(grpLockKey));
+        IgniteInternalFuture<?> lockFut = tx0.groupLockAsync(ctx, (Collection)F.asList(grpLockKey));
 
         try {
             lockFut.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
index e079a5c..10843ec 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxAdapter.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.transactions;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -557,7 +558,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
     }
 
     /** {@inheritDoc} */
-    @Override public <R> IgniteFuture<R> future() {
+    @Override public <R> IgniteInternalFuture<R> future() {
         throw new UnsupportedOperationException("future() should not be called on IgniteTxAdapter directly.");
     }
 
@@ -949,7 +950,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
 
     /** {@inheritDoc} */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
-    @Override public IgniteFuture<IgniteTx> finishFuture() {
+    @Override public IgniteInternalFuture<IgniteTx> finishFuture() {
         GridFutureAdapter<IgniteTx> fut = finFut.get();
 
         if (fut == null) {
@@ -1569,7 +1570,7 @@ public abstract class IgniteTxAdapter<K, V> extends GridMetadataAwareAdapter
         }
 
         /** {@inheritDoc} */
-        @Override public <R> IgniteFuture<R> future() {
+        @Override public <R> IgniteInternalFuture<R> future() {
             throw new IllegalStateException("Deserialized transaction can only be used as read-only.");
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEx.java
index c3961a5..63e4786 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxEx.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.transactions;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.lang.*;
@@ -382,7 +383,7 @@ public interface IgniteTxEx<K, V> extends IgniteTx, GridTimeoutObject {
      *
      * @return Future for prepare step.
      */
-    public IgniteFuture<IgniteTxEx<K, V>> prepareAsync();
+    public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync();
 
     /**
      * @param endVer End version (a.k.a. <tt>'tnc'</tt> or <tt>'transaction number counter'</tt>)
@@ -406,7 +407,7 @@ public interface IgniteTxEx<K, V> extends IgniteTx, GridTimeoutObject {
     /**
      * @return Future for transaction completion.
      */
-    public IgniteFuture<IgniteTx> finishFuture();
+    public IgniteInternalFuture<IgniteTx> finishFuture();
 
     /**
      * @param state Transaction state.
@@ -434,14 +435,14 @@ public interface IgniteTxEx<K, V> extends IgniteTx, GridTimeoutObject {
      *
      * @return Rollback future.
      */
-    public IgniteFuture<IgniteTx> rollbackAsync();
+    public IgniteInternalFuture<IgniteTx> rollbackAsync();
 
     /**
      * Asynchronously commits this transaction by initiating {@code two-phase-commit} process.
      *
      * @return Future for commit operation.
      */
-    public IgniteFuture<IgniteTx> commitAsync();
+    public IgniteInternalFuture<IgniteTx> commitAsync();
 
     /**
      * Callback invoked whenever there is a lock that has been acquired

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
index d539f15..fb94cd2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxHandler.java
@@ -19,11 +19,11 @@ package org.apache.ignite.internal.processors.cache.transactions;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
@@ -52,7 +52,7 @@ public class IgniteTxHandler<K, V> {
     /** Shared cache context. */
     private GridCacheSharedContext<K, V> ctx;
 
-    public IgniteFuture<IgniteTxEx<K, V>> processNearTxPrepareRequest(final UUID nearNodeId,
+    public IgniteInternalFuture<IgniteTxEx<K, V>> processNearTxPrepareRequest(final UUID nearNodeId,
         final GridNearTxPrepareRequest<K, V> req) {
         return prepareTx(nearNodeId, null, req);
     }
@@ -148,7 +148,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Near prepare request.
      * @return Future for transaction.
      */
-    public IgniteFuture<IgniteTxEx<K, V>> prepareTx(final UUID nearNodeId, @Nullable GridNearTxLocal<K, V> locTx,
+    public IgniteInternalFuture<IgniteTxEx<K, V>> prepareTx(final UUID nearNodeId, @Nullable GridNearTxLocal<K, V> locTx,
         final GridNearTxPrepareRequest<K, V> req) {
         assert nearNodeId != null;
         assert req != null;
@@ -174,20 +174,20 @@ public class IgniteTxHandler<K, V> {
      * @param req Near prepare request.
      * @return Prepare future.
      */
-    private IgniteFuture<IgniteTxEx<K, V>> prepareColocatedTx(final GridNearTxLocal<K, V> locTx,
+    private IgniteInternalFuture<IgniteTxEx<K, V>> prepareColocatedTx(final GridNearTxLocal<K, V> locTx,
         final GridNearTxPrepareRequest<K, V> req) {
 
-        IgniteFuture<Object> fut = new GridFinishedFutureEx<>(); // TODO force preload keys.
+        IgniteInternalFuture<Object> fut = new GridFinishedFutureEx<>(); // TODO force preload keys.
 
         return new GridEmbeddedFuture<>(
             ctx.kernalContext(),
             fut,
-            new C2<Object, Exception, IgniteFuture<IgniteTxEx<K, V>>>() {
-                @Override public IgniteFuture<IgniteTxEx<K, V>> apply(Object o, Exception ex) {
+            new C2<Object, Exception, IgniteInternalFuture<IgniteTxEx<K, V>>>() {
+                @Override public IgniteInternalFuture<IgniteTxEx<K, V>> apply(Object o, Exception ex) {
                     if (ex != null)
                         throw new GridClosureException(ex);
 
-                    IgniteFuture<IgniteTxEx<K, V>> fut = locTx.prepareAsyncLocal(req.reads(), req.writes(),
+                    IgniteInternalFuture<IgniteTxEx<K, V>> fut = locTx.prepareAsyncLocal(req.reads(), req.writes(),
                         req.transactionNodes(), req.last(), req.lastBackups());
 
                     if (locTx.isRollbackOnly())
@@ -220,7 +220,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Near prepare request.
      * @return Prepare future.
      */
-    private IgniteFuture<IgniteTxEx<K, V>> prepareNearTx(final UUID nearNodeId,
+    private IgniteInternalFuture<IgniteTxEx<K, V>> prepareNearTx(final UUID nearNodeId,
         final GridNearTxPrepareRequest<K, V> req) {
         ClusterNode nearNode = ctx.node(nearNodeId);
 
@@ -284,7 +284,7 @@ public class IgniteTxHandler<K, V> {
         }
 
         if (tx != null) {
-            IgniteFuture<IgniteTxEx<K, V>> fut = tx.prepareAsync(req.reads(), req.writes(),
+            IgniteInternalFuture<IgniteTxEx<K, V>> fut = tx.prepareAsync(req.reads(), req.writes(),
                 req.dhtVersions(), req.messageId(), req.miniId(), req.transactionNodes(), req.last(),
                 req.lastBackups());
 
@@ -299,8 +299,8 @@ public class IgniteTxHandler<K, V> {
 
             final GridDhtTxLocal<K, V> tx0 = tx;
 
-            fut.listenAsync(new CI1<IgniteFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteFuture<IgniteTxEx<K, V>> txFut) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> txFut) {
                     try {
                         txFut.get();
                     }
@@ -401,7 +401,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Request.
      * @return Future.
      */
-    @Nullable public IgniteFuture<IgniteTx> processNearTxFinishRequest(UUID nodeId, GridNearTxFinishRequest<K, V> req) {
+    @Nullable public IgniteInternalFuture<IgniteTx> processNearTxFinishRequest(UUID nodeId, GridNearTxFinishRequest<K, V> req) {
         return finish(nodeId, null, req);
     }
 
@@ -410,7 +410,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Request.
      * @return Future.
      */
-    @Nullable public IgniteFuture<IgniteTx> finish(UUID nodeId, @Nullable GridNearTxLocal<K, V> locTx,
+    @Nullable public IgniteInternalFuture<IgniteTx> finish(UUID nodeId, @Nullable GridNearTxLocal<K, V> locTx,
         GridNearTxFinishRequest<K, V> req) {
         assert nodeId != null;
         assert req != null;
@@ -422,12 +422,12 @@ public class IgniteTxHandler<K, V> {
         if (log.isDebugEnabled())
             log.debug("Processing near tx finish request [nodeId=" + nodeId + ", req=" + req + "]");
 
-        IgniteFuture<IgniteTx> colocatedFinishFut = null;
+        IgniteInternalFuture<IgniteTx> colocatedFinishFut = null;
 
         if (locTx != null && locTx.colocatedLocallyMapped())
             colocatedFinishFut = finishColocatedLocal(req.commit(), locTx);
 
-        IgniteFuture<IgniteTx> nearFinishFut = null;
+        IgniteInternalFuture<IgniteTx> nearFinishFut = null;
 
         if (locTx == null || locTx.nearLocallyMapped()) {
             if (locTx != null)
@@ -459,7 +459,7 @@ public class IgniteTxHandler<K, V> {
      * @param req Finish request.
      * @return Finish future.
      */
-    private IgniteFuture<IgniteTx> finishDhtLocal(UUID nodeId, @Nullable GridNearTxLocal<K, V> locTx,
+    private IgniteInternalFuture<IgniteTx> finishDhtLocal(UUID nodeId, @Nullable GridNearTxLocal<K, V> locTx,
         GridNearTxFinishRequest<K, V> req) {
         GridCacheVersion dhtVer = ctx.tm().mappedVersion(req.version());
 
@@ -563,7 +563,7 @@ public class IgniteTxHandler<K, V> {
                 if (tx.pessimistic())
                     tx.prepare();
 
-                IgniteFuture<IgniteTx> commitFut = tx.commitAsync();
+                IgniteInternalFuture<IgniteTx> commitFut = tx.commitAsync();
 
                 // Only for error logging.
                 commitFut.listenAsync(CU.errorLogger(log));
@@ -579,7 +579,7 @@ public class IgniteTxHandler<K, V> {
                 tx.nearFinishFutureId(req.futureId());
                 tx.nearFinishMiniId(req.miniId());
 
-                IgniteFuture<IgniteTx> rollbackFut = tx.rollbackAsync();
+                IgniteInternalFuture<IgniteTx> rollbackFut = tx.rollbackAsync();
 
                 // Only for error logging.
                 rollbackFut.listenAsync(CU.errorLogger(log));
@@ -591,7 +591,7 @@ public class IgniteTxHandler<K, V> {
             U.error(log, "Failed completing transaction [commit=" + req.commit() + ", tx=" + tx + ']', e);
 
             if (tx != null) {
-                IgniteFuture<IgniteTx> rollbackFut = tx.rollbackAsync();
+                IgniteInternalFuture<IgniteTx> rollbackFut = tx.rollbackAsync();
 
                 // Only for error logging.
                 rollbackFut.listenAsync(CU.errorLogger(log));
@@ -608,7 +608,7 @@ public class IgniteTxHandler<K, V> {
      * @param tx Transaction to commit.
      * @return Future.
      */
-    public IgniteFuture<IgniteTx> finishColocatedLocal(boolean commit, GridNearTxLocal<K, V> tx) {
+    public IgniteInternalFuture<IgniteTx> finishColocatedLocal(boolean commit, GridNearTxLocal<K, V> tx) {
         try {
             if (commit) {
                 if (!tx.markFinalizing(USER_FINISH)) {
@@ -1448,10 +1448,10 @@ public class IgniteTxHandler<K, V> {
         if (log.isDebugEnabled())
             log.debug("Processing check committed transaction request [nodeId=" + nodeId + ", req=" + req + ']');
 
-        IgniteFuture<GridCacheCommittedTxInfo<K, V>> infoFut = ctx.tm().checkPessimisticTxCommitted(req);
+        IgniteInternalFuture<GridCacheCommittedTxInfo<K, V>> infoFut = ctx.tm().checkPessimisticTxCommitted(req);
 
-        infoFut.listenAsync(new CI1<IgniteFuture<GridCacheCommittedTxInfo<K, V>>>() {
-            @Override public void apply(IgniteFuture<GridCacheCommittedTxInfo<K, V>> infoFut) {
+        infoFut.listenAsync(new CI1<IgniteInternalFuture<GridCacheCommittedTxInfo<K, V>>>() {
+            @Override public void apply(IgniteInternalFuture<GridCacheCommittedTxInfo<K, V>> infoFut) {
                 GridCacheCommittedTxInfo<K, V> info = null;
 
                 try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index 90e09d1..502c058 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.transactions;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -304,7 +305,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> loadMissing(
+    @Override public IgniteInternalFuture<Boolean> loadMissing(
         final GridCacheContext<K, V> cacheCtx,
         final boolean readThrough,
         boolean async,
@@ -1332,7 +1333,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
      * @param filter Filter.
      * @return Loaded key-value pairs.
      */
-    private IgniteFuture<Map<K, V>> checkMissed(
+    private IgniteInternalFuture<Map<K, V>> checkMissed(
         final GridCacheContext<K, V> cacheCtx,
         final Map<K, V> map,
         final Map<K, GridCacheVersion> missedMap,
@@ -1517,7 +1518,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(
+    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
         final GridCacheContext<K, V> cacheCtx,
         Collection<? extends K> keys,
         @Nullable GridCacheEntryEx<K, V> cached,
@@ -1563,7 +1564,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
 
                 long accessTtl = expiryPlc != null ? CU.toTtl(expiryPlc.getExpiryForAccess()) : -1L;
 
-                IgniteFuture<Boolean> fut = cacheCtx.cache().txLockAsync(lockKeys,
+                IgniteInternalFuture<Boolean> fut = cacheCtx.cache().txLockAsync(lockKeys,
                     lockTimeout(),
                     this,
                     true,
@@ -1574,7 +1575,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     CU.<K, V>empty());
 
                 PLC2<Map<K, V>> plc2 = new PLC2<Map<K, V>>() {
-                    @Override public IgniteFuture<Map<K, V>> postLock() throws IgniteCheckedException {
+                    @Override public IgniteInternalFuture<Map<K, V>> postLock() throws IgniteCheckedException {
                         if (log.isDebugEnabled())
                             log.debug("Acquired transaction lock for read on keys: " + lockKeys);
 
@@ -1676,7 +1677,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
 
                 if (fut.isDone()) {
                     try {
-                        IgniteFuture<Map<K, V>> fut1 = plc2.apply(fut.get(), null);
+                        IgniteInternalFuture<Map<K, V>> fut1 = plc2.apply(fut.get(), null);
 
                         return fut1.isDone() ?
                             new GridFinishedFutureEx<>(finClos.apply(fut1.get(), null)) :
@@ -1722,7 +1723,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                         checkMissed(cacheCtx, retMap, missed, redos, deserializePortable, filter),
                         // Closure that returns another future, based on result from first.
                         new PMC<Map<K, V>>() {
-                            @Override public IgniteFuture<Map<K, V>> postMiss(Map<K, V> map) {
+                            @Override public IgniteInternalFuture<Map<K, V>> postMiss(Map<K, V> map) {
                                 if (redos.isEmpty())
                                     return new GridFinishedFuture<>(cctx.kernalContext(),
                                         Collections.<K, V>emptyMap());
@@ -1769,7 +1770,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> putAllAsync(
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> putAllAsync(
         GridCacheContext<K, V> cacheCtx,
         Map<? extends K, ? extends V> map,
         boolean retval,
@@ -1777,7 +1778,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
         long ttl,
         IgnitePredicate<CacheEntry<K, V>>[] filter
     ) {
-        return (IgniteFuture<GridCacheReturn<V>>)putAllAsync0(cacheCtx,
+        return (IgniteInternalFuture<GridCacheReturn<V>>)putAllAsync0(cacheCtx,
             map,
             null,
             null,
@@ -1788,7 +1789,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllDrAsync(
+    @Override public IgniteInternalFuture<?> putAllDrAsync(
         GridCacheContext<K, V> cacheCtx,
         Map<? extends K, GridCacheDrInfo<V>> drMap
     ) {
@@ -1804,12 +1805,12 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <T> IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> invokeAsync(
+    @Override public <T> IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> invokeAsync(
         GridCacheContext<K, V> cacheCtx,
         @Nullable Map<? extends K, ? extends EntryProcessor<K, V, Object>> map,
         Object... invokeArgs
     ) {
-        return (IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>)putAllAsync0(cacheCtx,
+        return (IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>)putAllAsync0(cacheCtx,
             null,
             map,
             invokeArgs,
@@ -1820,7 +1821,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllDrAsync(
+    @Override public IgniteInternalFuture<?> removeAllDrAsync(
         GridCacheContext<K, V> cacheCtx,
         Map<? extends K, GridCacheVersion> drMap
     ) {
@@ -1860,7 +1861,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
      * @param drRmvMap DR remove map (optional).
      * @return Future with skipped keys (the ones that didn't pass filter for pessimistic transactions).
      */
-    protected IgniteFuture<Set<K>> enlistWrite(
+    protected IgniteInternalFuture<Set<K>> enlistWrite(
         final GridCacheContext<K, V> cacheCtx,
         Collection<? extends K> keys,
         @Nullable GridCacheEntryEx<K, V> cached,
@@ -2070,7 +2071,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                                         // one key in the keys collection.
                                         assert keys.size() == 1;
 
-                                        IgniteFuture<Boolean> fut = loadMissing(
+                                        IgniteInternalFuture<Boolean> fut = loadMissing(
                                             cacheCtx,
                                             op == TRANSFORM || cacheCtx.loadPreviousValue(),
                                             true,
@@ -2210,7 +2211,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
             assert optimistic();
             assert invokeMap != null;
 
-            IgniteFuture<Boolean> fut = loadMissing(
+            IgniteInternalFuture<Boolean> fut = loadMissing(
                 cacheCtx,
                 true,
                 true,
@@ -2439,7 +2440,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
      * @return Operation future.
      */
     @SuppressWarnings("unchecked")
-    private IgniteFuture putAllAsync0(
+    private IgniteInternalFuture putAllAsync0(
         final GridCacheContext<K, V> cacheCtx,
         @Nullable Map<? extends K, ? extends V> map,
         @Nullable Map<? extends K, ? extends EntryProcessor<K, V, Object>> invokeMap,
@@ -2545,7 +2546,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
 
             GridCacheProjectionImpl<K, V> prj = cacheCtx.projectionPerCall();
 
-            final IgniteFuture<Set<K>> loadFut = enlistWrite(
+            final IgniteInternalFuture<Set<K>> loadFut = enlistWrite(
                 cacheCtx,
                 keySet,
                 cached,
@@ -2582,7 +2583,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                 if (log.isDebugEnabled())
                     log.debug("Before acquiring transaction lock for put on keys: " + keys);
 
-                IgniteFuture<Boolean> fut = cacheCtx.cache().txLockAsync(keys,
+                IgniteInternalFuture<Boolean> fut = cacheCtx.cache().txLockAsync(keys,
                     lockTimeout(),
                     this,
                     false,
@@ -2635,8 +2636,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                         cctx.kernalContext());
             }
             else {
-                return loadFut.chain(new CX1<IgniteFuture<Set<K>>, GridCacheReturn<V>>() {
-                    @Override public GridCacheReturn<V> applyx(IgniteFuture<Set<K>> f) throws IgniteCheckedException {
+                return loadFut.chain(new CX1<IgniteInternalFuture<Set<K>>, GridCacheReturn<V>>() {
+                    @Override public GridCacheReturn<V> applyx(IgniteInternalFuture<Set<K>> f) throws IgniteCheckedException {
                         f.get();
 
                         return ret;
@@ -2652,7 +2653,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheReturn<V>> removeAllAsync(
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> removeAllAsync(
         GridCacheContext<K, V> cacheCtx,
         Collection<? extends K> keys,
         @Nullable GridCacheEntryEx<K, V> cached,
@@ -2671,7 +2672,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
      * @param filter Filter.
      * @return Future for asynchronous remove.
      */
-    private IgniteFuture<GridCacheReturn<V>> removeAllAsync0(
+    private IgniteInternalFuture<GridCacheReturn<V>> removeAllAsync0(
         final GridCacheContext<K, V> cacheCtx,
         @Nullable final Collection<? extends K> keys,
         @Nullable Map<? extends  K, GridCacheVersion> drMap,
@@ -2751,7 +2752,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
             else
                 plc = null;
 
-            final IgniteFuture<Set<K>> loadFut = enlistWrite(
+            final IgniteInternalFuture<Set<K>> loadFut = enlistWrite(
                 cacheCtx,
                 keys0,
                 /** cached entry */null,
@@ -2782,7 +2783,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                 if (log.isDebugEnabled())
                     log.debug("Before acquiring transaction lock for remove on keys: " + passedKeys);
 
-                IgniteFuture<Boolean> fut = cacheCtx.cache().txLockAsync(passedKeys,
+                IgniteInternalFuture<Boolean> fut = cacheCtx.cache().txLockAsync(passedKeys,
                     lockTimeout(),
                     this,
                     false,
@@ -2835,8 +2836,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                         cctx.kernalContext());
             }
             else {
-                return loadFut.chain(new CX1<IgniteFuture<Set<K>>, GridCacheReturn<V>>() {
-                    @Override public GridCacheReturn<V> applyx(IgniteFuture<Set<K>> f) throws IgniteCheckedException {
+                return loadFut.chain(new CX1<IgniteInternalFuture<Set<K>>, GridCacheReturn<V>>() {
+                    @Override public GridCacheReturn<V> applyx(IgniteInternalFuture<Set<K>> f) throws IgniteCheckedException {
                         f.get();
 
                         return ret;
@@ -2910,7 +2911,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
      * Performs keys locking for affinity-based group lock transactions.
      * @return Lock future.
      */
-    @Override public IgniteFuture<?> groupLockAsync(GridCacheContext<K, V> cacheCtx, Collection<K> keys) {
+    @Override public IgniteInternalFuture<?> groupLockAsync(GridCacheContext<K, V> cacheCtx, Collection<K> keys) {
         assert groupLock();
 
         try {
@@ -3349,7 +3350,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
      *
      * @param <T> Return type.
      */
-    protected abstract class PostLockClosure1<T> implements IgniteBiClosure<Boolean, Exception, IgniteFuture<T>> {
+    protected abstract class PostLockClosure1<T> implements IgniteBiClosure<Boolean, Exception, IgniteInternalFuture<T>> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -3380,13 +3381,13 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
         }
 
         /** {@inheritDoc} */
-        @Override public final IgniteFuture<T> apply(Boolean locked, @Nullable final Exception e) {
+        @Override public final IgniteInternalFuture<T> apply(Boolean locked, @Nullable final Exception e) {
             if (e != null) {
                 setRollbackOnly();
 
                 if (commit && commitAfterLock())
-                    return rollbackAsync().chain(new C1<IgniteFuture<IgniteTx>, T>() {
-                        @Override public T apply(IgniteFuture<IgniteTx> f) {
+                    return rollbackAsync().chain(new C1<IgniteInternalFuture<IgniteTx>, T>() {
+                        @Override public T apply(IgniteInternalFuture<IgniteTx> f) {
                             throw new GridClosureException(e);
                         }
                     });
@@ -3402,8 +3403,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     ", tx=" + this + ']'));
 
                 if (commit && commitAfterLock())
-                    return rollbackAsync().chain(new C1<IgniteFuture<IgniteTx>, T>() {
-                        @Override public T apply(IgniteFuture<IgniteTx> f) {
+                    return rollbackAsync().chain(new C1<IgniteInternalFuture<IgniteTx>, T>() {
+                        @Override public T apply(IgniteInternalFuture<IgniteTx> f) {
                             throw ex;
                         }
                     });
@@ -3420,8 +3421,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                 if (commit && commitAfterLock()) {
                     rollback = false;
 
-                    return commitAsync().chain(new CX1<IgniteFuture<IgniteTx>, T>() {
-                        @Override public T applyx(IgniteFuture<IgniteTx> f) throws IgniteCheckedException {
+                    return commitAsync().chain(new CX1<IgniteInternalFuture<IgniteTx>, T>() {
+                        @Override public T applyx(IgniteInternalFuture<IgniteTx> f) throws IgniteCheckedException {
                             f.get();
 
                             return r;
@@ -3435,8 +3436,8 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
             }
             catch (final IgniteCheckedException ex) {
                 if (commit && commitAfterLock())
-                    return rollbackAsync().chain(new C1<IgniteFuture<IgniteTx>, T>() {
-                        @Override public T apply(IgniteFuture<IgniteTx> f) {
+                    return rollbackAsync().chain(new C1<IgniteInternalFuture<IgniteTx>, T>() {
+                        @Override public T apply(IgniteInternalFuture<IgniteTx> f) {
                             throw new GridClosureException(ex);
                         }
                     });
@@ -3464,12 +3465,12 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
      *
      * @param <T> Return type.
      */
-    protected abstract class PostLockClosure2<T> implements IgniteBiClosure<Boolean, Exception, IgniteFuture<T>> {
+    protected abstract class PostLockClosure2<T> implements IgniteBiClosure<Boolean, Exception, IgniteInternalFuture<T>> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
-        @Override public final IgniteFuture<T> apply(Boolean locked, @Nullable Exception e) {
+        @Override public final IgniteInternalFuture<T> apply(Boolean locked, @Nullable Exception e) {
             boolean rollback = true;
 
             try {
@@ -3480,7 +3481,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     throw new GridClosureException(new IgniteTxTimeoutException("Failed to acquire lock " +
                         "within provided timeout for transaction [timeout=" + timeout() + ", tx=" + this + ']'));
 
-                IgniteFuture<T> fut = postLock();
+                IgniteInternalFuture<T> fut = postLock();
 
                 rollback = false;
 
@@ -3501,7 +3502,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
          * @return Future return value.
          * @throws IgniteCheckedException If operation failed.
          */
-        protected abstract IgniteFuture<T> postLock() throws IgniteCheckedException;
+        protected abstract IgniteInternalFuture<T> postLock() throws IgniteCheckedException;
     }
 
     /**
@@ -3509,19 +3510,19 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
      *
      * @param <T> Return type.
      */
-    protected abstract class PostMissClosure<T> implements IgniteBiClosure<T, Exception, IgniteFuture<T>> {
+    protected abstract class PostMissClosure<T> implements IgniteBiClosure<T, Exception, IgniteInternalFuture<T>> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
-        @Override public final IgniteFuture<T> apply(T t, Exception e) {
+        @Override public final IgniteInternalFuture<T> apply(T t, Exception e) {
             boolean rollback = true;
 
             try {
                 if (e != null)
                     throw new GridClosureException(e);
 
-                IgniteFuture<T> fut = postMiss(t);
+                IgniteInternalFuture<T> fut = postMiss(t);
 
                 rollback = false;
 
@@ -3543,7 +3544,7 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
          * @return Future return value.
          * @throws IgniteCheckedException If operation failed.
          */
-        protected abstract IgniteFuture<T> postMiss(T t) throws IgniteCheckedException;
+        protected abstract IgniteInternalFuture<T> postMiss(T t) throws IgniteCheckedException;
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
index 8568318..f7ace8b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalEx.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.transactions;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.lang.*;
@@ -71,7 +72,7 @@ public interface IgniteTxLocalEx<K, V> extends IgniteTxEx<K, V> {
      * @param filter Entry filter.
      * @return Future for this get.
      */
-    public IgniteFuture<Map<K, V>> getAllAsync(
+    public IgniteInternalFuture<Map<K, V>> getAllAsync(
         GridCacheContext<K, V> cacheCtx,
         Collection<? extends K> keys,
         @Nullable GridCacheEntryEx<K, V> cached,
@@ -87,7 +88,7 @@ public interface IgniteTxLocalEx<K, V> extends IgniteTxEx<K, V> {
      * @param ttl Time to live for entry. If negative, leave unchanged.
      * @return Future for put operation.
      */
-    public IgniteFuture<GridCacheReturn<V>> putAllAsync(
+    public IgniteInternalFuture<GridCacheReturn<V>> putAllAsync(
         GridCacheContext<K, V> cacheCtx,
         Map<? extends K, ? extends V> map,
         boolean retval,
@@ -101,7 +102,7 @@ public interface IgniteTxLocalEx<K, V> extends IgniteTxEx<K, V> {
      * @param invokeArgs Optional arguments for entry processor.
      * @return Transform operation future.
      */
-    public <T> IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> invokeAsync(
+    public <T> IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> invokeAsync(
         GridCacheContext<K, V> cacheCtx,
         Map<? extends K, ? extends EntryProcessor<K, V, Object>> map,
         Object... invokeArgs);
@@ -114,7 +115,7 @@ public interface IgniteTxLocalEx<K, V> extends IgniteTxEx<K, V> {
      * @param filter Filter.
      * @return Future for asynchronous remove.
      */
-    public IgniteFuture<GridCacheReturn<V>> removeAllAsync(
+    public IgniteInternalFuture<GridCacheReturn<V>> removeAllAsync(
         GridCacheContext<K, V> cacheCtx,
         Collection<? extends K> keys,
         @Nullable GridCacheEntryEx<K, V> cached,
@@ -126,7 +127,7 @@ public interface IgniteTxLocalEx<K, V> extends IgniteTxEx<K, V> {
      * @param drMap DR map to put.
      * @return Future for DR put operation.
      */
-    public IgniteFuture<?> putAllDrAsync(
+    public IgniteInternalFuture<?> putAllDrAsync(
         GridCacheContext<K, V> cacheCtx,
         Map<? extends K, GridCacheDrInfo<V>> drMap);
 
@@ -135,7 +136,7 @@ public interface IgniteTxLocalEx<K, V> extends IgniteTxEx<K, V> {
      * @param drMap DR map.
      * @return Future for asynchronous remove.
      */
-    public IgniteFuture<?> removeAllDrAsync(
+    public IgniteInternalFuture<?> removeAllDrAsync(
         GridCacheContext<K, V> cacheCtx,
         Map<? extends K, GridCacheVersion> drMap);
 
@@ -146,7 +147,7 @@ public interface IgniteTxLocalEx<K, V> extends IgniteTxEx<K, V> {
      * @param keys Keys to lock.
      * @return Lock future.
      */
-    public IgniteFuture<?> groupLockAsync(GridCacheContext<K, V> cacheCtx, Collection<K> keys);
+    public IgniteInternalFuture<?> groupLockAsync(GridCacheContext<K, V> cacheCtx, Collection<K> keys);
 
     /**
      * @return {@code True} if keys from the same partition are allowed to be enlisted in group-lock transaction.
@@ -171,7 +172,7 @@ public interface IgniteTxLocalEx<K, V> extends IgniteTxEx<K, V> {
      * @param deserializePortable Deserialize portable flag.
      * @return Future with {@code True} value if loading took place.
      */
-    public IgniteFuture<Boolean> loadMissing(
+    public IgniteInternalFuture<Boolean> loadMissing(
         GridCacheContext<K, V> cacheCtx,
         boolean readThrough,
         boolean async,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
index c73a291..9cbd479 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxManager.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.transactions;
 
 import org.apache.ignite.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -514,7 +515,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param topVer Topology version.
      * @return Future that will be completed when all ongoing transactions are finished.
      */
-    public IgniteFuture<Boolean> finishTxs(long topVer) {
+    public IgniteInternalFuture<Boolean> finishTxs(long topVer) {
         GridCompoundFuture<IgniteTx, Boolean> res =
             new GridCompoundFuture<>(context().kernalContext(),
                 new IgniteReducer<IgniteTx, Boolean>() {
@@ -1488,7 +1489,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param threadId Near tx thread ID.
      * @return {@code null} if ack was received or future that will be completed when ack is received.
      */
-    @Nullable public IgniteFuture<?> awaitFinishAckAsync(UUID rmtNodeId, long threadId) {
+    @Nullable public IgniteInternalFuture<?> awaitFinishAckAsync(UUID rmtNodeId, long threadId) {
         if (finishSyncDisabled)
             return null;
 
@@ -1978,7 +1979,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
      * @param req Check committed request.
      * @return Check committed future.
      */
-    public IgniteFuture<GridCacheCommittedTxInfo<K, V>> checkPessimisticTxCommitted(GridCachePessimisticCheckCommittedTxRequest req) {
+    public IgniteInternalFuture<GridCacheCommittedTxInfo<K, V>> checkPessimisticTxCommitted(GridCachePessimisticCheckCommittedTxRequest req) {
         // First check if we have near transaction with this ID.
         IgniteTxEx<K, V> tx = localTxForRecovery(req.nearXidVersion(), !req.nearOnlyCheck());
 
@@ -1992,8 +1993,8 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
 
             final IgniteTxEx<K, V> tx0 = tx;
 
-            return tx.finishFuture().chain(new C1<IgniteFuture<IgniteTx>, GridCacheCommittedTxInfo<K, V>>() {
-                @Override public GridCacheCommittedTxInfo<K, V> apply(IgniteFuture<IgniteTx> txFut) {
+            return tx.finishFuture().chain(new C1<IgniteInternalFuture<IgniteTx>, GridCacheCommittedTxInfo<K, V>>() {
+                @Override public GridCacheCommittedTxInfo<K, V> apply(IgniteInternalFuture<IgniteTx> txFut) {
                     GridCacheCommittedTxInfo<K, V> info = null;
 
                     if (tx0.state() == COMMITTED)
@@ -2189,7 +2190,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
     /**
      * Commit listener. Checks if commit succeeded and rollbacks if case of error.
      */
-    private class CommitListener implements CI1<IgniteFuture<IgniteTx>> {
+    private class CommitListener implements CI1<IgniteInternalFuture<IgniteTx>> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -2204,7 +2205,7 @@ public class IgniteTxManager<K, V> extends GridCacheSharedManagerAdapter<K, V> {
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(IgniteFuture<IgniteTx> t) {
+        @Override public void apply(IgniteInternalFuture<IgniteTx> t) {
             try {
                 t.get();
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java
index 3bfebc8..14311d3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxProxyImpl.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.transactions;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
@@ -47,7 +48,7 @@ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
     private boolean async;
 
     /** Async call result. */
-    private IgniteFuture asyncRes;
+    private IgniteInternalFuture asyncRes;
 
     /**
      * Empty constructor required for {@link Externalizable}.
@@ -196,7 +197,7 @@ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <R> IgniteFuture<R> future() {
+    @Override public <R> IgniteInternalFuture<R> future() {
         return asyncRes;
     }
 
@@ -232,7 +233,7 @@ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
         enter();
 
         try {
-            IgniteFuture<IgniteTx> commitFut = cctx.commitTxAsync(tx);
+            IgniteInternalFuture<IgniteTx> commitFut = cctx.commitTxAsync(tx);
 
             if (async)
                 asyncRes = commitFut;
@@ -261,7 +262,7 @@ public class IgniteTxProxyImpl<K, V> implements IgniteTxProxy, Externalizable {
         enter();
 
         try {
-            IgniteFuture rollbackFut = cctx.rollbackTxAsync(tx);
+            IgniteInternalFuture rollbackFut = cctx.rollbackTxAsync(tx);
 
             if (async)
                 asyncRes = rollbackFut;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
index a6c26e9..aefff2d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/closure/GridClosureProcessor.java
@@ -124,7 +124,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param nodes Grid nodes.
      * @return Task execution future.
      */
-    public IgniteFuture<?> runAsync(GridClosureCallMode mode, @Nullable Collection<? extends Runnable> jobs,
+    public IgniteInternalFuture<?> runAsync(GridClosureCallMode mode, @Nullable Collection<? extends Runnable> jobs,
         @Nullable Collection<ClusterNode> nodes) {
         return runAsync(mode, jobs, nodes, false);
     }
@@ -136,7 +136,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param sys If {@code true}, then system pool will be used.
      * @return Task execution future.
      */
-    public IgniteFuture<?> runAsync(GridClosureCallMode mode, @Nullable Collection<? extends Runnable> jobs,
+    public IgniteInternalFuture<?> runAsync(GridClosureCallMode mode, @Nullable Collection<? extends Runnable> jobs,
         @Nullable Collection<ClusterNode> nodes, boolean sys) {
         assert mode != null;
 
@@ -164,7 +164,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param nodes Grid nodes.
      * @return Task execution future.
      */
-    public IgniteFuture<?> runAsync(GridClosureCallMode mode, @Nullable Runnable job,
+    public IgniteInternalFuture<?> runAsync(GridClosureCallMode mode, @Nullable Runnable job,
         @Nullable Collection<ClusterNode> nodes) {
         return runAsync(mode, job, nodes, false);
     }
@@ -176,7 +176,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param sys If {@code true}, then system pool will be used.
      * @return Task execution future.
      */
-    public IgniteFuture<?> runAsync(GridClosureCallMode mode, @Nullable Runnable job,
+    public IgniteInternalFuture<?> runAsync(GridClosureCallMode mode, @Nullable Runnable job,
         @Nullable Collection<ClusterNode> nodes, boolean sys) {
         assert mode != null;
 
@@ -304,7 +304,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param <R2> Type.
      * @return Reduced result.
      */
-    public <R1, R2> IgniteFuture<R2> forkjoinAsync(GridClosureCallMode mode,
+    public <R1, R2> IgniteInternalFuture<R2> forkjoinAsync(GridClosureCallMode mode,
         @Nullable Collection<? extends Callable<R1>> jobs,
         @Nullable IgniteReducer<R1, R2> rdc, @Nullable Collection<ClusterNode> nodes) {
         assert mode != null;
@@ -334,7 +334,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param <R> Type.
      * @return Grid future for collection of closure results.
      */
-    public <R> IgniteFuture<Collection<R>> callAsync(
+    public <R> IgniteInternalFuture<Collection<R>> callAsync(
         GridClosureCallMode mode,
         @Nullable Collection<? extends Callable<R>> jobs,
         @Nullable Collection<ClusterNode> nodes) {
@@ -349,7 +349,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param <R> Type.
      * @return Grid future for collection of closure results.
      */
-    public <R> IgniteFuture<Collection<R>> callAsync(GridClosureCallMode mode,
+    public <R> IgniteInternalFuture<Collection<R>> callAsync(GridClosureCallMode mode,
         @Nullable Collection<? extends Callable<R>> jobs, @Nullable Collection<ClusterNode> nodes,
         boolean sys) {
         assert mode != null;
@@ -380,7 +380,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param <R> Type.
      * @return Grid future for collection of closure results.
      */
-    public <R> IgniteFuture<R> callAsync(GridClosureCallMode mode,
+    public <R> IgniteInternalFuture<R> callAsync(GridClosureCallMode mode,
         @Nullable Callable<R> job, @Nullable Collection<ClusterNode> nodes) {
         return callAsync(mode, job, nodes, false);
     }
@@ -392,7 +392,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param nodes Grid nodes.
      * @return Job future.
      */
-    public <R> IgniteFuture<R> affinityCall(@Nullable String cacheName, Object affKey, Callable<R> job,
+    public <R> IgniteInternalFuture<R> affinityCall(@Nullable String cacheName, Object affKey, Callable<R> job,
         @Nullable Collection<ClusterNode> nodes) {
         enterBusy();
 
@@ -422,7 +422,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param nodes Grid nodes.
      * @return Job future.
      */
-    public IgniteFuture<?> affinityRun(@Nullable String cacheName, Object affKey, Runnable job,
+    public IgniteInternalFuture<?> affinityRun(@Nullable String cacheName, Object affKey, Runnable job,
         @Nullable Collection<ClusterNode> nodes) {
         enterBusy();
 
@@ -453,7 +453,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param <R> Type.
      * @return Grid future for collection of closure results.
      */
-    public <R> IgniteFuture<R> callAsyncNoFailover(GridClosureCallMode mode, @Nullable Callable<R> job,
+    public <R> IgniteInternalFuture<R> callAsyncNoFailover(GridClosureCallMode mode, @Nullable Callable<R> job,
         @Nullable Collection<ClusterNode> nodes, boolean sys) {
         assert mode != null;
 
@@ -484,7 +484,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param <R> Type.
      * @return Grid future for collection of closure results.
      */
-    public <R> IgniteFuture<Collection<R>> callAsyncNoFailover(GridClosureCallMode mode,
+    public <R> IgniteInternalFuture<Collection<R>> callAsyncNoFailover(GridClosureCallMode mode,
         @Nullable Collection<? extends Callable<R>> jobs, @Nullable Collection<ClusterNode> nodes,
         boolean sys) {
         assert mode != null;
@@ -516,7 +516,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param <R> Type.
      * @return Grid future for collection of closure results.
      */
-    public <R> IgniteFuture<R> callAsync(GridClosureCallMode mode,
+    public <R> IgniteInternalFuture<R> callAsync(GridClosureCallMode mode,
         @Nullable Callable<R> job, @Nullable Collection<ClusterNode> nodes, boolean sys) {
         assert mode != null;
 
@@ -544,7 +544,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param nodes Grid nodes.
      * @return Grid future for execution result.
      */
-    public <T, R> IgniteFuture<R> callAsync(IgniteClosure<T, R> job, @Nullable T arg,
+    public <T, R> IgniteInternalFuture<R> callAsync(IgniteClosure<T, R> job, @Nullable T arg,
         @Nullable Collection<ClusterNode> nodes) {
         enterBusy();
 
@@ -567,7 +567,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param nodes Grid nodes.
      * @return Grid future for execution result.
      */
-    public <T, R> IgniteFuture<Collection<R>> broadcast(IgniteClosure<T, R> job, @Nullable T arg,
+    public <T, R> IgniteInternalFuture<Collection<R>> broadcast(IgniteClosure<T, R> job, @Nullable T arg,
         @Nullable Collection<ClusterNode> nodes) {
         enterBusy();
 
@@ -590,7 +590,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param nodes Grid nodes.
      * @return Grid future for execution result.
      */
-    public <T, R> IgniteFuture<Collection<R>> broadcastNoFailover(IgniteClosure<T, R> job, @Nullable T arg,
+    public <T, R> IgniteInternalFuture<Collection<R>> broadcastNoFailover(IgniteClosure<T, R> job, @Nullable T arg,
         @Nullable Collection<ClusterNode> nodes) {
         enterBusy();
 
@@ -614,7 +614,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param nodes Grid nodes.
      * @return Grid future for execution result.
      */
-    public <T, R> IgniteFuture<Collection<R>> callAsync(IgniteClosure<T, R> job, @Nullable Collection<? extends T> args,
+    public <T, R> IgniteInternalFuture<Collection<R>> callAsync(IgniteClosure<T, R> job, @Nullable Collection<? extends T> args,
         @Nullable Collection<ClusterNode> nodes) {
         enterBusy();
 
@@ -638,7 +638,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param nodes Grid nodes.
      * @return Grid future for execution result.
      */
-    public <T, R1, R2> IgniteFuture<R2> callAsync(IgniteClosure<T, R1> job,
+    public <T, R1, R2> IgniteInternalFuture<R2> callAsync(IgniteClosure<T, R1> job,
         Collection<? extends T> args, IgniteReducer<R1, R2> rdc, @Nullable Collection<ClusterNode> nodes) {
         enterBusy();
 
@@ -705,7 +705,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @return Future.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    private IgniteFuture<?> runLocal(@Nullable final Runnable c, boolean sys) throws IgniteCheckedException {
+    private IgniteInternalFuture<?> runLocal(@Nullable final Runnable c, boolean sys) throws IgniteCheckedException {
         return runLocal(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL);
     }
 
@@ -715,7 +715,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @return Future.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    private IgniteFuture<?> runLocal(@Nullable final Runnable c, GridClosurePolicy plc) throws IgniteCheckedException {
+    private IgniteInternalFuture<?> runLocal(@Nullable final Runnable c, GridClosurePolicy plc) throws IgniteCheckedException {
         if (c == null)
             return new GridFinishedFuture(ctx);
 
@@ -780,7 +780,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param c Closure to execute.
      * @return Future.
      */
-    public IgniteFuture<?> runLocalSafe(Runnable c) {
+    public IgniteInternalFuture<?> runLocalSafe(Runnable c) {
         return runLocalSafe(c, true);
     }
 
@@ -792,7 +792,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param sys If {@code true}, then system pool will be used, otherwise public pool will be used.
      * @return Future.
      */
-    public IgniteFuture<?> runLocalSafe(Runnable c, boolean sys) {
+    public IgniteInternalFuture<?> runLocalSafe(Runnable c, boolean sys) {
         return runLocalSafe(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL);
     }
 
@@ -804,7 +804,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param plc Policy to choose executor pool.
      * @return Future.
      */
-    public IgniteFuture<?> runLocalSafe(Runnable c, GridClosurePolicy plc) {
+    public IgniteInternalFuture<?> runLocalSafe(Runnable c, GridClosurePolicy plc) {
         try {
             return runLocal(c, plc);
         }
@@ -841,7 +841,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @return Future.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    private <R> IgniteFuture<R> callLocal(@Nullable final Callable<R> c, boolean sys) throws IgniteCheckedException {
+    private <R> IgniteInternalFuture<R> callLocal(@Nullable final Callable<R> c, boolean sys) throws IgniteCheckedException {
         return callLocal(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL);
     }
 
@@ -852,7 +852,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @return Future.
      * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    private <R> IgniteFuture<R> callLocal(@Nullable final Callable<R> c, GridClosurePolicy plc) throws IgniteCheckedException {
+    private <R> IgniteInternalFuture<R> callLocal(@Nullable final Callable<R> c, GridClosurePolicy plc) throws IgniteCheckedException {
         if (c == null)
             return new GridFinishedFuture<>(ctx);
 
@@ -915,7 +915,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param c Closure to execute.
      * @return Future.
      */
-    public <R> IgniteFuture<R> callLocalSafe(Callable<R> c) {
+    public <R> IgniteInternalFuture<R> callLocalSafe(Callable<R> c) {
         return callLocalSafe(c, true);
     }
 
@@ -927,7 +927,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param sys If {@code true}, then system pool will be used, otherwise public pool will be used.
      * @return Future.
      */
-    public <R> IgniteFuture<R> callLocalSafe(Callable<R> c, boolean sys) {
+    public <R> IgniteInternalFuture<R> callLocalSafe(Callable<R> c, boolean sys) {
         return callLocalSafe(c, sys ? GridClosurePolicy.SYSTEM_POOL : GridClosurePolicy.PUBLIC_POOL);
     }
 
@@ -939,7 +939,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param plc Policy to choose executor pool.
      * @return Future.
      */
-    public <R> IgniteFuture<R> callLocalSafe(Callable<R> c, GridClosurePolicy plc) {
+    public <R> IgniteInternalFuture<R> callLocalSafe(Callable<R> c, GridClosurePolicy plc) {
         try {
             return callLocal(c, plc);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
index c349b55..4e84eac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/continuous/GridContinuousProcessor.java
@@ -373,7 +373,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
      * @return Future.
      */
     @SuppressWarnings("TooBroadScope")
-    public IgniteFuture<UUID> startRoutine(GridContinuousHandler hnd,
+    public IgniteInternalFuture<UUID> startRoutine(GridContinuousHandler hnd,
         int bufSize,
         long interval,
         boolean autoUnsubscribe,
@@ -562,7 +562,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
      * @param routineId Consume ID.
      * @return Future.
      */
-    public IgniteFuture<?> stopRoutine(UUID routineId) {
+    public IgniteInternalFuture<?> stopRoutine(UUID routineId) {
         assert routineId != null;
 
         boolean doStop = false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessor.java
index 194d30d..9d3bf70 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessor.java
@@ -22,7 +22,6 @@ import org.apache.ignite.dataload.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.thread.*;
 import org.apache.ignite.internal.managers.communication.*;
@@ -153,8 +152,8 @@ public class GridDataLoaderProcessor<K, V> extends GridProcessorAdapter {
 
             ldrs.add(ldr);
 
-            ldr.future().listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> f) {
+            ldr.future().listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> f) {
                     boolean b = ldrs.remove(ldr);
 
                     assert b : "Loader has not been added to set: " + ldr;


[20/50] [abbrv] incubator-ignite git commit: # sprint-1 Updated hadoop XML files.

Posted by vo...@apache.org.
# sprint-1 Updated hadoop XML files.


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

Branch: refs/heads/ignite-16
Commit: 87d1de1349d45cde66f6c0d138997e9700902bf4
Parents: d19c73e
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Jan 29 06:39:54 2015 +0000
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Jan 29 06:39:54 2015 +0000

----------------------------------------------------------------------
 docs/core-site.ignite.xml   | 30 +++++++++++++++---------------
 docs/hive-site.ignite.xml   |  6 +++---
 docs/mapred-site.ignite.xml | 14 +++++++-------
 3 files changed, 25 insertions(+), 25 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/87d1de13/docs/core-site.ignite.xml
----------------------------------------------------------------------
diff --git a/docs/core-site.ignite.xml b/docs/core-site.ignite.xml
index 0054b60..0bb052c 100644
--- a/docs/core-site.ignite.xml
+++ b/docs/core-site.ignite.xml
@@ -19,47 +19,47 @@
 
 <!--
     This template file contains settings needed to run Apache Hadoop jobs
-    with GridGain's distributed in-memory file system GGFS.
+    with Apache Ignite's distributed in-memory file system IgniteFS.
 
     You can replace '$HADOOP_HOME/etc/hadoop/core-site.xml' file with this one
-    to work with GridGain GGFS nodes running on localhost (these local nodes can be
+    to work with IgniteFS nodes running on localhost (these local nodes can be
     a part of distributed cluster though). To work with file system on remote
     hosts you need to change the host of file system URI to any host running
-    GridGain's GGFS node.
+    IgniteFS node.
 
-    Note that GridGain jars must be in Apache Hadoop client classpath to work
+    Note that Ignite jars must be in Apache Hadoop client classpath to work
     with this configuration.
 
-    Run script '$GRIDGAIN_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop client setup.
+    Run script '$IGNITE_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop client setup.
 -->
 
 <configuration>
     <!--
-        Set default file system to GGFS instance named "ggfs" configured in GridGain.
+        Set default file system to IgniteFS instance named "ignitefs" configured in Ignite.
     -->
     <property>
         <name>fs.default.name</name>
-        <value>ggfs://ggfs@localhost</value>
+        <value>ignitefs://ignitefs@localhost</value>
     </property>
 
     <!--
-        Set Hadoop 1.* file system implementation class for GGFS.
+        Set Hadoop 1.* file system implementation class for IgniteFS.
     -->
     <property>
-        <name>fs.ggfs.impl</name>
+        <name>fs.ignitefs.impl</name>
         <value>org.apache.ignite.fs.hadoop.v1.GridGgfsHadoopFileSystem</value>
     </property>
 
     <!--
-        Set Hadoop 2.* file system implementation class for GGFS.
+        Set Hadoop 2.* file system implementation class for IgniteFS.
     -->
     <property>
-        <name>fs.AbstractFileSystem.ggfs.impl</name>
+        <name>fs.AbstractFileSystem.ignitefs.impl</name>
         <value>org.apache.ignite.fs.hadoop.v2.GridGgfsHadoopFileSystem</value>
     </property>
 
     <!--
-        Disallow data node replacement since it does not make sense for GridGain's GGFS nodes.
+        Disallow data node replacement since it does not make sense for IgniteFS nodes.
     -->
     <property>
         <name>dfs.client.block.write.replace-datanode-on-failure.policy</name>
@@ -67,11 +67,11 @@
     </property>
 
     <!--
-        Allow to write the job statistics into GGFS.
+        Allow to write the job statistics into IgniteFS.
     -->
     <!--
     <property>
-        <name>gridgain.counters.writer</name>
+        <name>ignite.counters.writer</name>
         <value>org.apache.ignite.internal.processors.hadoop.counter.GridHadoopFSCounterWriter</value>
     </property>
     -->
@@ -82,7 +82,7 @@
     -->
     <!--
     <property>
-        <name>gridgain.counters.fswriter.directory</name>
+        <name>ignite.counters.fswriter.directory</name>
         <value>/user/${USER}</value>
     </property>
     -->

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/87d1de13/docs/hive-site.ignite.xml
----------------------------------------------------------------------
diff --git a/docs/hive-site.ignite.xml b/docs/hive-site.ignite.xml
index e93be35..fbf5450 100644
--- a/docs/hive-site.ignite.xml
+++ b/docs/hive-site.ignite.xml
@@ -19,15 +19,15 @@
 
 <!--
     This template file contains settings needed to run Apache Hive queries
-    with GridGain In-Memory Accelerator.
+    with Ignite In-Memory Accelerator.
 
     You can replace '$HIVE_HOME/conf/hive-site.xml' file with this one or
-    run script '$GRIDGAIN_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop
+    run script '$IGNITE_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop
     and Hive client setup.
 -->
 <configuration>
     <!--
-        GridGain requires query plan to be passed not using local resource.
+        Ignite requires query plan to be passed not using local resource.
     -->
     <property>
         <name>hive.rpc.query.plan</name>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/87d1de13/docs/mapred-site.ignite.xml
----------------------------------------------------------------------
diff --git a/docs/mapred-site.ignite.xml b/docs/mapred-site.ignite.xml
index c9b95c4..832a828 100644
--- a/docs/mapred-site.ignite.xml
+++ b/docs/mapred-site.ignite.xml
@@ -33,30 +33,30 @@
 
 <!--
     This template file contains settings needed to run Apache Hadoop jobs
-    with GridGain In-Memory Accelerator.
+    with Apache Ignite In-Memory Accelerator.
 
     You can replace '$HADOOP_HOME/etc/hadoop/mapred-site.xml' file with this one
     to run jobs on localhost (local node can be a part of distributed cluster though).
     To run jobs on remote host you have to change jobtracker address to the REST address
-    of any running GridGain node.
+    of any running Ignite node.
 
-    Note that GridGain jars must be in Apache Hadoop client classpath to work
+    Note that Ignite jars must be in Apache Hadoop client classpath to work
     with this configuration.
 
-    Run script '$GRIDGAIN_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop client setup.
+    Run script '$IGNITE_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop client setup.
 -->
 
 <configuration>
     <!-- 
-        Framework name must be set to 'gridgain'.
+        Framework name must be set to 'ignite'.
     -->
     <property>
         <name>mapreduce.framework.name</name>
-        <value>gridgain</value>
+        <value>ignite</value>
     </property>
 
     <!--
-        Job tracker address must be set to the REST address of any running GridGain node.
+        Job tracker address must be set to the REST address of any running Ignite node.
     -->
     <property>
         <name>mapreduce.jobtracker.address</name>


[17/50] [abbrv] incubator-ignite git commit: # sprint-1 Fixed test to avoid hang

Posted by vo...@apache.org.
# sprint-1 Fixed test to avoid hang


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

Branch: refs/heads/ignite-16
Commit: 22860972ba4e74edb5472e81d7e5cc498c0355f4
Parents: c6e4f45
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jan 29 09:27:32 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jan 29 09:27:32 2015 +0300

----------------------------------------------------------------------
 .../ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java   | 14 +++++++++++---
 1 file changed, 11 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/22860972/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java
index 1d24f02..fac8c9b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java
@@ -18,7 +18,9 @@
 package org.apache.ignite.internal.util.ipc.shmem;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jdk8.backport.*;
 
@@ -103,9 +105,15 @@ public class GridIpcSharedMemorySpaceSelfTest extends GridCommonAbstractTest {
                 @Override public Object call() throws Exception {
                     GridIpcSharedMemorySpace inSpace;
 
-                    while ((inSpace = spaceRef.get()) == null) {
-                        // No-op;
-                    }
+                    GridTestUtils.waitForCondition(new GridAbsPredicate() {
+                        @Override public boolean apply() {
+                            return spaceRef.get() != null;
+                        }
+                    }, 10_000);
+
+                    inSpace = spaceRef.get();
+
+                    assertNotNull(inSpace);
 
                     try (GridIpcSharedMemorySpace space = new GridIpcSharedMemorySpace(tok, 0, 0, 128, true,
                         inSpace.sharedMemoryId(), log)) {


[41/50] [abbrv] incubator-ignite git commit: GridEx -> IgniteEx GridKernal -> IgniteKernal

Posted by vo...@apache.org.
GridEx -> IgniteEx
GridKernal -> IgniteKernal


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

Branch: refs/heads/ignite-16
Commit: c323a8e7b5302d3fe0affbbbb957774b393e5097
Parents: d5bef13
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jan 29 12:41:40 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jan 29 12:41:41 2015 +0300

----------------------------------------------------------------------
 .../ClientAbstractMultiThreadedSelfTest.java    |    2 +-
 .../ClientAbstractMultiNodeSelfTest.java        |    4 +-
 .../rest/RestBinaryProtocolSelfTest.java        |    2 +-
 .../rest/TaskCommandHandlerSelfTest.java        |    2 +-
 .../ignite/fs/mapreduce/IgniteFsTask.java       |    2 +-
 .../ignite/internal/ClusterGroupAdapter.java    |    2 +-
 .../java/org/apache/ignite/internal/GridEx.java |  143 -
 .../org/apache/ignite/internal/GridGainEx.java  |   20 +-
 .../org/apache/ignite/internal/GridKernal.java  | 3322 ------------------
 .../ignite/internal/GridKernalContext.java      |    2 +-
 .../ignite/internal/GridKernalContextImpl.java  |    6 +-
 .../ignite/internal/GridUpdateNotifier.java     |    2 +-
 .../ignite/internal/IgniteClusterAsyncImpl.java |    4 +-
 .../org/apache/ignite/internal/IgniteEx.java    |  143 +
 .../apache/ignite/internal/IgniteKernal.java    | 3322 ++++++++++++++++++
 .../processors/affinity/GridAffinityUtils.java  |    2 +-
 .../processors/cache/GridCacheAdapter.java      |    6 +-
 .../processors/cache/GridCacheContext.java      |    4 +-
 .../GridCacheDataStructuresManager.java         |    4 +-
 .../cache/query/GridCacheQueryManager.java      |    2 +-
 .../jdbc/GridCacheQueryJdbcMetadataTask.java    |    2 +-
 .../query/jdbc/GridCacheQueryJdbcTask.java      |    2 +-
 .../dr/GridDrDataLoadCacheUpdater.java          |    2 +-
 .../internal/processors/fs/GridGgfsImpl.java    |    2 +-
 .../resource/GridResourceProcessor.java         |    2 +-
 .../handlers/cache/GridCacheCommandHandler.java |    6 +-
 .../handlers/task/GridTaskCommandHandler.java   |    2 +-
 .../processors/service/GridServiceProxy.java    |    2 +-
 .../apache/ignite/internal/visor/VisorJob.java  |    2 +-
 .../internal/visor/VisorMultiNodeTask.java      |    2 +-
 .../ignite/internal/visor/cache/VisorCache.java |    2 +-
 .../compute/VisorComputeMonitoringHolder.java   |   10 +-
 .../visor/node/VisorBasicConfiguration.java     |    2 +-
 .../visor/node/VisorGridConfiguration.java      |    2 +-
 .../visor/node/VisorNodeDataCollectorJob.java   |    2 +-
 .../internal/visor/query/VisorQueryTask.java    |    2 +-
 .../StreamerCacheAffinityEventRouter.java       |    2 +-
 .../ignite/GridExceptionHelpLinksSelfTest.java  |  106 -
 .../ignite/GridExternalizableAbstractTest.java  |   41 -
 .../IgniteExceptionHelpLinksSelfTest.java       |  106 +
 .../IgniteExternalizableAbstractTest.java       |   41 +
 ...CachePartitionFairAffinityNodesSelfTest.java |    6 +-
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |    2 +-
 .../ignite/fs/GridGgfsFragmentizerSelfTest.java |    2 +-
 .../internal/GridDiscoveryEventSelfTest.java    |    3 +-
 .../ignite/internal/GridDiscoverySelfTest.java  |    4 +-
 .../internal/GridExecutorServiceTest.java       |  315 --
 .../GridExplicitImplicitDeploymentSelfTest.java |  476 ---
 .../ignite/internal/GridKernalTestUtils.java    |    3 +-
 .../internal/GridListenActorSelfTest.java       |    3 +-
 .../GridTopicExternalizableSelfTest.java        |    3 +-
 .../GridTopologyBuildVersionSelfTest.java       |    2 +-
 .../internal/GridUpdateNotifierSelfTest.java    |    3 +-
 .../internal/IgniteExecutorServiceTest.java     |  315 ++
 ...gniteExplicitImplicitDeploymentSelfTest.java |  476 +++
 .../GridCheckpointManagerAbstractSelfTest.java  |    2 +-
 .../GridCommunicationSendMessageSelfTest.java   |    4 +-
 .../GridDiscoveryManagerAliveCacheSelfTest.java |    2 +-
 .../discovery/GridDiscoveryManagerSelfTest.java |    6 +-
 .../swapspace/GridSwapSpaceManagerSelfTest.java |    2 +-
 .../GridAffinityProcessorAbstractSelfTest.java  |    6 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |    4 +-
 .../cache/GridCacheAbstractSelfTest.java        |    2 +-
 .../cache/GridCacheAffinityApiSelfTest.java     |    4 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |    4 +-
 .../cache/GridCacheEntryVersionSelfTest.java    |    4 +-
 .../GridCacheFinishPartitionsSelfTest.java      |    4 +-
 .../GridCacheGroupLockAbstractSelfTest.java     |    4 +-
 .../processors/cache/GridCacheLeakTest.java     |    2 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |    2 +-
 .../cache/GridCacheMultiUpdateLockSelfTest.java |    2 +-
 .../cache/GridCacheMvccFlagsTest.java           |    4 +-
 .../cache/GridCacheMvccManagerSelfTest.java     |    4 +-
 .../cache/GridCacheMvccPartitionedSelfTest.java |    4 +-
 .../processors/cache/GridCacheMvccSelfTest.java |    4 +-
 .../cache/GridCacheNestedTxAbstractTest.java    |    2 +-
 .../cache/GridCacheObjectToStringSelfTest.java  |    2 +-
 .../GridCacheOrderedPreloadingSelfTest.java     |    2 +-
 .../cache/GridCacheP2PUndeploySelfTest.java     |    6 +-
 .../cache/GridCachePartitionedGetSelfTest.java  |    2 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |    6 +-
 .../GridCacheQueryInternalKeysSelfTest.java     |    2 +-
 .../cache/GridCacheSlowTxWarnTest.java          |    4 +-
 .../cache/GridCacheStoreValueBytesSelfTest.java |    4 +-
 .../cache/GridCacheSwapReloadSelfTest.java      |    2 +-
 .../cache/GridCacheTtlManagerLoadTest.java      |    2 +-
 .../cache/GridCacheTtlManagerSelfTest.java      |    2 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |    2 +-
 .../processors/cache/IgniteTxAbstractTest.java  |    2 +-
 .../IgniteTxConcurrentGetAbstractTest.java      |    3 +-
 .../IgniteTxExceptionAbstractSelfTest.java      |    2 +-
 .../cache/IgniteTxMultiNodeAbstractTest.java    |    6 +-
 .../IgniteTxStoreExceptionAbstractSelfTest.java |    2 +-
 ...actQueueFailoverDataConsistencySelfTest.java |    2 +-
 .../GridCacheCountDownLatchSelfTest.java        |    2 +-
 .../GridCacheQueueCleanupSelfTest.java          |    2 +-
 .../GridCacheSetAbstractSelfTest.java           |    8 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |    2 +-
 .../GridCacheAtomicTimeoutSelfTest.java         |    2 +-
 .../distributed/GridCacheEventAbstractTest.java |    2 +-
 ...heExpiredEntriesPreloadAbstractSelfTest.java |    2 +-
 .../GridCacheMultiNodeLockAbstractTest.java     |    2 +-
 ...iteTxConsistencyRestartAbstractSelfTest.java |    4 +-
 ...xOriginatingNodeFailureAbstractSelfTest.java |   10 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   20 +-
 .../dht/GridCacheAtomicNearCacheSelfTest.java   |    2 +-
 .../dht/GridCacheColocatedDebugTest.java        |    6 +-
 .../dht/GridCacheDhtEntrySelfTest.java          |    2 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |    4 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |    4 +-
 .../dht/GridCacheDhtMappingSelfTest.java        |    2 +-
 .../dht/GridCacheDhtPreloadDelayedSelfTest.java |    2 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |    2 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |   32 +-
 ...itionedTxOriginatingNodeFailureSelfTest.java |    8 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |    2 +-
 .../atomic/GridCacheAtomicPreloadSelfTest.java  |    4 +-
 .../near/GridCacheNearMultiNodeSelfTest.java    |    6 +-
 .../near/GridCacheNearReadersSelfTest.java      |    4 +-
 .../near/GridCacheNearTxMultiNodeSelfTest.java  |    2 +-
 .../GridCachePartitionedFullApiSelfTest.java    |    4 +-
 ...achePartitionedMultiNodeCounterSelfTest.java |    2 +-
 .../GridCachePartitionedProjectionSelfTest.java |    4 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |    2 +-
 .../IgniteCacheExpiryPolicyAbstractTest.java    |    2 +-
 ...ridCacheContinuousQueryAbstractSelfTest.java |    4 +-
 .../clock/GridTimeSyncProcessorSelfTest.java    |    6 +-
 .../continuous/GridEventConsumeSelfTest.java    |    2 +-
 .../processors/fs/GridGgfsAbstractSelfTest.java |    2 +-
 .../processors/fs/GridGgfsModesSelfTest.java    |    4 +-
 .../fs/GridGgfsProcessorSelfTest.java           |    2 +-
 ...IpcEndpointRegistrationAbstractSelfTest.java |    2 +-
 .../processors/fs/GridGgfsSizeSelfTest.java     |    8 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |    6 +-
 .../GridServiceReassignmentSelfTest.java        |    2 +-
 .../util/future/GridFinishedFutureSelfTest.java |    2 +-
 .../communication/GridIoManagerBenchmark.java   |   10 +-
 .../communication/GridIoManagerBenchmark0.java  |   12 +-
 .../ignite/loadtests/dsi/GridDsiPerfJob.java    |    2 +-
 .../marshaller/GridMarshallerAbstractTest.java  |    6 +-
 .../tcp/GridTcpDiscoveryMultiThreadedTest.java  |    2 +-
 .../discovery/tcp/GridTcpDiscoverySelfTest.java |   14 +-
 .../ignite/testframework/GridTestUtils.java     |    4 +-
 .../testframework/junits/GridAbstractTest.java  |   12 +-
 .../junits/GridTestKernalContext.java           |    2 +-
 .../junits/common/GridCommonAbstractTest.java   |   22 +-
 .../ignite/testsuites/IgniteBasicTestSuite.java |    2 +-
 .../testsuites/IgniteComputeGridTestSuite.java  |    4 +-
 .../GridHadoopDefaultMapReducePlanner.java      |    2 +-
 .../GridGgfsHadoopFileSystemClientSelfTest.java |    2 +-
 ...idHadoopDefaultMapReducePlannerSelfTest.java |    4 +-
 .../hadoop/GridHadoopJobTrackerSelfTest.java    |    2 +-
 .../cache/GridCacheAbstractQuerySelfTest.java   |    6 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |    4 +-
 .../cache/GridCacheQueryLoadSelfTest.java       |    2 +-
 .../GridCacheQueryMultiThreadedSelfTest.java    |    8 +-
 .../GridCacheReplicatedFieldsQuerySelfTest.java |    2 +-
 .../GridCacheReplicatedQuerySelfTest.java       |    4 +-
 .../query/h2/sql/GridQueryParsingTest.java      |    2 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |    2 +-
 .../GridTmLookupLifecycleAwareSelfTest.java     |    2 +-
 .../cache/spring/SpringDynamicCacheManager.java |    2 +-
 .../scala/org/apache/ignite/visor/visor.scala   |    6 +-
 163 files changed, 4701 insertions(+), 4707 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java
index b5c5a85..e72e684 100644
--- a/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java
@@ -506,7 +506,7 @@ public abstract class ClientAbstractMultiThreadedSelfTest extends GridCommonAbst
     private void printAffinityState(Iterable<Ignite> grids) {
         for (Ignite g : grids) {
             GridAffinityAssignmentCache affCache = getFieldValue(
-                ((GridKernal)g).internalCache(PARTITIONED_CACHE_NAME).context().affinity(),
+                ((IgniteKernal)g).internalCache(PARTITIONED_CACHE_NAME).context().affinity(),
                 "aff");
 
             CacheAffinityFunction aff = getFieldValue(affCache, "aff");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java
index 363bfbc..d1f3319 100644
--- a/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/client/integration/ClientAbstractMultiNodeSelfTest.java
@@ -399,7 +399,7 @@ public abstract class ClientAbstractMultiNodeSelfTest extends GridCommonAbstract
      * @throws Exception If failed.
      */
     public void testInvalidateFlag() throws Exception {
-        GridEx g0 = grid(0);
+        IgniteEx g0 = grid(0);
 
         GridCache<String, String> cache = g0.cache(PARTITIONED_CACHE_NAME);
 
@@ -783,7 +783,7 @@ public abstract class ClientAbstractMultiNodeSelfTest extends GridCommonAbstract
             if (!(o instanceof GridDistributedLockRequest))
                 return;
 
-            GridKernal g = (GridKernal)G.ignite(ignite.configuration().getNodeId());
+            IgniteKernal g = (IgniteKernal)G.ignite(ignite.configuration().getNodeId());
 
             GridCacheContext<Object, Object> cacheCtx = g.internalCache(REPLICATED_ASYNC_CACHE_NAME).context();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestBinaryProtocolSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestBinaryProtocolSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestBinaryProtocolSelfTest.java
index f14a4ad..3e3a36c 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestBinaryProtocolSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestBinaryProtocolSelfTest.java
@@ -181,7 +181,7 @@ public class RestBinaryProtocolSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testFailure() throws Exception {
-        GridKernal kernal = ((GridKernal)grid());
+        IgniteKernal kernal = ((IgniteKernal)grid());
 
         GridRestProcessor proc = kernal.context().rest();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TaskCommandHandlerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TaskCommandHandlerSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TaskCommandHandlerSelfTest.java
index 1f15f1d..84656de 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TaskCommandHandlerSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/TaskCommandHandlerSelfTest.java
@@ -159,7 +159,7 @@ public class TaskCommandHandlerSelfTest extends GridCommonAbstractTest {
 
         GridClientFactory.stop(client.id(), true);
 
-        GridKernal g = (GridKernal)grid(0);
+        IgniteKernal g = (IgniteKernal)grid(0);
 
         Map<GridRestCommand, GridRestCommandHandler> handlers = U.field(g.context().rest(), "handlers");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/fs/mapreduce/IgniteFsTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/fs/mapreduce/IgniteFsTask.java b/modules/core/src/main/java/org/apache/ignite/fs/mapreduce/IgniteFsTask.java
index 29a5fb5..ce51c3d 100644
--- a/modules/core/src/main/java/org/apache/ignite/fs/mapreduce/IgniteFsTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/fs/mapreduce/IgniteFsTask.java
@@ -89,7 +89,7 @@ public abstract class IgniteFsTask<T, R> extends ComputeTaskAdapter<IgniteFsTask
         assert args != null;
 
         IgniteFs ggfs = ignite.fileSystem(args.ggfsName());
-        IgniteFsProcessorAdapter ggfsProc = ((GridKernal) ignite).context().ggfs();
+        IgniteFsProcessorAdapter ggfsProc = ((IgniteKernal) ignite).context().ggfs();
 
         Map<ComputeJob, ClusterNode> splitMap = new HashMap<>();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
index 1086fbf..b3f8cc0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/ClusterGroupAdapter.java
@@ -638,7 +638,7 @@ public class ClusterGroupAdapter implements ClusterGroupEx, Externalizable {
      */
     protected Object readResolve() throws ObjectStreamException {
         try {
-            GridKernal g = GridGainEx.gridx(gridName);
+            IgniteKernal g = GridGainEx.gridx(gridName);
 
             return ids != null ? new ClusterGroupAdapter(g, g.context(), subjId, ids) :
                 p != null ? new ClusterGroupAdapter(g, g.context(), subjId, p) : g;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/GridEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEx.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEx.java
deleted file mode 100644
index 6e004fc..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEx.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.internal.processors.interop.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-/**
- * Extended Grid interface which provides some additional methods required for kernal and Visor.
- */
-public interface GridEx extends Ignite, ClusterGroupEx, IgniteCluster {
-    /**
-     * Gets utility cache.
-     *
-     * @param keyCls Key class.
-     * @param valCls Value class.
-     * @return Utility cache.
-     */
-    public <K extends GridCacheUtilityKey, V> GridCacheProjectionEx<K, V> utilityCache(Class<K> keyCls, Class<V> valCls);
-
-    /**
-     * Gets the cache instance for the given name if one is configured or
-     * <tt>null</tt> otherwise returning even non-public caches.
-     *
-     * @param <K> Key type.
-     * @param <V> Value type.
-     * @param name Cache name.
-     * @return Cache instance for given name or <tt>null</tt> if one does not exist.
-     */
-    @Nullable public <K, V> GridCache<K, V> cachex(@Nullable String name);
-
-    /**
-     * Gets default cache instance if one is configured or <tt>null</tt> otherwise returning even non-public caches.
-     * The {@link org.apache.ignite.cache.GridCache#name()} method on default instance returns <tt>null</tt>.
-     *
-     * @param <K> Key type.
-     * @param <V> Value type.
-     * @return Default cache instance.
-     */
-    @Nullable public <K, V> GridCache<K, V> cachex();
-
-    /**
-     * Gets configured cache instance that satisfy all provided predicates including non-public caches. If no
-     * predicates provided - all configured caches will be returned.
-     *
-     * @param p Predicates. If none provided - all configured caches will be returned.
-     * @return Configured cache instances that satisfy all provided predicates.
-     */
-    public Collection<GridCache<?, ?>> cachesx(@Nullable IgnitePredicate<? super GridCache<?, ?>>... p);
-
-    /**
-     * Checks if the event type is user-recordable.
-     *
-     * @param type Event type to check.
-     * @return {@code true} if passed event should be recorded, {@code false} - otherwise.
-     */
-    public boolean eventUserRecordable(int type);
-
-    /**
-     * Checks whether all provided events are user-recordable.
-     * <p>
-     * Note that this method supports only predefined GridGain events.
-     *
-     * @param types Event types.
-     * @return Whether all events are recordable.
-     * @throws IllegalArgumentException If {@code types} contains user event type.
-     */
-    public boolean allEventsUserRecordable(int[] types);
-
-    /**
-     * Gets list of compatible versions.
-     *
-     * @return Compatible versions.
-     */
-    public Collection<String> compatibleVersions();
-
-    /**
-     * @return Grace period left in minutes if bursting or {@code -1} otherwise.
-     */
-    public long licenseGracePeriodLeft();
-
-    /**
-     * Whether or not remote JMX management is enabled for this node.
-     *
-     * @return {@code True} if remote JMX management is enabled - {@code false} otherwise.
-     */
-    public boolean isJmxRemoteEnabled();
-
-    /**
-     * Whether or not node restart is enabled.
-     *
-     * @return {@code True} if restart mode is enabled, {@code false} otherwise.
-     */
-    public boolean isRestartEnabled();
-
-    /**
-     * Whether or not SMTP is configured.
-     *
-     * @return {@code True} if SMTP is configured - {@code false} otherwise.
-     */
-    public boolean isSmtpEnabled();
-
-    /**
-     * Schedule sending of given email to all configured admin emails.
-     */
-    IgniteInternalFuture<Boolean> sendAdminEmailAsync(String subj, String body, boolean html);
-
-    /**
-     * Get GGFS instance returning null if it doesn't exist.
-     *
-     * @param name GGFS name.
-     * @return GGFS.
-     */
-    @Nullable public IgniteFs ggfsx(@Nullable String name);
-
-    /**
-     * Gets interop processor.
-     *
-     * @return Interop processor.
-     */
-    public GridInteropProcessor interop();
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/GridGainEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridGainEx.java b/modules/core/src/main/java/org/apache/ignite/internal/GridGainEx.java
index 553d874..65c2776 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridGainEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridGainEx.java
@@ -920,7 +920,7 @@ public class GridGainEx {
         GridNamedInstance dfltGrid0 = dfltGrid;
 
         if (dfltGrid0 != null) {
-            GridKernal g = dfltGrid0.grid();
+            IgniteKernal g = dfltGrid0.grid();
 
             if (g != null)
                 allIgnites.add(g);
@@ -947,14 +947,14 @@ public class GridGainEx {
         GridNamedInstance dfltGrid0 = dfltGrid;
 
         if (dfltGrid0 != null) {
-            GridKernal g = dfltGrid0.grid();
+            IgniteKernal g = dfltGrid0.grid();
 
             if (g != null && g.getLocalNodeId().equals(locNodeId))
                 return g;
         }
 
         for (GridNamedInstance grid : grids.values()) {
-            GridKernal g = grid.grid();
+            IgniteKernal g = grid.grid();
 
             if (g != null && g.getLocalNodeId().equals(locNodeId))
                 return g;
@@ -998,10 +998,10 @@ public class GridGainEx {
      * @param name Grid name.
      * @return Grid instance.
      */
-    public static GridKernal gridx(@Nullable String name) {
+    public static IgniteKernal gridx(@Nullable String name) {
         GridNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
 
-        GridKernal res;
+        IgniteKernal res;
 
         if (grid == null || (res = grid.gridx()) == null)
             throw new IllegalStateException("Grid instance was not properly started or was already stopped: " + name);
@@ -1151,7 +1151,7 @@ public class GridGainEx {
         private final String name;
 
         /** Grid instance. */
-        private volatile GridKernal grid;
+        private volatile IgniteKernal grid;
 
         /** Executor service. */
         private ExecutorService execSvc;
@@ -1255,7 +1255,7 @@ public class GridGainEx {
          *
          * @return Grid instance.
          */
-        GridKernal grid() {
+        IgniteKernal grid() {
             if (starterThread != Thread.currentThread())
                 U.awaitQuiet(startLatch);
 
@@ -1267,7 +1267,7 @@ public class GridGainEx {
          *
          * @return Grid instance.
          */
-        public GridKernal gridx() {
+        public IgniteKernal gridx() {
             return grid;
         }
 
@@ -1930,7 +1930,7 @@ public class GridGainEx {
             boolean started = false;
 
             try {
-                GridKernal grid0 = new GridKernal(startCtx.springContext());
+                IgniteKernal grid0 = new IgniteKernal(startCtx.springContext());
 
                 // Init here to make grid available to lifecycle listeners.
                 grid = grid0;
@@ -2103,7 +2103,7 @@ public class GridGainEx {
          *      should be cancelled.
          */
         private synchronized void stop0(boolean cancel) {
-            GridKernal grid0 = grid;
+            IgniteKernal grid0 = grid;
 
             // Double check.
             if (grid0 == null) {


[07/50] [abbrv] incubator-ignite git commit: minor fix for data loader (add of single key optimizations) and example

Posted by vo...@apache.org.
minor fix for data loader (add of single key optimizations) and example


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

Branch: refs/heads/ignite-16
Commit: ddbd18e4a3872cd31c036693650990e506334c99
Parents: 38164d0
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Wed Jan 28 19:12:59 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Wed Jan 28 19:12:59 2015 +0300

----------------------------------------------------------------------
 .../datagrid/CacheDataLoaderExample.java        | 15 ++++++----
 .../dataload/IgniteDataLoaderImpl.java          | 29 ++++++++++++++------
 2 files changed, 31 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ddbd18e4/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheDataLoaderExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheDataLoaderExample.java b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheDataLoaderExample.java
index a7da7a6..853a72d 100644
--- a/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheDataLoaderExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/datagrid/CacheDataLoaderExample.java
@@ -58,11 +58,16 @@ public class CacheDataLoaderExample {
             // Clean up caches on all nodes before run.
             g.cache(CACHE_NAME).globalClearAll(0);
 
+            System.out.println();
+            System.out.println(">>> Cache clear finished.");
+
+            long start = System.currentTimeMillis();
+
             try (IgniteDataLoader<Integer, String> ldr = g.dataLoader(CACHE_NAME)) {
                 // Configure loader.
                 ldr.perNodeBufferSize(1024);
-
-                long start = System.currentTimeMillis();
+                ldr.perNodeParallelLoadOperations(8);
+                ldr.isolated(true);
 
                 for (int i = 0; i < ENTRY_COUNT; i++) {
                     ldr.addData(i, Integer.toString(i));
@@ -71,11 +76,11 @@ public class CacheDataLoaderExample {
                     if (i > 0 && i % 10000 == 0)
                         System.out.println("Loaded " + i + " keys.");
                 }
+            }
 
-                long end = System.currentTimeMillis();
+            long end = System.currentTimeMillis();
 
-                System.out.println(">>> Loaded " + ENTRY_COUNT + " keys in " + (end - start) + "ms.");
-            }
+            System.out.println(">>> Loaded " + ENTRY_COUNT + " keys in " + (end - start) + "ms.");
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/ddbd18e4/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java
index 4b1f50c..fd9ab76 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java
@@ -371,14 +371,18 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
         try {
             GridFutureAdapter<Object> resFut = new GridFutureAdapter<>(ctx);
 
+            resFut.listenAsync(rmvActiveFut);
+
             activeFuts.add(resFut);
 
-            resFut.listenAsync(rmvActiveFut);
+            Collection<K> keys = null;
 
-            Collection<K> keys = new GridConcurrentHashSet<>(entries.size(), 1.0f, 16);
+            if (entries.size() > 1) {
+                keys = new GridConcurrentHashSet<>(entries.size(), U.capacity(entries.size()), 1);
 
-            for (Map.Entry<K, V> entry : entries)
-                keys.add(entry.getKey());
+                for (Map.Entry<K, V> entry : entries)
+                    keys.add(entry.getKey());
+            }
 
             load0(entries, resFut, keys, 0);
 
@@ -420,7 +424,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     private void load0(
         Collection<? extends Map.Entry<K, V>> entries,
         final GridFutureAdapter<Object> resFut,
-        final Collection<K> activeKeys,
+        @Nullable final Collection<K> activeKeys,
         final int remaps
     ) {
         assert entries != null;
@@ -492,11 +496,20 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                     try {
                         t.get();
 
-                        for (Map.Entry<K, V> e : entriesForNode)
-                            activeKeys.remove(e.getKey());
+                        if (activeKeys != null) {
+                            for (Map.Entry<K, V> e : entriesForNode)
+                                activeKeys.remove(e.getKey());
 
-                        if (activeKeys.isEmpty())
+                            if (activeKeys.isEmpty())
+                                resFut.onDone();
+                        }
+                        else {
+                            assert entriesForNode.size() == 1;
+
+                            // That has been a single key,
+                            // so complete result future right away.
                             resFut.onDone();
+                        }
                     }
                     catch (IgniteCheckedException e1) {
                         if (log.isDebugEnabled())


[10/50] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/sprint-1' into ignite-91

Posted by vo...@apache.org.
Merge remote-tracking branch 'remotes/origin/sprint-1' into ignite-91


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

Branch: refs/heads/ignite-16
Commit: a92f08456d85ccc5fcb1f9f9ec670795b45ddd6a
Parents: 27827c8 6cab420
Author: sboikov <se...@inria.fr>
Authored: Wed Jan 28 21:42:56 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Wed Jan 28 21:42:56 2015 +0300

----------------------------------------------------------------------
 .../hibernate/example-hibernate-L2-cache.xml    |  14 +-
 .../datagrid/CacheDataLoaderExample.java        |  15 +-
 .../org/apache/ignite/internal/GridKernal.java  |  30 +--
 .../processors/cache/GridCacheContext.java      |  43 +++++
 .../processors/cache/GridCacheEntryEx.java      |   8 +
 .../processors/cache/GridCacheMapEntry.java     | 116 +++++++++---
 .../cache/GridCacheUpdateAtomicResult.java      |   6 +-
 .../processors/cache/GridDrResolveResult.java   |  63 -------
 .../GridDistributedTxRemoteAdapter.java         |  47 +++--
 .../dht/atomic/GridDhtAtomicCache.java          |  10 +-
 .../processors/cache/dr/GridCacheDrManager.java |  57 ++----
 .../cache/dr/os/GridOsCacheDrManager.java       |  35 +---
 .../cache/transactions/IgniteTxAdapter.java     |  45 +++++
 .../transactions/IgniteTxLocalAdapter.java      |  52 +++--
 ...ridCacheVersionAbstractConflictResolver.java |  56 ++++++
 .../GridCacheVersionConflictContext.java        |  73 +++++++
 .../GridCacheVersionConflictContextImpl.java    | 188 +++++++++++++++++++
 .../GridCacheVersionConflictResolver.java       |  59 ++++++
 .../version/GridCacheVersionedEntryEx.java      |   2 +-
 .../dataload/IgniteDataLoaderImpl.java          |  29 ++-
 .../apache/ignite/internal/util/GridUtils.java  |   2 +-
 .../processors/cache/GridCacheTestEntryEx.java  |   5 +
 ...ClusterMetricsSnapshotSerializeSelfTest.java |  52 ++++-
 .../hibernate/GridHibernateRegionFactory.java   |  10 +-
 24 files changed, 773 insertions(+), 244 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/a92f0845/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------


[16/50] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'remotes/origin/ignite-91' into sprint-1

Posted by vo...@apache.org.
Merge remote-tracking branch 'remotes/origin/ignite-91' into sprint-1


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

Branch: refs/heads/ignite-16
Commit: c6e4f450755c8912b088ef1bb16873eb20e45b13
Parents: 029e96b 645aa34
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jan 29 09:19:12 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jan 29 09:19:12 2015 +0300

----------------------------------------------------------------------
 .../integration/ClientAbstractSelfTest.java     |   8 +-
 .../processors/cache/GridCacheAdapter.java      |  26 +-
 .../dht/GridPartitionedGetFuture.java           |  11 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  25 ++
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  29 +-
 .../dht/colocated/GridDhtColocatedCache.java    |   3 +
 .../distributed/near/GridNearGetFuture.java     |  11 +-
 .../local/atomic/GridLocalAtomicCache.java      |  70 ++++-
 .../transactions/IgniteTxLocalAdapter.java      |  13 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java | 301 +++++++++++++++++--
 .../dht/GridCacheAtomicFullApiSelfTest.java     |  11 +-
 11 files changed, 438 insertions(+), 70 deletions(-)
----------------------------------------------------------------------



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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
index c2c330e..a4d0fa6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManager.java
@@ -947,7 +947,7 @@ public class GridDiscoveryManager extends GridManagerAdapter<DiscoverySpi> {
      * @param awaitVer Topology version to await.
      * @return Future.
      */
-    public IgniteFuture<Long> topologyFuture(final long awaitVer) {
+    public IgniteInternalFuture<Long> topologyFuture(final long awaitVer) {
         long topVer = topologyVersion();
 
         if (topVer >= awaitVer)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
index 6e4670a..ea2d85e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/managers/eventstorage/GridEventStorageManager.java
@@ -661,7 +661,7 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
      * @param types Event types to wait for.
      * @return Event future.
      */
-    public <T extends IgniteEvent> IgniteFuture<T> waitForEvent(@Nullable final IgnitePredicate<T> p,
+    public <T extends IgniteEvent> IgniteInternalFuture<T> waitForEvent(@Nullable final IgnitePredicate<T> p,
         @Nullable int... types) {
         final GridFutureAdapter<T> fut = new GridFutureAdapter<>(ctx);
 
@@ -760,7 +760,7 @@ public class GridEventStorageManager extends GridManagerAdapter<EventStorageSpi>
      * @param timeout Maximum time to wait for result, if {@code 0}, then wait until result is received.
      * @return Collection of events.
      */
-    public <T extends IgniteEvent> IgniteFuture<List<T>> remoteEventsAsync(final IgnitePredicate<T> p,
+    public <T extends IgniteEvent> IgniteInternalFuture<List<T>> remoteEventsAsync(final IgnitePredicate<T> p,
         final Collection<? extends ClusterNode> nodes, final long timeout) {
         assert p != null;
         assert nodes != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
index 3436a9f..c98ab73 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityAssignmentCache.java
@@ -23,7 +23,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.portables.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -212,7 +211,7 @@ public class GridAffinityAssignmentCache {
      * @param topVer Topology version to await for.
      * @return Future that will be completed after affinity for topology version {@code topVer} is calculated.
      */
-    public IgniteFuture<Long> readyFuture(long topVer) {
+    public IgniteInternalFuture<Long> readyFuture(long topVer) {
         GridAffinityAssignment aff = head.get();
 
         if (aff.topologyVersion() >= topVer) {
@@ -346,7 +345,7 @@ public class GridAffinityAssignmentCache {
                 log.debug("Will wait for topology version [locNodeId=" + ctx.localNodeId() +
                 ", topVer=" + topVer + ']');
 
-            IgniteFuture<Long> fut = readyFuture(topVer);
+            IgniteInternalFuture<Long> fut = readyFuture(topVer);
 
             if (fut != null)
                 fut.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
index ecc94df..10888a6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessor.java
@@ -61,7 +61,7 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
     private static final String NULL_NAME = U.id8(UUID.randomUUID());
 
     /** Affinity map. */
-    private final ConcurrentMap<AffinityAssignmentKey, IgniteFuture<AffinityInfo>> affMap = new ConcurrentHashMap8<>();
+    private final ConcurrentMap<AffinityAssignmentKey, IgniteInternalFuture<AffinityInfo>> affMap = new ConcurrentHashMap8<>();
 
     /** Listener. */
     private final GridLocalEventListener lsnr = new GridLocalEventListener() {
@@ -266,7 +266,7 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
     private AffinityInfo affinityCache(@Nullable final String cacheName, long topVer) throws IgniteCheckedException {
         AffinityAssignmentKey key = new AffinityAssignmentKey(cacheName, topVer);
 
-        IgniteFuture<AffinityInfo> fut = affMap.get(key);
+        IgniteInternalFuture<AffinityInfo> fut = affMap.get(key);
 
         if (fut != null)
             return fut.get();
@@ -283,7 +283,7 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
                 new GridAffinityAssignment(topVer, cctx.affinity().assignments(topVer)),
                 cctx.portableEnabled());
 
-            IgniteFuture<AffinityInfo> old = affMap.putIfAbsent(key, new GridFinishedFuture<>(ctx, info));
+            IgniteInternalFuture<AffinityInfo> old = affMap.putIfAbsent(key, new GridFinishedFuture<>(ctx, info));
 
             if (old != null)
                 info = old.get();
@@ -304,7 +304,7 @@ public class GridAffinityProcessor extends GridProcessorAdapter {
 
         GridFutureAdapter<AffinityInfo> fut0 = new GridFutureAdapter<>();
 
-        IgniteFuture<AffinityInfo> old = affMap.putIfAbsent(key, fut0);
+        IgniteInternalFuture<AffinityInfo> old = affMap.putIfAbsent(key, fut0);
 
         if (old != null)
             return old.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java
index d918499..8788eac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/CacheLockImpl.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
 import javax.cache.*;
@@ -127,7 +127,7 @@ class CacheLockImpl<K, V> implements Lock {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
-            IgniteFuture<Boolean> fut = delegate.lockAllAsync(keys, unit.toMillis(time));
+            IgniteInternalFuture<Boolean> fut = delegate.lockAllAsync(keys, unit.toMillis(time));
 
             try {
                 boolean res = fut.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 4f401c0..92d2039 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -92,8 +92,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
     /** {@link GridCacheReturn}-to-value conversion. */
     private static final IgniteClosure RET2VAL =
-        new CX1<IgniteFuture<GridCacheReturn<Object>>, Object>() {
-            @Nullable @Override public Object applyx(IgniteFuture<GridCacheReturn<Object>> fut) throws IgniteCheckedException {
+        new CX1<IgniteInternalFuture<GridCacheReturn<Object>>, Object>() {
+            @Nullable @Override public Object applyx(IgniteInternalFuture<GridCacheReturn<Object>> fut) throws IgniteCheckedException {
                 return fut.get().value();
             }
 
@@ -104,8 +104,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
     /** {@link GridCacheReturn}-to-success conversion. */
     private static final IgniteClosure RET2FLAG =
-        new CX1<IgniteFuture<GridCacheReturn<Object>>, Boolean>() {
-            @Override public Boolean applyx(IgniteFuture<GridCacheReturn<Object>> fut) throws IgniteCheckedException {
+        new CX1<IgniteInternalFuture<GridCacheReturn<Object>>, Boolean>() {
+            @Override public Boolean applyx(IgniteInternalFuture<GridCacheReturn<Object>> fut) throws IgniteCheckedException {
                 return fut.get().success();
             }
 
@@ -562,7 +562,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param filter Optional filter.
      * @return Locks future.
      */
-    public abstract IgniteFuture<Boolean> txLockAsync(
+    public abstract IgniteInternalFuture<Boolean> txLockAsync(
         Collection<? extends K> keys,
         long timeout,
         IgniteTxLocalEx<K, V> tx,
@@ -645,7 +645,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> containsKeyAsync(K key) {
+    @Override public IgniteInternalFuture<Boolean> containsKeyAsync(K key) {
         return containsKeyAsync(key, null);
     }
 
@@ -654,7 +654,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param filter Filter.
      * @return Future.
      */
-    public IgniteFuture<Boolean> containsKeyAsync(K key, @Nullable IgnitePredicate<CacheEntry<K, V>> filter) {
+    public IgniteInternalFuture<Boolean> containsKeyAsync(K key, @Nullable IgnitePredicate<CacheEntry<K, V>> filter) {
         return new GridFinishedFuture<>(ctx.kernalContext(), containsKey(key, filter));
     }
 
@@ -1295,7 +1295,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             // Send job to remote nodes only.
             Collection<ClusterNode> nodes = ctx.grid().forCache(name()).forRemotes().nodes();
 
-            IgniteFuture<Object> fut = null;
+            IgniteInternalFuture<Object> fut = null;
 
             if (!nodes.isEmpty()) {
                 ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, timeout);
@@ -1392,14 +1392,14 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getForcePrimaryAsync(final K key) {
+    @Override public IgniteInternalFuture<V> getForcePrimaryAsync(final K key) {
         ctx.denyOnFlag(LOCAL);
 
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(Collections.singletonList(key), /*force primary*/true, /*skip tx*/false, null, null,
-            taskName, true).chain(new CX1<IgniteFuture<Map<K, V>>, V>() {
-            @Override public V applyx(IgniteFuture<Map<K, V>> e) throws IgniteCheckedException {
+            taskName, true).chain(new CX1<IgniteInternalFuture<Map<K, V>>, V>() {
+            @Override public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException {
                 return e.get().get(key);
             }
         });
@@ -1413,7 +1413,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllOutTxAsync(List<K> keys) {
+    @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(List<K> keys) {
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllAsync(keys, !ctx.config().isReadFromBackup(), /*skip tx*/true, null, null, taskName, true);
@@ -1425,7 +1425,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> reloadAsync(K key) {
+    @Override public IgniteInternalFuture<V> reloadAsync(K key) {
         return reloadAsync(key, (IgnitePredicate<CacheEntry<K, V>>[])null);
     }
 
@@ -1435,7 +1435,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys) {
+    @Override public IgniteInternalFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys) {
         return reloadAllAsync(keys, (IgnitePredicate<CacheEntry<K, V>>[])null);
     }
 
@@ -1447,7 +1447,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> reloadAllAsync() {
+    @Override public IgniteInternalFuture<?> reloadAllAsync() {
         ctx.denyOnFlags(F.asList(LOCAL, READ));
 
         return reloadAllAsync(keySet(), (IgnitePredicate<CacheEntry<K, V>>[])null);
@@ -1463,7 +1463,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param vis Visitor.
      * @return Future.
      */
-    public IgniteFuture<Object> readThroughAllAsync(final Collection<? extends K> keys,
+    public IgniteInternalFuture<Object> readThroughAllAsync(final Collection<? extends K> keys,
         boolean reload,
         @Nullable final IgniteTxEx<K, V> tx,
         IgnitePredicate<CacheEntry<K, V>>[] filter,
@@ -1506,7 +1506,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param filter Filter.
      * @return Future.
      */
-    public IgniteFuture<Map<K, V>> reloadAllAsync(@Nullable Collection<? extends K> keys, boolean ret,
+    public IgniteInternalFuture<Map<K, V>> reloadAllAsync(@Nullable Collection<? extends K> keys, boolean ret,
         @Nullable UUID subjId, String taskName, @Nullable final IgnitePredicate<CacheEntry<K, V>>... filter) {
         ctx.denyOnFlag(READ);
 
@@ -1566,7 +1566,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
                 final Collection<K> loadedKeys = new GridConcurrentHashSet<>();
 
-                IgniteFuture<Object> readFut =
+                IgniteInternalFuture<Object> readFut =
                     readThroughAllAsync(absentKeys, true, null, filter, subjId, taskName, new CI2<K, V>() {
                         /** Version for all loaded entries. */
                         private GridCacheVersion nextVer = ctx.versions().next();
@@ -1635,8 +1635,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                         }
                     });
 
-                return readFut.chain(new CX1<IgniteFuture<Object>, Map<K, V>>() {
-                    @Override public Map<K, V> applyx(IgniteFuture<Object> e) throws IgniteCheckedException {
+                return readFut.chain(new CX1<IgniteInternalFuture<Object>, Map<K, V>>() {
+                    @Override public Map<K, V> applyx(IgniteInternalFuture<Object> e) throws IgniteCheckedException {
                         // Touch all not loaded keys.
                         for (K key : absentKeys) {
                             if (!loadedKeys.contains(key)) {
@@ -1705,18 +1705,18 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getAsync(final K key) {
+    @Override public IgniteInternalFuture<V> getAsync(final K key) {
         A.notNull(key, "key");
 
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
-        IgniteFuture<V> fut = getAsync(key, true, null);
+        IgniteInternalFuture<V> fut = getAsync(key, true, null);
 
         if (ctx.config().getInterceptor() != null)
-            fut =  fut.chain(new CX1<IgniteFuture<V>, V>() {
-                @Override public V applyx(IgniteFuture<V> f) throws IgniteCheckedException {
+            fut =  fut.chain(new CX1<IgniteInternalFuture<V>, V>() {
+                @Override public V applyx(IgniteInternalFuture<V> f) throws IgniteCheckedException {
                     return (V)ctx.config().getInterceptor().onGet(key, f.get());
                 }
             });
@@ -1747,18 +1747,18 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(@Nullable final Collection<? extends K> keys) {
+    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable final Collection<? extends K> keys) {
         A.notNull(keys, "keys");
 
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
-        IgniteFuture<Map<K, V>> fut = getAllAsync(keys, true, null);
+        IgniteInternalFuture<Map<K, V>> fut = getAllAsync(keys, true, null);
 
         if (ctx.config().getInterceptor() != null)
-            return fut.chain(new CX1<IgniteFuture<Map<K, V>>, Map<K, V>>() {
-                @Override public Map<K, V> applyx(IgniteFuture<Map<K, V>> f) throws IgniteCheckedException {
+            return fut.chain(new CX1<IgniteInternalFuture<Map<K, V>>, Map<K, V>>() {
+                @Override public Map<K, V> applyx(IgniteInternalFuture<Map<K, V>> f) throws IgniteCheckedException {
                     return interceptGet(keys, f.get());
                 }
             });
@@ -1811,7 +1811,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    protected IgniteFuture<Map<K, V>> getAllAsync(
+    protected IgniteInternalFuture<Map<K, V>> getAllAsync(
         @Nullable Collection<? extends K> keys,
         boolean forcePrimary,
         boolean skipTx,
@@ -1838,7 +1838,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    public IgniteFuture<Map<K, V>> getAllAsync(@Nullable final Collection<? extends K> keys,
+    public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable final Collection<? extends K> keys,
         boolean readThrough,
         @Nullable GridCacheEntryEx<K, V> cached,
         boolean checkTx,
@@ -2054,8 +2054,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                                 return map;
                             }
                         }), true),
-                        new C2<Map<K, V>, Exception, IgniteFuture<Map<K, V>>>() {
-                            @Override public IgniteFuture<Map<K, V>> apply(Map<K, V> map, Exception e) {
+                        new C2<Map<K, V>, Exception, IgniteInternalFuture<Map<K, V>>>() {
+                            @Override public IgniteInternalFuture<Map<K, V>> apply(Map<K, V> map, Exception e) {
                                 if (e != null)
                                     return new GridFinishedFuture<>(ctx.kernalContext(), e);
 
@@ -2115,7 +2115,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             final GridCacheEntryEx<K, V> cached0 = cached;
 
             return asyncOp(tx, new AsyncOp<Map<K, V>>(keys) {
-                @Override public IgniteFuture<Map<K, V>> op(IgniteTxLocalAdapter<K, V> tx) {
+                @Override public IgniteInternalFuture<Map<K, V>> op(IgniteTxLocalAdapter<K, V> tx) {
                     return ctx.wrapCloneMap(tx.getAllAsync(ctx, keys, cached0, deserializePortable, filter));
                 }
             });
@@ -2186,13 +2186,13 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putAsync(K key, V val,
+    @Override public IgniteInternalFuture<V> putAsync(K key, V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
-        IgniteFuture<V> fut = putAsync(key, val, null, -1, filter);
+        IgniteInternalFuture<V> fut = putAsync(key, val, null, -1, filter);
 
         if (statsEnabled)
             fut.listenAsync(new UpdatePutAndGetTimeStatClosure<V>(metrics0(), start));
@@ -2201,7 +2201,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putAsync(final K key, final V val, @Nullable final GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<V> putAsync(final K key, final V val, @Nullable final GridCacheEntryEx<K, V> entry,
         final long ttl, @Nullable final IgnitePredicate<CacheEntry<K, V>>... filter) {
         A.notNull(key, "key", val, "val");
 
@@ -2213,9 +2213,9 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         ctx.denyOnLocalRead();
 
         return ctx.wrapClone(asyncOp(new AsyncOp<V>(key) {
-            @Override public IgniteFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), true, entry, ttl, filter)
-                    .chain((IgniteClosure<IgniteFuture<GridCacheReturn<V>>, V>)RET2VAL);
+                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, V>)RET2VAL);
             }
 
             @Override public String toString() {
@@ -2279,7 +2279,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllDrAsync(final Map<? extends K, GridCacheDrInfo<V>> drMap)
+    @Override public IgniteInternalFuture<?> putAllDrAsync(final Map<? extends K, GridCacheDrInfo<V>> drMap)
         throws IgniteCheckedException {
         if (F.isEmpty(drMap))
             return new GridFinishedFuture<Object>(ctx.kernalContext());
@@ -2289,7 +2289,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         ctx.denyOnLocalRead();
 
         return asyncOp(new AsyncInOp(drMap.keySet()) {
-            @Override public IgniteFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.putAllDrAsync(ctx, drMap);
             }
 
@@ -2317,7 +2317,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap =
                     Collections.singletonMap(key, (EntryProcessor<K, V, Object>)entryProcessor);
 
-                IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut =
+                IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut =
                     tx.invokeAsync(ctx, invokeMap, args);
 
                 Map<K, EntryProcessorResult<T>> resMap = fut.get().value();
@@ -2353,7 +2353,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                     }
                 });
 
-                IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut =
+                IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut =
                     tx.invokeAsync(ctx, invokeMap, args);
 
                 Map<K, EntryProcessorResult<T>> res = fut.get().value();
@@ -2364,7 +2364,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<EntryProcessorResult<T>> invokeAsync(
+    @Override public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(
         final K key,
         final EntryProcessor<K, V, T> entryProcessor,
         final Object... args)
@@ -2376,8 +2376,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         ctx.denyOnLocalRead();
 
-        IgniteFuture<?> fut = asyncOp(new AsyncInOp(key) {
-            @Override public IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> inOp(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<?> fut = asyncOp(new AsyncInOp(key) {
+            @Override public IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> inOp(IgniteTxLocalAdapter<K, V> tx) {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap =
                     Collections.singletonMap(key, (EntryProcessor<K, V, Object>)entryProcessor);
 
@@ -2389,11 +2389,11 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             }
         });
 
-        IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut0 =
-            (IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>)fut;
+        IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut0 =
+            (IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>)fut;
 
-        return fut0.chain(new CX1<IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>, EntryProcessorResult<T>>() {
-            @Override public EntryProcessorResult<T> applyx(IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut)
+        return fut0.chain(new CX1<IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>, EntryProcessorResult<T>>() {
+            @Override public EntryProcessorResult<T> applyx(IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut)
                 throws IgniteCheckedException {
                 GridCacheReturn<Map<K, EntryProcessorResult<T>>> ret = fut.get();
 
@@ -2411,7 +2411,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         final Set<? extends K> keys,
         final EntryProcessor<K, V, T> entryProcessor,
         final Object... args) {
@@ -2422,8 +2422,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         ctx.denyOnLocalRead();
 
-        IgniteFuture<?> fut = asyncOp(new AsyncInOp(keys) {
-            @Override public IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> inOp(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<?> fut = asyncOp(new AsyncInOp(keys) {
+            @Override public IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> inOp(IgniteTxLocalAdapter<K, V> tx) {
                 Map<? extends K, EntryProcessor<K, V, Object>> invokeMap = F.viewAsMap(keys, new C1<K, EntryProcessor<K, V, Object>>() {
                     @Override public EntryProcessor apply(K k) {
                         return entryProcessor;
@@ -2438,11 +2438,11 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             }
         });
 
-        IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut0 =
-            (IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>)fut;
+        IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut0 =
+            (IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>)fut;
 
-        return fut0.chain(new CX1<IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>, Map<K, EntryProcessorResult<T>>>() {
-            @Override public Map<K, EntryProcessorResult<T>> applyx(IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut)
+        return fut0.chain(new CX1<IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>, Map<K, EntryProcessorResult<T>>>() {
+            @Override public Map<K, EntryProcessorResult<T>> applyx(IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut)
                 throws IgniteCheckedException {
                 GridCacheReturn<Map<K, EntryProcessorResult<T>>> ret = fut.get();
 
@@ -2454,7 +2454,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         final Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         final Object... args) {
         A.notNull(map, "map");
@@ -2464,8 +2464,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         ctx.denyOnLocalRead();
 
-        IgniteFuture<?> fut = asyncOp(new AsyncInOp(map.keySet()) {
-            @Override public IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> inOp(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<?> fut = asyncOp(new AsyncInOp(map.keySet()) {
+            @Override public IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> inOp(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.invokeAsync(ctx, (Map<? extends K, ? extends EntryProcessor<K, V, Object>>)map, args);
             }
 
@@ -2474,12 +2474,12 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             }
         });
 
-        IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut0 =
-            (IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>)fut;
+        IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut0 =
+            (IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>)fut;
 
-        return fut0.chain(new CX1<IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>, Map<K, EntryProcessorResult<T>>>() {
+        return fut0.chain(new CX1<IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>>, Map<K, EntryProcessorResult<T>>>() {
             @Override public Map<K, EntryProcessorResult<T>> applyx(
-                IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut)
+                IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut)
                 throws IgniteCheckedException {
                 GridCacheReturn<Map<K, EntryProcessorResult<T>>> ret = fut.get();
 
@@ -2504,7 +2504,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         return syncOp(new SyncOp<Map<K, EntryProcessorResult<T>>>(map.size() == 1) {
             @Nullable @Override public Map<K, EntryProcessorResult<T>> op(IgniteTxLocalAdapter tx)
                 throws IgniteCheckedException {
-                IgniteFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut = tx.invokeAsync(ctx, map, args);
+                IgniteInternalFuture<GridCacheReturn<Map<K, EntryProcessorResult<T>>>> fut = tx.invokeAsync(ctx, map, args);
 
                 return fut.get().value();
             }
@@ -2512,13 +2512,13 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxAsync(K key, V val,
+    @Override public IgniteInternalFuture<Boolean> putxAsync(K key, V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
-        IgniteFuture<Boolean> fut = putxAsync(key, val, null, -1, filter);
+        IgniteInternalFuture<Boolean> fut = putxAsync(key, val, null, -1, filter);
 
         if (statsEnabled)
             fut.listenAsync(new UpdatePutTimeStatClosure<Boolean>(metrics0(), start));
@@ -2527,7 +2527,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxAsync(final K key, final V val,
+    @Override public IgniteInternalFuture<Boolean> putxAsync(final K key, final V val,
         @Nullable final GridCacheEntryEx<K, V> entry, final long ttl,
         @Nullable final IgnitePredicate<CacheEntry<K, V>>... filter) {
         A.notNull(key, "key", val, "val");
@@ -2540,9 +2540,9 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         ctx.denyOnLocalRead();
 
         return asyncOp(new AsyncOp<Boolean>(key) {
-            @Override public IgniteFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), false, entry, ttl, filter).chain(
-                    (IgniteClosure<IgniteFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
             }
 
             @Override public String toString() {
@@ -2574,7 +2574,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putIfAbsentAsync(final K key, final V val) {
+    @Override public IgniteInternalFuture<V> putIfAbsentAsync(final K key, final V val) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -2588,10 +2588,10 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         ctx.denyOnLocalRead();
 
-        IgniteFuture<V> fut = ctx.wrapClone(asyncOp(new AsyncOp<V>(key) {
-            @Override public IgniteFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<V> fut = ctx.wrapClone(asyncOp(new AsyncOp<V>(key) {
+            @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), true, null, -1, ctx.noPeekArray())
-                    .chain((IgniteClosure<IgniteFuture<GridCacheReturn<V>>, V>)RET2VAL);
+                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, V>)RET2VAL);
             }
 
             @Override public String toString() {
@@ -2637,7 +2637,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxIfAbsentAsync(final K key, final V val) {
+    @Override public IgniteInternalFuture<Boolean> putxIfAbsentAsync(final K key, final V val) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -2651,10 +2651,10 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         ctx.denyOnLocalRead();
 
-        IgniteFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>(key) {
-            @Override public IgniteFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>(key) {
+            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), false, null, -1, ctx.noPeekArray()).chain(
-                    (IgniteClosure<IgniteFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
             }
 
             @Override public String toString() {
@@ -2691,7 +2691,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(final K key, final V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(final K key, final V val) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -2705,10 +2705,10 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         ctx.denyOnLocalRead();
 
-        IgniteFuture<V> fut = ctx.wrapClone(asyncOp(new AsyncOp<V>(key) {
-            @Override public IgniteFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<V> fut = ctx.wrapClone(asyncOp(new AsyncOp<V>(key) {
+            @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), true, null, -1, ctx.hasPeekArray()).chain(
-                    (IgniteClosure<IgniteFuture<GridCacheReturn<V>>, V>)RET2VAL);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, V>)RET2VAL);
             }
 
             @Override public String toString() {
@@ -2745,7 +2745,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(final K key, final V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(final K key, final V val) {
         A.notNull(key, "key", val, "val");
 
         if (keyCheck)
@@ -2756,9 +2756,9 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         ctx.denyOnLocalRead();
 
         return asyncOp(new AsyncOp<Boolean>(key) {
-            @Override public IgniteFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), false, null, -1, ctx.hasPeekArray()).chain(
-                    (IgniteClosure<IgniteFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
             }
 
             @Override public String toString() {
@@ -2797,7 +2797,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(final K key, final V oldVal, final V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(final K key, final V oldVal, final V newVal) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -2813,8 +2813,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         ctx.denyOnLocalRead();
 
-        IgniteFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>(key) {
-            @Override public IgniteFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>(key) {
+            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
                 // Register before hiding in the filter.
                 if (ctx.deploymentEnabled()) {
                     try {
@@ -2826,7 +2826,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                 }
 
                 return tx.putAllAsync(ctx, F.t(key, newVal), false, null, -1, ctx.equalsPeekArray(oldVal)).chain(
-                    (IgniteClosure<IgniteFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
             }
 
             @Override public String toString() {
@@ -2872,7 +2872,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllAsync(final Map<? extends K, ? extends V> m,
+    @Override public IgniteInternalFuture<?> putAllAsync(final Map<? extends K, ? extends V> m,
         @Nullable final IgnitePredicate<CacheEntry<K, V>>... filter) {
         if (F.isEmpty(m))
             return new GridFinishedFuture<Object>(ctx.kernalContext());
@@ -2885,7 +2885,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         ctx.denyOnLocalRead();
 
         return asyncOp(new AsyncInOp(m.keySet()) {
-            @Override public IgniteFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.putAllAsync(ctx, m, false, null, -1, filter);
             }
 
@@ -2937,12 +2937,12 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(K key, IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<V> removeAsync(K key, IgnitePredicate<CacheEntry<K, V>>... filter) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
 
-        IgniteFuture<V> fut = removeAsync(key, null, filter);
+        IgniteInternalFuture<V> fut = removeAsync(key, null, filter);
 
         if (statsEnabled)
             fut.listenAsync(new UpdateRemoveTimeStatClosure<V>(metrics0(), start));
@@ -2951,7 +2951,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(final K key, @Nullable final GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<V> removeAsync(final K key, @Nullable final GridCacheEntryEx<K, V> entry,
         @Nullable final IgnitePredicate<CacheEntry<K, V>>... filter) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
@@ -2964,11 +2964,11 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         if (keyCheck)
             validateCacheKey(key);
 
-        IgniteFuture<V> fut = ctx.wrapClone(asyncOp(new AsyncOp<V>(key) {
-            @Override public IgniteFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<V> fut = ctx.wrapClone(asyncOp(new AsyncOp<V>(key) {
+            @Override public IgniteInternalFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
                 // TODO should we invoke interceptor here?
                 return tx.removeAllAsync(ctx, Collections.singletonList(key), null, true, filter)
-                    .chain((IgniteClosure<IgniteFuture<GridCacheReturn<V>>, V>) RET2VAL);
+                    .chain((IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, V>) RET2VAL);
             }
 
             @Override public String toString() {
@@ -3025,7 +3025,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(@Nullable final Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<?> removeAllAsync(@Nullable final Collection<? extends K> keys,
         final IgnitePredicate<CacheEntry<K, V>>... filter) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
@@ -3039,8 +3039,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         ctx.denyOnLocalRead();
 
-        IgniteFuture<Object> fut = asyncOp(new AsyncInOp(keys) {
-            @Override public IgniteFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<Object> fut = asyncOp(new AsyncInOp(keys) {
+            @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.removeAllAsync(ctx, keys, null, false, filter);
             }
 
@@ -3101,14 +3101,14 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(K key, IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<Boolean> removexAsync(K key, IgnitePredicate<CacheEntry<K, V>>... filter) {
         A.notNull(key, "key");
 
         return removexAsync(key, null, filter);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(final K key, @Nullable final GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<Boolean> removexAsync(final K key, @Nullable final GridCacheEntryEx<K, V> entry,
         @Nullable final IgnitePredicate<CacheEntry<K, V>>... filter) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
@@ -3121,10 +3121,10 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         if (keyCheck)
             validateCacheKey(key);
 
-        IgniteFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>(key) {
-            @Override public IgniteFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>(key) {
+            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.removeAllAsync(ctx, Collections.singletonList(key), entry, false, filter).chain(
-                    (IgniteClosure<IgniteFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
             }
 
             @Override public String toString() {
@@ -3184,7 +3184,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllDrAsync(final Map<? extends K, GridCacheVersion> drMap)
+    @Override public IgniteInternalFuture<?> removeAllDrAsync(final Map<? extends K, GridCacheVersion> drMap)
         throws IgniteCheckedException {
         ctx.denyOnLocalRead();
 
@@ -3194,7 +3194,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         ctx.dr().onReceiveCacheEntriesReceived(drMap.size());
 
         return asyncOp(new AsyncInOp(drMap.keySet()) {
-            @Override public IgniteFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.removeAllDrAsync(ctx, drMap);
             }
 
@@ -3229,7 +3229,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(final K key, final V val) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> removexAsync(final K key, final V val) {
         ctx.denyOnLocalRead();
 
         A.notNull(key, "key", val, "val");
@@ -3238,7 +3238,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             validateCacheKey(key);
 
         return asyncOp(new AsyncOp<GridCacheReturn<V>>(key) {
-            @Override public IgniteFuture<GridCacheReturn<V>> op(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteInternalFuture<GridCacheReturn<V>> op(IgniteTxLocalAdapter<K, V> tx) {
                 // Register before hiding in the filter.
                 try {
                     if (ctx.deploymentEnabled())
@@ -3259,7 +3259,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheReturn<V>> replacexAsync(final K key, final V oldVal, final V newVal) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> replacexAsync(final K key, final V oldVal, final V newVal) {
         A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
 
         if (keyCheck)
@@ -3268,7 +3268,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         ctx.denyOnLocalRead();
 
         return asyncOp(new AsyncOp<GridCacheReturn<V>>(key) {
-            @Override public IgniteFuture<GridCacheReturn<V>> op(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteInternalFuture<GridCacheReturn<V>> op(IgniteTxLocalAdapter<K, V> tx) {
                 // Register before hiding in the filter.
                 try {
                     if (ctx.deploymentEnabled())
@@ -3329,7 +3329,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(final K key, final V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(final K key, final V val) {
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -3343,8 +3343,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         validateCacheValue(val);
 
-        IgniteFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>(key) {
-            @Override public IgniteFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
+        IgniteInternalFuture<Boolean> fut = asyncOp(new AsyncOp<Boolean>(key) {
+            @Override public IgniteInternalFuture<Boolean> op(IgniteTxLocalAdapter<K, V> tx) {
                 // Register before hiding in the filter.
                 if (ctx.deploymentEnabled()) {
                     try {
@@ -3368,7 +3368,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
                 return tx.removeAllAsync(ctx, Collections.singletonList(key0), null, false,
                     ctx.vararg(F.<K, V>cacheContainsPeek(val))).chain(
-                    (IgniteClosure<IgniteFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
+                    (IgniteClosure<IgniteInternalFuture<GridCacheReturn<V>>, Boolean>)RET2FLAG);
             }
 
             @Override public String toString() {
@@ -3403,13 +3403,13 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(final IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<?> removeAllAsync(final IgnitePredicate<CacheEntry<K, V>>... filter) {
         ctx.denyOnLocalRead();
 
         final Set<? extends K> keys = keySet(filter);
 
         return asyncOp(new AsyncInOp(keys) {
-            @Override public IgniteFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.removeAllAsync(ctx, keys, null, false, null);
             }
 
@@ -3460,7 +3460,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         if (keyCheck)
             validateCacheKeys(keys);
 
-        IgniteFuture<Boolean> fut = lockAllAsync(keys, timeout, filter);
+        IgniteInternalFuture<Boolean> fut = lockAllAsync(keys, timeout, filter);
 
         boolean isInterrupted = false;
 
@@ -3482,7 +3482,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAsync(K key, long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAsync(K key, long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         A.notNull(key, "key");
 
@@ -3690,7 +3690,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> loadCacheAsync(final IgniteBiPredicate<K, V> p, final long ttl, final Object[] args) {
+    @Override public IgniteInternalFuture<?> loadCacheAsync(final IgniteBiPredicate<K, V> p, final long ttl, final Object[] args) {
         return ctx.closures().callLocalSafe(
             ctx.projectSafe(new Callable<Object>() {
                 @Nullable @Override public Object call() throws IgniteCheckedException {
@@ -3706,7 +3706,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param replaceExisting Replace existing values flag.
      * @return Load future.
      */
-    public IgniteFuture<?> loadAll(
+    public IgniteInternalFuture<?> loadAll(
         final Set<? extends K> keys,
         boolean replaceExisting
     ) {
@@ -3831,7 +3831,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param args Arguments.
      * @throws IgniteCheckedException If failed.
      */
-    IgniteFuture<?> globalLoadCacheAsync(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args)
+    IgniteInternalFuture<?> globalLoadCacheAsync(@Nullable IgniteBiPredicate<K, V> p, @Nullable Object... args)
         throws IgniteCheckedException {
         ClusterGroup nodes = ctx.kernalContext().grid().cluster().forCache(ctx.name());
 
@@ -4043,18 +4043,18 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @return Transaction commit future.
      */
     @SuppressWarnings("unchecked")
-    public IgniteFuture<IgniteTx> commitTxAsync(final IgniteTxEx tx) {
+    public IgniteInternalFuture<IgniteTx> commitTxAsync(final IgniteTxEx tx) {
         FutureHolder holder = lastFut.get();
 
         holder.lock();
 
         try {
-            IgniteFuture fut = holder.future();
+            IgniteInternalFuture fut = holder.future();
 
             if (fut != null && !fut.isDone()) {
-                IgniteFuture<IgniteTx> f = new GridEmbeddedFuture<>(fut,
-                    new C2<Object, Exception, IgniteFuture<IgniteTx>>() {
-                        @Override public IgniteFuture<IgniteTx> apply(Object o, Exception e) {
+                IgniteInternalFuture<IgniteTx> f = new GridEmbeddedFuture<>(fut,
+                    new C2<Object, Exception, IgniteInternalFuture<IgniteTx>>() {
+                        @Override public IgniteInternalFuture<IgniteTx> apply(Object o, Exception e) {
                             return tx.commitAsync();
                         }
                     }, ctx.kernalContext());
@@ -4064,7 +4064,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                 return f;
             }
 
-            IgniteFuture<IgniteTx> f = tx.commitAsync();
+            IgniteInternalFuture<IgniteTx> f = tx.commitAsync();
 
             saveFuture(holder, f);
 
@@ -4120,7 +4120,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     public void awaitLastFut() {
         FutureHolder holder = lastFut.get();
 
-        IgniteFuture fut = holder.future();
+        IgniteInternalFuture fut = holder.future();
 
         if (fut != null && !fut.isDone()) {
             try {
@@ -4145,7 +4145,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             // Send job to remote nodes only.
             Collection<ClusterNode> nodes = ctx.grid().forCache(name()).forRemotes().nodes();
 
-            IgniteFuture<Collection<Integer>> fut = null;
+            IgniteInternalFuture<Collection<Integer>> fut = null;
 
             if (!nodes.isEmpty()) {
                 ctx.kernalContext().task().setThreadContext(TC_TIMEOUT, gridCfg.getNetworkTimeout());
@@ -4255,7 +4255,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    private <T> IgniteFuture<T> asyncOp(final AsyncOp<T> op) {
+    private <T> IgniteInternalFuture<T> asyncOp(final AsyncOp<T> op) {
         try {
             checkJta();
         }
@@ -4296,8 +4296,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    protected <T> IgniteFuture<T> asyncOp(IgniteTxLocalAdapter<K, V> tx, final AsyncOp<T> op) {
-        IgniteFuture<T> fail = asyncOpAcquire();
+    protected <T> IgniteInternalFuture<T> asyncOp(IgniteTxLocalAdapter<K, V> tx, final AsyncOp<T> op) {
+        IgniteInternalFuture<T> fail = asyncOpAcquire();
 
         if (fail != null)
             return fail;
@@ -4307,14 +4307,14 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         holder.lock();
 
         try {
-            IgniteFuture fut = holder.future();
+            IgniteInternalFuture fut = holder.future();
 
             if (fut != null && !fut.isDone()) {
                 final IgniteTxLocalAdapter<K, V> tx0 = tx;
 
-                IgniteFuture<T> f = new GridEmbeddedFuture<>(fut,
-                    new C2<T, Exception, IgniteFuture<T>>() {
-                        @Override public IgniteFuture<T> apply(T t, Exception e) {
+                IgniteInternalFuture<T> f = new GridEmbeddedFuture<>(fut,
+                    new C2<T, Exception, IgniteInternalFuture<T>>() {
+                        @Override public IgniteInternalFuture<T> apply(T t, Exception e) {
                             return op.op(tx0);
                         }
                     }, ctx.kernalContext());
@@ -4324,7 +4324,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                 return f;
             }
 
-            IgniteFuture<T> f = op.op(tx);
+            IgniteInternalFuture<T> f = op.op(tx);
 
             saveFuture(holder, f);
 
@@ -4345,7 +4345,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param holder Future holder.
      * @param fut Future to save.
      */
-    protected void saveFuture(final FutureHolder holder, IgniteFuture<?> fut) {
+    protected void saveFuture(final FutureHolder holder, IgniteInternalFuture<?> fut) {
         assert holder != null;
         assert fut != null;
         assert holder.holdsLock();
@@ -4358,8 +4358,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             asyncOpRelease();
         }
         else {
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> f) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> f) {
                     asyncOpRelease();
 
                     if (!holder.tryLock())
@@ -4382,7 +4382,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      *
      * @return Failed future if waiting was interrupted.
      */
-    @Nullable protected <T> IgniteFuture<T> asyncOpAcquire() {
+    @Nullable protected <T> IgniteInternalFuture<T> asyncOpAcquire() {
         try {
             if (asyncOpsSem != null)
                 asyncOpsSem.acquire();
@@ -4441,7 +4441,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> forceRepartition() {
+    @Override public IgniteInternalFuture<?> forceRepartition() {
         ctx.preloader().forcePreload();
 
         return ctx.preloader().syncFuture();
@@ -4829,7 +4829,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param filter Filter to evaluate.
      * @return Read operation future.
      */
-    public final IgniteFuture<V> getAsync(final K key, boolean deserializePortable,
+    public final IgniteInternalFuture<V> getAsync(final K key, boolean deserializePortable,
         @Nullable IgnitePredicate<CacheEntry<K, V>> filter) {
         ctx.denyOnFlag(LOCAL);
 
@@ -4840,9 +4840,9 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
             return new GridFinishedFuture<>(ctx.kernalContext(), e);
         }
 
-        return getAllAsync(Collections.singletonList(key), deserializePortable, filter).chain(new CX1<IgniteFuture<Map<K, V>>, V>() {
+        return getAllAsync(Collections.singletonList(key), deserializePortable, filter).chain(new CX1<IgniteInternalFuture<Map<K, V>>, V>() {
             @Override
-            public V applyx(IgniteFuture<Map<K, V>> e) throws IgniteCheckedException {
+            public V applyx(IgniteInternalFuture<Map<K, V>> e) throws IgniteCheckedException {
                 return e.get().get(key);
             }
         });
@@ -4913,7 +4913,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param filter Filter to evaluate.
      * @return Reload future.
      */
-    public IgniteFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys,
+    public IgniteInternalFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         UUID subjId = ctx.subjectIdPerCall(null);
 
@@ -4927,7 +4927,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param filter Filter to evaluate.
      * @return Reload future.
      */
-    public IgniteFuture<V> reloadAsync(final K key,
+    public IgniteInternalFuture<V> reloadAsync(final K key,
         @Nullable final IgnitePredicate<CacheEntry<K, V>>... filter) {
         ctx.denyOnFlags(F.asList(LOCAL, READ));
 
@@ -4956,7 +4956,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param filter Filter to evaluate.
      * @return Reload future.
      */
-    public IgniteFuture<?> reloadAllAsync(@Nullable final IgnitePredicate<CacheEntry<K, V>> filter) {
+    public IgniteInternalFuture<?> reloadAllAsync(@Nullable final IgnitePredicate<CacheEntry<K, V>> filter) {
         ctx.denyOnFlag(READ);
 
         return ctx.closures().callLocalSafe(ctx.projectSafe(new GPC() {
@@ -4974,7 +4974,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      * @param filter Filter to evaluate.
      * @return Read future.
      */
-    public IgniteFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys,
+    public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys,
         boolean deserializePortable,
         @Nullable IgnitePredicate<CacheEntry<K, V>> filter) {
         String taskName = ctx.kernalContext().job().currentTaskName();
@@ -5183,7 +5183,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
          * @param tx Transaction.
          * @return Operation return value.
          */
-        public abstract IgniteFuture<T> op(IgniteTxLocalAdapter<K, V> tx);
+        public abstract IgniteInternalFuture<T> op(IgniteTxLocalAdapter<K, V> tx);
     }
 
     /**
@@ -5206,15 +5206,15 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         /** {@inheritDoc} */
         @SuppressWarnings({"unchecked"})
-        @Override public final IgniteFuture<Object> op(IgniteTxLocalAdapter<K, V> tx) {
-            return (IgniteFuture<Object>)inOp(tx);
+        @Override public final IgniteInternalFuture<Object> op(IgniteTxLocalAdapter<K, V> tx) {
+            return (IgniteInternalFuture<Object>)inOp(tx);
         }
 
         /**
          * @param tx Transaction.
          * @return Operation return value.
          */
-        public abstract IgniteFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx);
+        public abstract IgniteInternalFuture<?> inOp(IgniteTxLocalAdapter<K, V> tx);
     }
 
     /**
@@ -5328,7 +5328,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         private final ReentrantLock lock = new ReentrantLock();
 
         /** Future. */
-        private IgniteFuture fut;
+        private IgniteInternalFuture fut;
 
         /**
          * Tries to acquire lock.
@@ -5366,7 +5366,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
          *
          * @return Future.
          */
-        public IgniteFuture future() {
+        public IgniteInternalFuture future() {
             return fut;
         }
 
@@ -5375,7 +5375,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
          *
          * @param fut Future.
          */
-        public void future(@Nullable IgniteFuture fut) {
+        public void future(@Nullable IgniteInternalFuture fut) {
             this.fut = fut;
         }
     }
@@ -5707,7 +5707,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     /**
      *
      */
-    protected static abstract class UpdateTimeStatClosure<T> implements CI1<IgniteFuture<T>> {
+    protected static abstract class UpdateTimeStatClosure<T> implements CI1<IgniteInternalFuture<T>> {
         /** */
         protected final CacheMetricsImpl metrics;
 
@@ -5724,7 +5724,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         }
 
         /** {@inheritDoc} */
-        @Override public void apply(IgniteFuture<T> fut) {
+        @Override public void apply(IgniteInternalFuture<T> fut) {
             try {
                 if (!fut.isCancelled()) {
                     fut.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
index d1c1a70..5d02248 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityManager.java
@@ -20,8 +20,8 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.affinity.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -98,10 +98,10 @@ public class GridCacheAffinityManager<K, V> extends GridCacheManagerAdapter<K, V
      * @param topVer Topology version to wait.
      * @return Affinity ready future.
      */
-    public IgniteFuture<Long> affinityReadyFuture(long topVer) {
+    public IgniteInternalFuture<Long> affinityReadyFuture(long topVer) {
         assert !cctx.isLocal();
 
-        IgniteFuture<Long> fut = aff.readyFuture(topVer);
+        IgniteInternalFuture<Long> fut = aff.readyFuture(topVer);
 
         return fut != null ? fut : new GridFinishedFutureEx<>(topVer);
     }
@@ -113,7 +113,7 @@ public class GridCacheAffinityManager<K, V> extends GridCacheManagerAdapter<K, V
      * @param topVer Topology version to wait.
      * @return Affinity ready future or {@code null}.
      */
-    @Nullable public IgniteFuture<Long> affinityReadyFuturex(long topVer) {
+    @Nullable public IgniteInternalFuture<Long> affinityReadyFuturex(long topVer) {
         assert !cctx.isLocal();
 
         return aff.readyFuture(topVer);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 3fb5329..0336832 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -1278,12 +1278,12 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @param f Target future.
      * @return Wrapped future that is aware of cloning behaviour.
      */
-    public IgniteFuture<V> wrapClone(IgniteFuture<V> f) {
+    public IgniteInternalFuture<V> wrapClone(IgniteInternalFuture<V> f) {
         if (!hasFlag(CLONE))
             return f;
 
-        return f.chain(new CX1<IgniteFuture<V>, V>() {
-            @Override public V applyx(IgniteFuture<V> f) throws IgniteCheckedException {
+        return f.chain(new CX1<IgniteInternalFuture<V>, V>() {
+            @Override public V applyx(IgniteInternalFuture<V> f) throws IgniteCheckedException {
                 return cloneValue(f.get());
             }
         });
@@ -1293,12 +1293,12 @@ public class GridCacheContext<K, V> implements Externalizable {
      * @param f Target future.
      * @return Wrapped future that is aware of cloning behaviour.
      */
-    public IgniteFuture<Map<K, V>> wrapCloneMap(IgniteFuture<Map<K, V>> f) {
+    public IgniteInternalFuture<Map<K, V>> wrapCloneMap(IgniteInternalFuture<Map<K, V>> f) {
         if (!hasFlag(CLONE))
             return f;
 
-        return f.chain(new CX1<IgniteFuture<Map<K, V>>, Map<K, V>>() {
-            @Override public Map<K, V> applyx(IgniteFuture<Map<K, V>> f) throws IgniteCheckedException {
+        return f.chain(new CX1<IgniteInternalFuture<Map<K, V>>, Map<K, V>>() {
+            @Override public Map<K, V> applyx(IgniteInternalFuture<Map<K, V>> f) throws IgniteCheckedException {
                 Map<K, V> map = new GridLeanMap<>();
 
                 for (Map.Entry<K, V> e : f.get().entrySet())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.java
index 6d59e26..f018a8a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEntryImpl.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.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
@@ -377,7 +378,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> reloadAsync() {
+    @Override public IgniteInternalFuture<V> reloadAsync() {
         GridCacheProjectionImpl<K, V> old = ctx.gate().enter(proxy.gateProjection());
 
         try {
@@ -409,7 +410,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getAsync() {
+    @Override public IgniteInternalFuture<V> getAsync() {
         return proxy.getAsync(key);
     }
 
@@ -420,7 +421,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> setAsync(V val, IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<V> setAsync(V val, IgnitePredicate<CacheEntry<K, V>>[] filter) {
         // Should not pass dht entries as to near cache.
         return proxy.putAsync(key, val, isNearEnabled(ctx) ? null : cached, ttl, filter);
     }
@@ -432,7 +433,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> setxAsync(V val, IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<Boolean> setxAsync(V val, IgnitePredicate<CacheEntry<K, V>>[] filter) {
         // Should not pass dht entries as to near cache.
         return proxy.putxAsync(key, val, isNearEnabled(ctx) ? null : cached, ttl, filter);
     }
@@ -443,7 +444,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(V val) {
         return setAsync(val, ctx.hasPeekArray());
     }
 
@@ -453,7 +454,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(V oldVal, V newVal) {
         return setxAsync(newVal, ctx.equalsPeekArray(newVal));
     }
 
@@ -499,7 +500,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> setIfAbsentAsync(V val) {
+    @Override public IgniteInternalFuture<V> setIfAbsentAsync(V val) {
         return setAsync(val, ctx.noPeekArray());
     }
 
@@ -509,7 +510,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> setxIfAbsentAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> setxIfAbsentAsync(V val) {
         return setxAsync(val, ctx.noPeekArray());
     }
 
@@ -519,7 +520,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(V val) {
         return setxAsync(val, ctx.hasPeekArray());
     }
 
@@ -529,7 +530,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<V> removeAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return proxy.removeAsync(key, isNearEnabled(ctx) ? null : cached, filter);
     }
 
@@ -539,7 +540,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<Boolean> removexAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return proxy.removexAsync(key, isNearEnabled(ctx) ? null : cached, filter);
     }
 
@@ -549,7 +550,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(V val) {
         return proxy.removeAsync(key, val);
     }
 
@@ -649,7 +650,7 @@ public class GridCacheEntryImpl<K, V> implements CacheEntry<K, V>, Externalizabl
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAsync(long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAsync(long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return proxy.lockAsync(key, timeout, filter);
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.java
index f106337..20e6e74 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionEntry.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.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -212,7 +213,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> reloadAsync() {
+    @Override public IgniteInternalFuture<V> reloadAsync() {
         throw unsupported();
     }
 
@@ -249,7 +250,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getAsync() {
+    @Override public IgniteInternalFuture<V> getAsync() {
         throw unsupported();
     }
 
@@ -259,7 +260,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> setAsync(V val, IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<V> setAsync(V val, IgnitePredicate<CacheEntry<K, V>>[] filter) {
         throw unsupported();
     }
 
@@ -269,7 +270,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> setxAsync(V val, IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<Boolean> setxAsync(V val, IgnitePredicate<CacheEntry<K, V>>[] filter) {
         throw unsupported();
     }
 
@@ -279,7 +280,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(V val) {
         throw unsupported();
     }
 
@@ -289,7 +290,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(V oldVal, V newVal) {
         throw unsupported();
     }
 
@@ -309,7 +310,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> setIfAbsentAsync(V val) {
+    @Override public IgniteInternalFuture<V> setIfAbsentAsync(V val) {
         throw unsupported();
     }
 
@@ -319,7 +320,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> setxIfAbsentAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> setxIfAbsentAsync(V val) {
         throw unsupported();
     }
 
@@ -329,7 +330,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(V val) {
         throw unsupported();
     }
 
@@ -339,7 +340,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<V> removeAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
         throw unsupported();
     }
 
@@ -349,7 +350,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<Boolean> removexAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
         throw unsupported();
     }
 
@@ -359,7 +360,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(V val) {
         throw unsupported();
     }
 
@@ -417,7 +418,7 @@ public class GridCacheEvictionEntry<K, V> implements CacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAsync(long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAsync(long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         throw unsupported();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
index 1b44aca..837a6e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheEvictionManager.java
@@ -1067,8 +1067,8 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
                         // Thread that prepares future should remove it and install listener.
                         curEvictFut.compareAndSet(fut, null);
 
-                        fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                            @Override public void apply(IgniteFuture<?> f) {
+                        fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                            @Override public void apply(IgniteInternalFuture<?> f) {
                                 if (!busyLock.enterBusy()) {
                                     if (log.isDebugEnabled())
                                         log.debug("Will not notify eviction future completion (grid is stopping): " +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheExplicitLockSpan.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheExplicitLockSpan.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheExplicitLockSpan.java
index 2e8edfc..3c020a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheExplicitLockSpan.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheExplicitLockSpan.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.managers.discovery.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -243,7 +243,7 @@ public class GridCacheExplicitLockSpan<K> extends ReentrantLock {
      *
      * @return Release future.
      */
-    public IgniteFuture<Object> releaseFuture() {
+    public IgniteInternalFuture<Object> releaseFuture() {
         return releaseFut;
     }
 


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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
index 7c77dbc..d5e9714 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/preloader/GridDhtPreloader.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.dht.preloader;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
@@ -261,12 +262,12 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
     /**
      * @return Start future.
      */
-    @Override public IgniteFuture<Object> startFuture() {
+    @Override public IgniteInternalFuture<Object> startFuture() {
         return startFut;
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> syncFuture() {
+    @Override public IgniteInternalFuture<?> syncFuture() {
         return demandPool.syncFuture();
     }
 
@@ -315,13 +316,13 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
      * @param msg Force keys message.
      */
     private void processForceKeysRequest(final ClusterNode node, final GridDhtForceKeysRequest<K, V> msg) {
-        IgniteFuture<?> fut = cctx.mvcc().finishKeys(msg.keys(), msg.topologyVersion());
+        IgniteInternalFuture<?> fut = cctx.mvcc().finishKeys(msg.keys(), msg.topologyVersion());
 
         if (fut.isDone())
             processForceKeysRequest0(node, msg);
         else
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> t) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> t) {
                     processForceKeysRequest0(node, msg);
                 }
             });
@@ -426,8 +427,8 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
         if (log.isDebugEnabled())
             log.debug("Processing affinity assignment request [node=" + node + ", req=" + req + ']');
 
-        cctx.affinity().affinityReadyFuture(req.topologyVersion()).listenAsync(new CI1<IgniteFuture<Long>>() {
-            @Override public void apply(IgniteFuture<Long> fut) {
+        cctx.affinity().affinityReadyFuture(req.topologyVersion()).listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+            @Override public void apply(IgniteInternalFuture<Long> fut) {
                 if (log.isDebugEnabled())
                     log.debug("Affinity is ready for topology version, will send response [topVer=" + topVer +
                         ", node=" + node + ']');
@@ -481,27 +482,27 @@ public class GridDhtPreloader<K, V> extends GridCachePreloaderAdapter<K, V> {
     @Override public GridDhtFuture<Object> request(Collection<? extends K> keys, long topVer) {
         final GridDhtForceKeysFuture<K, V> fut = new GridDhtForceKeysFuture<>(cctx, topVer, keys, this);
 
-        IgniteFuture<?> topReadyFut = cctx.affinity().affinityReadyFuturex(topVer);
+        IgniteInternalFuture<?> topReadyFut = cctx.affinity().affinityReadyFuturex(topVer);
 
         if (startFut.isDone() && topReadyFut == null)
             fut.init();
         else {
             if (topReadyFut == null)
-                startFut.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> syncFut) {
+                startFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> syncFut) {
                         fut.init();
                     }
                 });
             else {
                 GridCompoundFuture<Object, Object> compound = new GridCompoundFuture<>(cctx.kernalContext());
 
-                compound.add((IgniteFuture<Object>)startFut);
-                compound.add((IgniteFuture<Object>)topReadyFut);
+                compound.add((IgniteInternalFuture<Object>)startFut);
+                compound.add((IgniteInternalFuture<Object>)topReadyFut);
 
                 compound.markInitialized();
 
-                compound.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> syncFut) {
+                compound.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> syncFut) {
                         fut.init();
                     }
                 });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
index 9ab8d86..5d8d96f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearAtomicCache.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -362,7 +363,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteFuture<Map<K, V>> getAllAsync(
+    @Override protected IgniteInternalFuture<Map<K, V>> getAllAsync(
         @Nullable Collection<? extends K> keys,
         boolean forcePrimary,
         boolean skipTx,
@@ -422,7 +423,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> putAsync(K key,
+    @Override public IgniteInternalFuture<V> putAsync(K key,
         V val,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl,
@@ -432,7 +433,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> putxAsync(K key,
+    @Override public IgniteInternalFuture<Boolean> putxAsync(K key,
         V val,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl,
@@ -446,7 +447,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> putIfAbsentAsync(K key, V val) {
         return dht.putIfAbsentAsync(key, val);
     }
 
@@ -456,7 +457,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> putxIfAbsentAsync(K key, V val) {
         return dht.putxIfAbsentAsync(key, val);
     }
 
@@ -466,7 +467,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(K key, V val) {
         return dht.replaceAsync(key, val);
     }
 
@@ -476,7 +477,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(K key, V val) {
         return dht.replacexAsync(key, val);
     }
 
@@ -486,7 +487,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
         return dht.replaceAsync(key, oldVal, newVal);
     }
 
@@ -502,13 +503,13 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
         return dht.removexAsync(key, val);
     }
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
         return dht.replacexAsync(key, oldVal, newVal);
     }
 
@@ -519,7 +520,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
+    @Override public IgniteInternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return dht.putAllAsync(m, filter);
     }
@@ -530,7 +531,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap) throws IgniteCheckedException {
         return dht.putAllDrAsync(drMap);
     }
 
@@ -549,14 +550,14 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args) {
         return dht.invokeAllAsync(map, args);
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<EntryProcessorResult<T>> invokeAsync(K key,
+    @Override public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) throws EntryProcessorException {
         return dht.invokeAsync(key, entryProcessor, args);
@@ -570,7 +571,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
         return dht.invokeAllAsync(keys, entryProcessor, args);
@@ -585,7 +586,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> removeAsync(K key,
+    @Override public IgniteInternalFuture<V> removeAsync(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         return dht.removeAsync(key, entry, filter);
@@ -598,7 +599,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<?> removeAllAsync(Collection<? extends K> keys,
         IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return dht.removeAllAsync(keys, filter);
     }
@@ -612,7 +613,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> removexAsync(K key,
+    @Override public IgniteInternalFuture<Boolean> removexAsync(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         return dht.removexAsync(key, entry, filter);
@@ -624,7 +625,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
         return dht.removeAsync(key, val);
     }
 
@@ -634,7 +635,7 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<?> removeAllAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return dht.removeAllAsync(keySet(filter));
     }
 
@@ -644,12 +645,12 @@ public class GridNearAtomicCache<K, V> extends GridNearCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException {
         return dht.removeAllDrAsync(drMap);
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
+    @Override protected IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
         long timeout,
         @Nullable IgniteTxLocalEx<K, V> tx,
         boolean isInvalidate,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
index 9fa97cd..7d7fb35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearCacheAdapter.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -172,10 +173,10 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "RedundantCast"})
-    @Override public IgniteFuture<Object> readThroughAllAsync(Collection<? extends K> keys, boolean reload,
+    @Override public IgniteInternalFuture<Object> readThroughAllAsync(Collection<? extends K> keys, boolean reload,
         IgniteTxEx<K, V> tx, IgnitePredicate<CacheEntry<K, V>>[] filter, @Nullable UUID subjId, String taskName,
         IgniteBiInClosure<K, V> vis) {
-        return (IgniteFuture)loadAsync(tx,
+        return (IgniteInternalFuture)loadAsync(tx,
             keys,
             reload,
             false,
@@ -196,7 +197,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         GridCompoundFuture fut = new GridCompoundFuture(ctx.kernalContext());
 
@@ -235,7 +236,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked"})
-    @Override public IgniteFuture<?> reloadAllAsync() {
+    @Override public IgniteInternalFuture<?> reloadAllAsync() {
         GridCompoundFuture fut = new GridCompoundFuture(ctx.kernalContext());
 
         fut.add(super.reloadAllAsync());
@@ -248,7 +249,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked"})
-    @Override public IgniteFuture<?> reloadAllAsync(@Nullable IgnitePredicate<CacheEntry<K, V>> filter) {
+    @Override public IgniteInternalFuture<?> reloadAllAsync(@Nullable IgnitePredicate<CacheEntry<K, V>> filter) {
         GridCompoundFuture fut = new GridCompoundFuture(ctx.kernalContext());
 
         fut.add(super.reloadAllAsync());
@@ -271,7 +272,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
      * @param expiryPlc Expiry policy.
      * @return Loaded values.
      */
-    public IgniteFuture<Map<K, V>> loadAsync(@Nullable IgniteTxEx tx,
+    public IgniteInternalFuture<Map<K, V>> loadAsync(@Nullable IgniteTxEx tx,
         @Nullable Collection<? extends K> keys,
         boolean reload,
         boolean forcePrimary,
@@ -319,7 +320,7 @@ public abstract class GridNearCacheAdapter<K, V> extends GridDistributedCacheAda
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> loadCacheAsync(IgniteBiPredicate<K, V> p, long ttl, Object[] args) {
+    @Override public IgniteInternalFuture<?> loadCacheAsync(IgniteBiPredicate<K, V> p, long ttl, Object[] args) {
         return dht().loadCacheAsync(p, ttl, args);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 7eb5013..bdb6e7a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -204,8 +205,8 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
     /** {@inheritDoc} */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<Map<K, V>>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<Map<K, V>> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<Map<K, V>>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(IgniteInternalFuture<Map<K, V>> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -216,7 +217,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        for (IgniteFuture<Map<K, V>> fut : futures())
+        for (IgniteInternalFuture<Map<K, V>> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -235,7 +236,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
      * @param res Result.
      */
     void onResult(UUID nodeId, GridNearGetResponse<K, V> res) {
-        for (IgniteFuture<Map<K, V>> fut : futures())
+        for (IgniteInternalFuture<Map<K, V>> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -266,7 +267,7 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<Map<K, V>> f) {
+    private boolean isMini(IgniteInternalFuture<Map<K, V>> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -357,8 +358,8 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                 }
 
                 // Add new future.
-                add(fut.chain(new C1<IgniteFuture<Collection<GridCacheEntryInfo<K, V>>>, Map<K, V>>() {
-                    @Override public Map<K, V> apply(IgniteFuture<Collection<GridCacheEntryInfo<K, V>>> fut) {
+                add(fut.chain(new C1<IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>>, Map<K, V>>() {
+                    @Override public Map<K, V> apply(IgniteInternalFuture<Collection<GridCacheEntryInfo<K, V>>> fut) {
                         try {
                             return loadEntries(n.id(), mappedKeys.keySet(), fut.get(), saved, topVer);
                         }
@@ -756,8 +757,8 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
             else {
                 final RemapTimeoutObject timeout = new RemapTimeoutObject(ctx.config().getNetworkTimeout(), topVer, e);
 
-                ctx.discovery().topologyFuture(topVer + 1).listenAsync(new CI1<IgniteFuture<Long>>() {
-                    @Override public void apply(IgniteFuture<Long> longIgniteFuture) {
+                ctx.discovery().topologyFuture(topVer + 1).listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                    @Override public void apply(IgniteInternalFuture<Long> longIgniteFuture) {
                         if (timeout.finish()) {
                             ctx.timeout().removeTimeoutObject(timeout);
 
@@ -806,10 +807,10 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
                     log.debug("Remapping mini get future [invalidParts=" + invalidParts + ", fut=" + this + ']');
 
                 // Need to wait for next topology version to remap.
-                IgniteFuture<Long> topFut = ctx.discovery().topologyFuture(rmtTopVer);
+                IgniteInternalFuture<Long> topFut = ctx.discovery().topologyFuture(rmtTopVer);
 
-                topFut.listenAsync(new CIX1<IgniteFuture<Long>>() {
-                    @Override public void applyx(IgniteFuture<Long> fut) throws IgniteCheckedException {
+                topFut.listenAsync(new CIX1<IgniteInternalFuture<Long>>() {
+                    @Override public void applyx(IgniteInternalFuture<Long> fut) throws IgniteCheckedException {
                         long readyTopVer = fut.get();
 
                         // This will append new futures to compound list.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
index e34f775..d1b56e5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearLockFuture.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -186,8 +187,8 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
      */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<?>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(IgniteInternalFuture<?> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -415,7 +416,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
     @Override public boolean onNodeLeft(UUID nodeId) {
         boolean found = false;
 
-        for (IgniteFuture<?> fut : futures()) {
+        for (IgniteInternalFuture<?> fut : futures()) {
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -449,7 +450,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
             if (log.isDebugEnabled())
                 log.debug("Received lock response from node [nodeId=" + nodeId + ", res=" + res + ", fut=" + this + ']');
 
-            for (IgniteFuture<Boolean> fut : pending()) {
+            for (IgniteInternalFuture<Boolean> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture mini = (MiniFuture)fut;
 
@@ -640,7 +641,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -700,8 +701,8 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
                     markInitialized();
                 }
                 else {
-                    fut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                        @Override public void apply(IgniteFuture<Long> t) {
+                    fut.listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                        @Override public void apply(IgniteInternalFuture<Long> t) {
                             mapOnTopology();
                         }
                     });
@@ -968,7 +969,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
             if (log.isDebugEnabled())
                 log.debug("Before locally locking near request: " + req);
 
-            IgniteFuture<GridNearLockResponse<K, V>> fut = dht().lockAllAsync(cctx, cctx.localNode(), req, filter);
+            IgniteInternalFuture<GridNearLockResponse<K, V>> fut = dht().lockAllAsync(cctx, cctx.localNode(), req, filter);
 
             // Add new future.
             add(new GridEmbeddedFuture<>(
@@ -1109,7 +1110,7 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
 
             add(fut); // Append new future.
 
-            IgniteFuture<?> txSync = null;
+            IgniteInternalFuture<?> txSync = null;
 
             if (inTx())
                 txSync = cctx.tm().awaitFinishAckAsync(node.id(), tx.threadId());
@@ -1128,8 +1129,8 @@ public final class GridNearLockFuture<K, V> extends GridCompoundIdentityFuture<B
                 }
             }
             else {
-                txSync.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> t) {
+                txSync.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> t) {
                         try {
                             if (log.isDebugEnabled())
                                 log.debug("Sending near lock request [node=" + node.id() + ", req=" + req + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
index df26576..d790ac3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTransactionalCache.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -93,7 +94,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(
+    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
         @Nullable final Collection<? extends K> keys,
         boolean forcePrimary,
         boolean skipTx,
@@ -113,7 +114,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
 
         if (tx != null && !tx.implicit() && !skipTx) {
             return asyncOp(tx, new AsyncOp<Map<K, V>>(keys) {
-                @Override public IgniteFuture<Map<K, V>> op(IgniteTxLocalAdapter<K, V> tx) {
+                @Override public IgniteInternalFuture<Map<K, V>> op(IgniteTxLocalAdapter<K, V> tx) {
                     return ctx.wrapCloneMap(tx.getAllAsync(ctx, keys, entry, deserializePortable, filter));
                 }
             });
@@ -143,7 +144,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
      * @param expiryPlc Expiry policy.
      * @return Future.
      */
-    IgniteFuture<Map<K, V>> txLoadAsync(GridNearTxLocal<K, V> tx,
+    IgniteInternalFuture<Map<K, V>> txLoadAsync(GridNearTxLocal<K, V> tx,
         @Nullable Collection<? extends K> keys,
         boolean readThrough,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter,
@@ -407,7 +408,7 @@ public class GridNearTransactionalCache<K, V> extends GridNearCacheAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
+    @Override protected IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
         long timeout,
         IgniteTxLocalEx<K, V> tx,
         boolean isInvalidate,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 7e76518..ae0efd5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -119,8 +120,8 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<?>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(IgniteInternalFuture<?> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -131,7 +132,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        for (IgniteFuture<?> fut : futures())
+        for (IgniteInternalFuture<?> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -198,7 +199,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      */
     public void onResult(UUID nodeId, GridNearTxFinishResponse<K, V> res) {
         if (!isDone())
-            for (IgniteFuture<IgniteTx> fut : futures()) {
+            for (IgniteInternalFuture<IgniteTx> fut : futures()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -254,7 +255,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -281,7 +282,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             // Finish local mapping only as we need send commit message to backups.
             for (GridDistributedTxMapping<K, V> m : mappings.values()) {
                 if (m.node().isLocal()) {
-                    IgniteFuture<IgniteTx> fut = cctx.tm().txHandler().finishColocatedLocal(commit, tx);
+                    IgniteInternalFuture<IgniteTx> fut = cctx.tm().txHandler().finishColocatedLocal(commit, tx);
 
                     // Add new future.
                     if (fut != null)
@@ -302,7 +303,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
             if (!isSync()) {
                 boolean complete = true;
 
-                for (IgniteFuture<?> f : pending())
+                for (IgniteInternalFuture<?> f : pending())
                     // Mini-future in non-sync mode gets done when message gets sent.
                     if (isMini(f) && !f.isDone())
                         complete = false;
@@ -368,7 +369,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
         if (n.isLocal()) {
             req.miniId(IgniteUuid.randomUuid());
 
-            IgniteFuture<IgniteTx> fut = cctx.tm().txHandler().finish(n.id(), tx, req);
+            IgniteInternalFuture<IgniteTx> fut = cctx.tm().txHandler().finish(n.id(), tx, req);
 
             // Add new future.
             if (fut != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
index 3c967df..ac5404b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -61,7 +61,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
         new ConcurrentHashMap8<>();
 
     /** Future. */
-    private final AtomicReference<IgniteFuture<IgniteTxEx<K, V>>> prepFut =
+    private final AtomicReference<IgniteInternalFuture<IgniteTxEx<K, V>>> prepFut =
         new AtomicReference<>();
 
     /** */
@@ -181,7 +181,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override protected IgniteFuture<Boolean> addReader(long msgId, GridDhtCacheEntry<K, V> cached,
+    @Override protected IgniteInternalFuture<Boolean> addReader(long msgId, GridDhtCacheEntry<K, V> cached,
         IgniteTxEntry<K, V> entry, long topVer) {
         // We are in near transaction, do not add local node as reader.
         return null;
@@ -278,7 +278,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> loadMissing(
+    @Override public IgniteInternalFuture<Boolean> loadMissing(
         GridCacheContext<K, V> cacheCtx,
         boolean readThrough,
         boolean async,
@@ -292,8 +292,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
                 readThrough,
                 CU.<K, V>empty(),
                 deserializePortable,
-                accessPolicy(cacheCtx, keys)).chain(new C1<IgniteFuture<Map<K, V>>, Boolean>() {
-                @Override public Boolean apply(IgniteFuture<Map<K, V>> f) {
+                accessPolicy(cacheCtx, keys)).chain(new C1<IgniteInternalFuture<Map<K, V>>, Boolean>() {
+                @Override public Boolean apply(IgniteInternalFuture<Map<K, V>> f) {
                     try {
                         Map<K, V> map = f.get();
 
@@ -322,8 +322,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
                 resolveTaskName(),
                 deserializePortable,
                 null,
-                accessPolicy(cacheCtx, keys)).chain(new C1<IgniteFuture<Map<K, V>>, Boolean>() {
-                    @Override public Boolean apply(IgniteFuture<Map<K, V>> f) {
+                accessPolicy(cacheCtx, keys)).chain(new C1<IgniteInternalFuture<Map<K, V>>, Boolean>() {
+                    @Override public Boolean apply(IgniteInternalFuture<Map<K, V>> f) {
                         try {
                             Map<K, V> map = f.get();
 
@@ -694,8 +694,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<IgniteTxEx<K, V>> prepareAsync() {
-        IgniteFuture<IgniteTxEx<K, V>> fut = prepFut.get();
+    @Override public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync() {
+        IgniteInternalFuture<IgniteTxEx<K, V>> fut = prepFut.get();
 
         if (fut == null) {
             // Future must be created before any exception can be thrown.
@@ -762,7 +762,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"ThrowableInstanceNeverThrown"})
-    @Override public IgniteFuture<IgniteTx> commitAsync() {
+    @Override public IgniteInternalFuture<IgniteTx> commitAsync() {
         if (log.isDebugEnabled())
             log.debug("Committing near local tx: " + this);
 
@@ -775,10 +775,10 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
         cctx.mvcc().addFuture(fut);
 
-        IgniteFuture<IgniteTxEx<K, V>> prepareFut = prepFut.get();
+        IgniteInternalFuture<IgniteTxEx<K, V>> prepareFut = prepFut.get();
 
-        prepareFut.listenAsync(new CI1<IgniteFuture<IgniteTxEx<K, V>>>() {
-            @Override public void apply(IgniteFuture<IgniteTxEx<K, V>> f) {
+        prepareFut.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
+            @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
                 GridNearTxFinishFuture<K, V> fut0 = commitFut.get();
 
                 try {
@@ -808,7 +808,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<IgniteTx> rollbackAsync() {
+    @Override public IgniteInternalFuture<IgniteTx> rollbackAsync() {
         if (log.isDebugEnabled())
             log.debug("Rolling back near tx: " + this);
 
@@ -822,7 +822,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
         cctx.mvcc().addFuture(fut);
 
-        IgniteFuture<IgniteTxEx<K, V>> prepFut = this.prepFut.get();
+        IgniteInternalFuture<IgniteTxEx<K, V>> prepFut = this.prepFut.get();
 
         if (prepFut == null || prepFut.isDone()) {
             try {
@@ -846,8 +846,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             }
         }
         else {
-            prepFut.listenAsync(new CI1<IgniteFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteFuture<IgniteTxEx<K, V>> f) {
+            prepFut.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
                     try {
                         // Check for errors in prepare future.
                         f.get();
@@ -890,7 +890,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
      * @return Future that will be completed when locks are acquired.
      */
     @SuppressWarnings("TypeMayBeWeakened")
-    public IgniteFuture<IgniteTxEx<K, V>> prepareAsyncLocal(@Nullable Collection<IgniteTxEntry<K, V>> reads,
+    public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsyncLocal(@Nullable Collection<IgniteTxEntry<K, V>> reads,
         @Nullable Collection<IgniteTxEntry<K, V>> writes, Map<UUID, Collection<UUID>> txNodes, boolean last,
         Collection<UUID> lastBackups) {
         assert optimistic();
@@ -956,7 +956,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
      *
      * @return Commit future.
      */
-    public IgniteFuture<IgniteTx> commitAsyncLocal() {
+    public IgniteInternalFuture<IgniteTx> commitAsyncLocal() {
         if (log.isDebugEnabled())
             log.debug("Committing colocated tx locally: " + this);
 
@@ -964,12 +964,12 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
         if (pessimistic())
             prepareAsync();
 
-        IgniteFuture<IgniteTxEx<K, V>> prep = prepFut.get();
+        IgniteInternalFuture<IgniteTxEx<K, V>> prep = prepFut.get();
 
         // Do not create finish future if there are no remote nodes.
         if (F.isEmpty(dhtMap) && F.isEmpty(nearMap)) {
             if (prep != null)
-                return (IgniteFuture<IgniteTx>)(IgniteFuture)prep;
+                return (IgniteInternalFuture<IgniteTx>)(IgniteInternalFuture)prep;
 
             return new GridFinishedFuture<IgniteTx>(cctx.kernalContext(), this);
         }
@@ -1000,8 +1000,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             }
         }
         else
-            prep.listenAsync(new CI1<IgniteFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteFuture<IgniteTxEx<K, V>> f) {
+            prep.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
                     try {
                         f.get(); // Check for errors of a parent future.
 
@@ -1029,7 +1029,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
      *
      * @return Commit future.
      */
-    public IgniteFuture<IgniteTx> rollbackAsyncLocal() {
+    public IgniteInternalFuture<IgniteTx> rollbackAsyncLocal() {
         if (log.isDebugEnabled())
             log.debug("Rolling back colocated tx locally: " + this);
 
@@ -1037,7 +1037,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
         cctx.mvcc().addFuture(fut);
 
-        IgniteFuture<IgniteTxEx<K, V>> prep = prepFut.get();
+        IgniteInternalFuture<IgniteTxEx<K, V>> prep = prepFut.get();
 
         if (prep == null || prep.isDone()) {
             try {
@@ -1053,8 +1053,8 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
             fut.finish();
         }
         else
-            prep.listenAsync(new CI1<IgniteFuture<IgniteTxEx<K, V>>>() {
-                @Override public void apply(IgniteFuture<IgniteTxEx<K, V>> f) {
+            prep.listenAsync(new CI1<IgniteInternalFuture<IgniteTxEx<K, V>>>() {
+                @Override public void apply(IgniteInternalFuture<IgniteTxEx<K, V>> f) {
                     try {
                         f.get(); // Check for errors of a parent future.
                     }
@@ -1071,7 +1071,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    public IgniteFuture<GridCacheReturn<V>> lockAllAsync(GridCacheContext<K, V> cacheCtx,
+    public IgniteInternalFuture<GridCacheReturn<V>> lockAllAsync(GridCacheContext<K, V> cacheCtx,
         final Collection<? extends K> keys,
         boolean implicit,
         boolean read,
@@ -1095,7 +1095,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
         if (log.isDebugEnabled())
             log.debug("Before acquiring transaction lock on keys: " + keys);
 
-        IgniteFuture<Boolean> fut = cacheCtx.colocated().lockAllAsyncInternal(keys,
+        IgniteInternalFuture<Boolean> fut = cacheCtx.colocated().lockAllAsyncInternal(keys,
             lockTimeout(),
             this,
             isInvalidate(),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
index 12aac5e..8df91e0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearTxPrepareFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed.near;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
@@ -141,8 +142,8 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
      */
     @Override public Collection<? extends ClusterNode> nodes() {
         return
-            F.viewReadOnly(futures(), new IgniteClosure<IgniteFuture<?>, ClusterNode>() {
-                @Nullable @Override public ClusterNode apply(IgniteFuture<?> f) {
+            F.viewReadOnly(futures(), new IgniteClosure<IgniteInternalFuture<?>, ClusterNode>() {
+                @Nullable @Override public ClusterNode apply(IgniteInternalFuture<?> f) {
                     if (isMini(f))
                         return ((MiniFuture)f).node();
 
@@ -165,7 +166,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
     @Override public boolean onNodeLeft(UUID nodeId) {
         boolean found = false;
 
-        for (IgniteFuture<?> fut : futures())
+        for (IgniteInternalFuture<?> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -261,7 +262,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
      */
     public void onResult(UUID nodeId, GridNearTxPrepareResponse<K, V> res) {
         if (!isDone()) {
-            for (IgniteFuture<IgniteTxEx<K, V>> fut : pending()) {
+            for (IgniteInternalFuture<IgniteTxEx<K, V>> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -300,7 +301,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 
@@ -373,8 +374,8 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
             else {
                 topFut.syncNotify(false);
 
-                topFut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                    @Override public void apply(IgniteFuture<Long> t) {
+                topFut.listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                    @Override public void apply(IgniteInternalFuture<Long> t) {
                         prepare();
                     }
                 });
@@ -599,7 +600,7 @@ public final class GridNearTxPrepareFuture<K, V> extends GridCompoundIdentityFut
             // At this point, if any new node joined, then it is
             // waiting for this transaction to complete, so
             // partition reassignments are not possible here.
-            IgniteFuture<IgniteTxEx<K, V>> fut = cctx.tm().txHandler().prepareTx(n.id(), tx, req);
+            IgniteInternalFuture<IgniteTxEx<K, V>> fut = cctx.tm().txHandler().prepareTx(n.id(), tx, req);
 
             // Add new future.
             add(new GridEmbeddedFuture<>(

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
index c2cb47b..560f02b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalCache.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.local;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.lang.*;
@@ -95,7 +96,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> txLockAsync(Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<Boolean> txLockAsync(Collection<? extends K> keys,
         long timeout,
         IgniteTxLocalEx<K, V> tx,
         boolean isRead,
@@ -108,7 +109,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
         IgnitePredicate<CacheEntry<K, V>>[] filter) {
         IgniteTxLocalEx<K, V> tx = ctx.tm().localTx();
 
@@ -122,7 +123,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
      * @param filter Filter.
      * @return Future.
      */
-    public IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
+    public IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
         @Nullable IgniteTxLocalEx<K, V> tx, IgnitePredicate<CacheEntry<K, V>>[] filter) {
         if (F.isEmpty(keys))
             return new GridFinishedFuture<>(ctx.kernalContext(), true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
index a1ce867..17181d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/GridLocalTx.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.processors.cache.local;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.future.*;
@@ -107,7 +107,7 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<IgniteTxEx<K, V>> prepareAsync() {
+    @Override public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync() {
         try {
             prepare();
 
@@ -146,7 +146,7 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings( {"unchecked", "RedundantCast"})
-    @Override public IgniteFuture<IgniteTx> commitAsync() {
+    @Override public IgniteInternalFuture<IgniteTx> commitAsync() {
         try {
             prepare();
         }
@@ -164,11 +164,11 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
 
                 fut.checkLocks();
 
-                return (IgniteFuture)fut;
+                return (IgniteInternalFuture)fut;
             }
         }
 
-        return (IgniteFuture)this.fut.get();
+        return (IgniteInternalFuture)this.fut.get();
     }
 
     /** {@inheritDoc} */
@@ -177,7 +177,7 @@ class GridLocalTx<K, V> extends IgniteTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<IgniteTx> rollbackAsync() {
+    @Override public IgniteInternalFuture<IgniteTx> rollbackAsync() {
         try {
             state(ROLLING_BACK);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 6e72c84..75f8ae8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.local.atomic;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -168,7 +169,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> putAsync(K key,
+    @Override public IgniteInternalFuture<V> putAsync(K key,
         V val,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl,
@@ -187,7 +188,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> putxAsync(K key,
+    @Override public IgniteInternalFuture<Boolean> putxAsync(K key,
         V val,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl,
@@ -211,7 +212,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> putIfAbsentAsync(K key, V val) {
         return putAsync(key, val, ctx.noPeekArray());
     }
 
@@ -221,7 +222,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> putxIfAbsentAsync(K key, V val) {
         return putxAsync(key, val, ctx.noPeekArray());
     }
 
@@ -232,7 +233,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(K key, V val) {
         return putAsync(key, val, ctx.hasPeekArray());
     }
 
@@ -242,7 +243,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(K key, V val) {
         return putxAsync(key, val, ctx.hasPeekArray());
     }
 
@@ -254,7 +255,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
         return putxAsync(key, newVal, ctx.equalsPeekArray(oldVal));
     }
 
@@ -296,7 +297,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
         A.notNull(key, "key", val, "val");
 
         ctx.denyOnLocalRead();
@@ -306,7 +307,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
         A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
 
         ctx.denyOnLocalRead();
@@ -336,7 +337,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
+    @Override public IgniteInternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         ctx.denyOnLocalRead();
 
@@ -368,7 +369,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<V> removeAsync(K key,
+    @Override public IgniteInternalFuture<V> removeAsync(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         ctx.denyOnLocalRead();
@@ -394,7 +395,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<?> removeAllAsync(Collection<? extends K> keys,
         IgnitePredicate<CacheEntry<K, V>>[] filter) {
         ctx.denyOnLocalRead();
 
@@ -432,7 +433,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> removexAsync(K key,
+    @Override public IgniteInternalFuture<Boolean> removexAsync(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         A.notNull(key, "key");
@@ -460,7 +461,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
         return removexAsync(key, ctx.equalsPeekArray(val));
     }
 
@@ -471,7 +472,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<?> removeAllAsync(IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return removeAllAsync(keySet(filter), filter);
     }
 
@@ -518,7 +519,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(
+    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(
         @Nullable final Collection<? extends K> keys,
         final boolean forcePrimary,
         boolean skipTx,
@@ -695,7 +696,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <T> IgniteFuture<EntryProcessorResult<T>> invokeAsync(K key,
+    @Override public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) throws EntryProcessorException {
         A.notNull(key, "key", entryProcessor, "entryProcessor");
@@ -708,15 +709,15 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         Map<? extends K, EntryProcessor> invokeMap =
             Collections.singletonMap(key, (EntryProcessor)entryProcessor);
 
-        IgniteFuture<Map<K, EntryProcessorResult<T>>> fut = updateAllAsync0(null,
+        IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut = updateAllAsync0(null,
             invokeMap,
             args,
             true,
             false,
             null);
 
-        return fut.chain(new CX1<IgniteFuture<Map<K, EntryProcessorResult<T>>>, EntryProcessorResult<T>>() {
-            @Override public EntryProcessorResult<T> applyx(IgniteFuture<Map<K, EntryProcessorResult<T>>> fut)
+        return fut.chain(new CX1<IgniteInternalFuture<Map<K, EntryProcessorResult<T>>>, EntryProcessorResult<T>>() {
+            @Override public EntryProcessorResult<T> applyx(IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> fut)
                 throws IgniteCheckedException {
                 Map<K, EntryProcessorResult<T>> resMap = fut.get();
 
@@ -733,7 +734,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         Set<? extends K> keys,
         final EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
@@ -783,7 +784,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args) {
         A.notNull(map, "map");
@@ -812,7 +813,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      * @param filter Cache entry filter for atomic updates.
      * @return Completion future.
      */
-    private IgniteFuture updateAllAsync0(
+    private IgniteInternalFuture updateAllAsync0(
         @Nullable final Map<? extends K, ? extends V> map,
         @Nullable final Map<? extends K, ? extends EntryProcessor> invokeMap,
         @Nullable final Object[] invokeArgs,
@@ -855,7 +856,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      * @param filter Cache entry filter.
      * @return Completion future.
      */
-    private IgniteFuture removeAllAsync0(
+    private IgniteInternalFuture removeAllAsync0(
         @Nullable final Collection<? extends K> keys,
         final boolean retval,
         final boolean rawRetval,
@@ -869,7 +870,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
         final ExpiryPolicy expiryPlc = expiryPerCall();
 
-        IgniteFuture fut = asyncOp(new Callable<Object>() {
+        IgniteInternalFuture fut = asyncOp(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 return updateAllInternal(DELETE,
                     keys,
@@ -1489,7 +1490,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> txLockAsync(Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<Boolean> txLockAsync(Collection<? extends K> keys,
         long timeout,
         IgniteTxLocalEx<K, V> tx,
         boolean isRead,
@@ -1504,7 +1505,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public IgniteFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys,
         long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         return new GridFinishedFutureEx<>(new UnsupportedOperationException("Locks are not supported for " +
@@ -1538,8 +1539,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    protected IgniteFuture asyncOp(final Callable<?> op) {
-        IgniteFuture fail = asyncOpAcquire();
+    protected IgniteInternalFuture asyncOp(final Callable<?> op) {
+        IgniteInternalFuture fail = asyncOpAcquire();
 
         if (fail != null)
             return fail;
@@ -1549,12 +1550,12 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         holder.lock();
 
         try {
-            IgniteFuture fut = holder.future();
+            IgniteInternalFuture fut = holder.future();
 
             if (fut != null && !fut.isDone()) {
-                IgniteFuture f = new GridEmbeddedFuture(fut,
-                    new C2<Object, Exception, IgniteFuture>() {
-                        @Override public IgniteFuture apply(Object t, Exception e) {
+                IgniteInternalFuture f = new GridEmbeddedFuture(fut,
+                    new C2<Object, Exception, IgniteInternalFuture>() {
+                        @Override public IgniteInternalFuture apply(Object t, Exception e) {
                             return ctx.closures().callLocalSafe(op);
                         }
                     }, ctx.kernalContext());
@@ -1564,7 +1565,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                 return f;
             }
 
-            IgniteFuture f = ctx.closures().callLocalSafe(op);
+            IgniteInternalFuture f = ctx.closures().callLocalSafe(op);
 
             saveFuture(holder, f);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
index 7b8013e..4f668e6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedFieldsQueryFuture.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.util.future.*;
 import org.jetbrains.annotations.*;
@@ -94,7 +94,7 @@ public class GridCacheDistributedFieldsQueryFuture
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<List<GridQueryFieldMetadata>> metadata() {
+    @Override public IgniteInternalFuture<List<GridQueryFieldMetadata>> metadata() {
         return metaFut;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
index 5be7d9f..cb7e9eb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.managers.eventstorage.*;
@@ -521,8 +522,8 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             cctx.io().addOrderedHandler(topic, resHnd);
 
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> fut) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> fut) {
                     cctx.io().removeOrderedHandler(topic);
                 }
             });
@@ -630,8 +631,8 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             cctx.io().addOrderedHandler(topic, resHnd);
 
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> fut) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> fut) {
                     cctx.io().removeOrderedHandler(topic);
                 }
             });

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
index 253afc2..782d9da 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalFieldsQueryFuture.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.internal.processors.cache.query;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.query.*;
 import org.apache.ignite.internal.util.future.*;
 import org.jetbrains.annotations.*;
@@ -74,7 +74,7 @@ public class GridCacheLocalFieldsQueryFuture
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<List<GridQueryFieldMetadata>> metadata() {
+    @Override public IgniteInternalFuture<List<GridQueryFieldMetadata>> metadata() {
         return metaFut;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
index e5ac5f7..7f0c8e4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheLocalQueryFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
@@ -39,7 +40,7 @@ public class GridCacheLocalQueryFuture<K, V, R> extends GridCacheQueryFutureAdap
     private Runnable run;
 
     /** */
-    private IgniteFuture<?> fut;
+    private IgniteInternalFuture<?> fut;
 
     /**
      * Required by {@link Externalizable}.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java
index 2869878..4f39316 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesEx.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.cache.query;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.query.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 
 import java.util.*;
 
@@ -56,7 +56,7 @@ public interface GridCacheQueriesEx<K, V> extends CacheQueries<K, V> {
      * @param qry Query.
      * @return Future.
      */
-    public IgniteFuture<GridCacheSqlResult> execute(String space, GridCacheTwoStepQuery qry);
+    public IgniteInternalFuture<GridCacheSqlResult> execute(String space, GridCacheTwoStepQuery qry);
 
     /**
      * @param space Space.
@@ -64,5 +64,5 @@ public interface GridCacheQueriesEx<K, V> extends CacheQueries<K, V> {
      * @param params Parameters.
      * @return Result.
      */
-    public IgniteFuture<GridCacheSqlResult> executeTwoStepQuery(String space, String sqlQry, Object... params);
+    public IgniteInternalFuture<GridCacheSqlResult> executeTwoStepQuery(String space, String sqlQry, Object... params);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java
index d8edfc1..1d215b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesImpl.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.query;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.indexing.*;
@@ -166,7 +167,7 @@ public class GridCacheQueriesImpl<K, V> implements GridCacheQueriesEx<K, V>, Ext
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheSqlResult> execute(String space, GridCacheTwoStepQuery qry) {
+    @Override public IgniteInternalFuture<GridCacheSqlResult> execute(String space, GridCacheTwoStepQuery qry) {
         return ctx.kernalContext().query().queryTwoStep(space, qry);
     }
 
@@ -176,7 +177,7 @@ public class GridCacheQueriesImpl<K, V> implements GridCacheQueriesEx<K, V>, Ext
      * @param params Parameters.
      * @return Result.
      */
-    public IgniteFuture<GridCacheSqlResult> executeTwoStepQuery(String space, String sqlQry, Object[] params) {
+    public IgniteInternalFuture<GridCacheSqlResult> executeTwoStepQuery(String space, String sqlQry, Object[] params) {
         return ctx.kernalContext().query().queryTwoStep(space, sqlQry, params);
     }
 
@@ -186,21 +187,21 @@ public class GridCacheQueriesImpl<K, V> implements GridCacheQueriesEx<K, V>, Ext
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildIndexes(Class<?> cls) {
+    @Override public IgniteInternalFuture<?> rebuildIndexes(Class<?> cls) {
         A.notNull(cls, "cls");
 
         return ctx.queries().rebuildIndexes(cls);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildIndexes(String typeName) {
+    @Override public IgniteInternalFuture<?> rebuildIndexes(String typeName) {
         A.notNull("typeName", typeName);
 
         return ctx.queries().rebuildIndexes(typeName);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildAllIndexes() {
+    @Override public IgniteInternalFuture<?> rebuildAllIndexes() {
         return ctx.queries().rebuildAllIndexes();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java
index 068fa14..0266629 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueriesProxy.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.query;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
@@ -174,7 +175,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheSqlResult> execute(String space, GridCacheTwoStepQuery qry) {
+    @Override public IgniteInternalFuture<GridCacheSqlResult> execute(String space, GridCacheTwoStepQuery qry) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -186,7 +187,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheSqlResult> executeTwoStepQuery(String space, String sqlQry, Object[] params) {
+    @Override public IgniteInternalFuture<GridCacheSqlResult> executeTwoStepQuery(String space, String sqlQry, Object[] params) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -198,7 +199,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildIndexes(Class<?> cls) {
+    @Override public IgniteInternalFuture<?> rebuildIndexes(Class<?> cls) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -210,7 +211,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildIndexes(String typeName) {
+    @Override public IgniteInternalFuture<?> rebuildIndexes(String typeName) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -222,7 +223,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> rebuildAllIndexes() {
+    @Override public IgniteInternalFuture<?> rebuildAllIndexes() {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {


[18/50] [abbrv] incubator-ignite git commit: # sprint-1 Updated readme files under docs folder.

Posted by vo...@apache.org.
# sprint-1 Updated readme files under docs folder.


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

Branch: refs/heads/ignite-16
Commit: f5d64b2e1fe87d6d81d7b9eb532ba6776200dd91
Parents: 2286097
Author: Dmitiry Setrakyan <ds...@gridgain.com>
Authored: Thu Jan 29 06:30:57 2015 +0000
Committer: Dmitiry Setrakyan <ds...@gridgain.com>
Committed: Thu Jan 29 06:31:15 2015 +0000

----------------------------------------------------------------------
 docs/core-site.gridgain.xml   |  89 ----------------------
 docs/core-site.ignite.xml     |  89 ++++++++++++++++++++++
 docs/gridgain_readme.md       | 149 -------------------------------------
 docs/gridgain_readme.pdf      | Bin 407698 -> 0 bytes
 docs/hadoop_readme.md         |  80 +++++++-------------
 docs/hadoop_readme.pdf        | Bin 411788 -> 82297 bytes
 docs/hive-site.gridgain.xml   |  36 ---------
 docs/hive-site.ignite.xml     |  36 +++++++++
 docs/mapred-site.gridgain.xml |  79 --------------------
 docs/mapred-site.ignite.xml   |  79 ++++++++++++++++++++
 10 files changed, 230 insertions(+), 407 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/core-site.gridgain.xml
----------------------------------------------------------------------
diff --git a/docs/core-site.gridgain.xml b/docs/core-site.gridgain.xml
deleted file mode 100644
index 0054b60..0000000
--- a/docs/core-site.gridgain.xml
+++ /dev/null
@@ -1,89 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-  -->
-
-<!--
-    This template file contains settings needed to run Apache Hadoop jobs
-    with GridGain's distributed in-memory file system GGFS.
-
-    You can replace '$HADOOP_HOME/etc/hadoop/core-site.xml' file with this one
-    to work with GridGain GGFS nodes running on localhost (these local nodes can be
-    a part of distributed cluster though). To work with file system on remote
-    hosts you need to change the host of file system URI to any host running
-    GridGain's GGFS node.
-
-    Note that GridGain jars must be in Apache Hadoop client classpath to work
-    with this configuration.
-
-    Run script '$GRIDGAIN_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop client setup.
--->
-
-<configuration>
-    <!--
-        Set default file system to GGFS instance named "ggfs" configured in GridGain.
-    -->
-    <property>
-        <name>fs.default.name</name>
-        <value>ggfs://ggfs@localhost</value>
-    </property>
-
-    <!--
-        Set Hadoop 1.* file system implementation class for GGFS.
-    -->
-    <property>
-        <name>fs.ggfs.impl</name>
-        <value>org.apache.ignite.fs.hadoop.v1.GridGgfsHadoopFileSystem</value>
-    </property>
-
-    <!--
-        Set Hadoop 2.* file system implementation class for GGFS.
-    -->
-    <property>
-        <name>fs.AbstractFileSystem.ggfs.impl</name>
-        <value>org.apache.ignite.fs.hadoop.v2.GridGgfsHadoopFileSystem</value>
-    </property>
-
-    <!--
-        Disallow data node replacement since it does not make sense for GridGain's GGFS nodes.
-    -->
-    <property>
-        <name>dfs.client.block.write.replace-datanode-on-failure.policy</name>
-        <value>NEVER</value>
-    </property>
-
-    <!--
-        Allow to write the job statistics into GGFS.
-    -->
-    <!--
-    <property>
-        <name>gridgain.counters.writer</name>
-        <value>org.apache.ignite.internal.processors.hadoop.counter.GridHadoopFSCounterWriter</value>
-    </property>
-    -->
-
-    <!--
-        By default data is placed into the file /user/<user_name>/<job_id>/performance
-        You can override this path with using macro ${USER} that is to injection of submitter user name.
-    -->
-    <!--
-    <property>
-        <name>gridgain.counters.fswriter.directory</name>
-        <value>/user/${USER}</value>
-    </property>
-    -->
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/core-site.ignite.xml
----------------------------------------------------------------------
diff --git a/docs/core-site.ignite.xml b/docs/core-site.ignite.xml
new file mode 100644
index 0000000..0054b60
--- /dev/null
+++ b/docs/core-site.ignite.xml
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+  -->
+
+<!--
+    This template file contains settings needed to run Apache Hadoop jobs
+    with GridGain's distributed in-memory file system GGFS.
+
+    You can replace '$HADOOP_HOME/etc/hadoop/core-site.xml' file with this one
+    to work with GridGain GGFS nodes running on localhost (these local nodes can be
+    a part of distributed cluster though). To work with file system on remote
+    hosts you need to change the host of file system URI to any host running
+    GridGain's GGFS node.
+
+    Note that GridGain jars must be in Apache Hadoop client classpath to work
+    with this configuration.
+
+    Run script '$GRIDGAIN_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop client setup.
+-->
+
+<configuration>
+    <!--
+        Set default file system to GGFS instance named "ggfs" configured in GridGain.
+    -->
+    <property>
+        <name>fs.default.name</name>
+        <value>ggfs://ggfs@localhost</value>
+    </property>
+
+    <!--
+        Set Hadoop 1.* file system implementation class for GGFS.
+    -->
+    <property>
+        <name>fs.ggfs.impl</name>
+        <value>org.apache.ignite.fs.hadoop.v1.GridGgfsHadoopFileSystem</value>
+    </property>
+
+    <!--
+        Set Hadoop 2.* file system implementation class for GGFS.
+    -->
+    <property>
+        <name>fs.AbstractFileSystem.ggfs.impl</name>
+        <value>org.apache.ignite.fs.hadoop.v2.GridGgfsHadoopFileSystem</value>
+    </property>
+
+    <!--
+        Disallow data node replacement since it does not make sense for GridGain's GGFS nodes.
+    -->
+    <property>
+        <name>dfs.client.block.write.replace-datanode-on-failure.policy</name>
+        <value>NEVER</value>
+    </property>
+
+    <!--
+        Allow to write the job statistics into GGFS.
+    -->
+    <!--
+    <property>
+        <name>gridgain.counters.writer</name>
+        <value>org.apache.ignite.internal.processors.hadoop.counter.GridHadoopFSCounterWriter</value>
+    </property>
+    -->
+
+    <!--
+        By default data is placed into the file /user/<user_name>/<job_id>/performance
+        You can override this path with using macro ${USER} that is to injection of submitter user name.
+    -->
+    <!--
+    <property>
+        <name>gridgain.counters.fswriter.directory</name>
+        <value>/user/${USER}</value>
+    </property>
+    -->
+</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/gridgain_readme.md
----------------------------------------------------------------------
diff --git a/docs/gridgain_readme.md b/docs/gridgain_readme.md
deleted file mode 100644
index 2bd7370..0000000
--- a/docs/gridgain_readme.md
+++ /dev/null
@@ -1,149 +0,0 @@
-<center>
-![GridGain Logo](http://www.gridgain.com/images/logo/logo_mid.png "GridGain Logo")
-</center>
-
-<div style="height: 5px"></div>
-
-## 1. GridGain In-Memory Data Fabric
-
-GridGain’s In-Memory Data Fabric is designed to deliver uncompromised performance for the widest array of in-memory computing use cases.
-
-Following components are included in the fabric:
-* `In-Memory High Performance Computing (HPC)` - includes distributed clustering, messaging, events, and computational features.
-* `In-Memory Data Grid` - partitioned in-memory key-value store with support for ACID transactions, off-heap memory, SQL, and more.
-* `In-Memory Streaming` - supports event workflow, rolling data windows and indexing, continuous querying, and more.
-
-Enterprise Features:
-* Portable Objects
-* Dynamic data structure changes without restarting cluster
-* Cross-language support, including Java (JVM), .NET (C#), C++
-* Datacenter replication
-* Rolling production updates
-* Local recoverable store
-* Network segmentation protection
-* Secure authentication and Secure client sessions
-* GUI Management & Monitoring
-
-## 2. GridGain Installation
-GridGain distribution comes in a ZIP file that simply needs to be unzipped, and `GRIDGAIN_HOME` environment variable
-can optionally be set to point to it.
-
-There are no additional steps required for GridGain installation in such multi machine setup.
-
-Installation requirements:
-
-1. Windows, Linux, or MacOS environment.
-2. Java 7 or 8 (latest update is advisable).
-3. Point `JAVA_HOME` environment variable to your JDK or JRE installation.
-4. Optional: point `GRIDGAIN_HOME` environment variable to the GridGain installation folder.
-
-### 2.1 Check GridGain Installation
-
-To verify GridGain installation, you can execute the GridGain startup script.
-
-The following command will startup GridGain with default configuration using Multicast node discovery.
-
-    bin/ggstart.{sh|bat}
-
-The following command will startup GridGain with example configuration.
-
-    bin/ggstart.{sh|bat} examples/config/example-compute.xml
-
-If GridGain was installed successfully, the output from above commands should produce no exceptions or errors.
-Note that you may see some warnings during startup, but this is OK as they are meant to inform that certain
-functionality is turned on or off by default.
-
-You can execute the above commands multiple times on the same machine and make sure that nodes discover each other.
-Here is an example of log printout when 2 nodes join topology:
-
-    ... Topology snapshot [nodes=2, CPUs=8, hash=0xD551B245]
-
-You can also start GridGain Management Console, called Visor, and observe started nodes show up on Visor Dashboard.
-To startup Visor, you should execute the following script:
-
-    /bin/ggvisorcmd.{sh|bat} (Command-line, available in open source)
-    /bin/ggvisorui.{sh|bat} (GUI mode, enterprise version only)
-
-### 2.2 Running GridGain Examples
-
-GridGain comes with many well documented examples. All examples have documentation about how they should be
-started and what the expected outcome should be.
-
-> Use provided pom.xml to import examples into IDE of your choice.
-
-## 3. Maven
-GridGain provides repository for its Maven artifacts.
-
-- GridGain Open Source repository is hosted at Maven Central (no additional URL required)
-- GridGain Enterprise repository is located at http://www.gridgainsystems.com/nexus/content/repositories/external
-
-### 3.1 Maven Artifacts
-GridGain Maven repository has `4` artifacts (add `'-ent'` for enterprise edition):
-
-* gridgain-hpc - contains jars and dependencies for In-Memory High Performance Computing (HPC)
-* gridgain-datagrid - contains jars and dependencies for In-Memory Data Grid
-* gridgain-streaming - contains jars and dependencies for In-Memory Streaming
-* gridgain-fabric - contains jars and dependencies for all GridGain editions
-
-### 3.2 Maven Example
-
-#### 3.2.1 Open Source
-
-    <dependency>
-        <groupId>org.gridgain</groupId>
-        <artifactId>gridgain-fabric</artifactId>
-        <version>${ignite.version}</version>
-        <type>pom</type>
-    </dependency>
-
-#### 3.2.2 Enterprise
-
-    <repository>
-        <id>GridGain External Repository</id>
-        <url>http://www.gridgainsystems.com/nexus/content/repositories/external</url>
-    </repository>
-    ...
-    <dependency>
-        <groupId>org.gridgain</groupId>
-        <artifactId>gridgain-fabric-ent</artifactId>
-        <version>${ignite.version}</version>
-        <type>pom</type>
-    </dependency>
-
-## 4. Starting Grid Nodes
-Grid nodes can be started by executing `bin/ggstart.{sh|bat}` script and passing a relative path to GridGain configuration file. If no file is passed, then grid nodes are started with default configuration using Multicast discovery protocol.
-
-Here is an example of how to start GridGain node with non-default configuration:
-
-    `bin/ggstart.sh examples/config/example-cache.xml`
-
-## 5. Management & Monitoring with Visor
-GridGain comes with GUI and CLI (command) based DevOps Managements Consoles delivering advance set of management and monitoring capabilities. Visor GUI is based on a standalone Java application and CLI version is built on top of Scala REPL providing fully scriptable and customizable DevOps environment.
-
-To start Visor GUI console (enterprise edition only), execute the following command:
-
-    `bin/ggvisorui.sh`
-
-To start Visor in console mode you should execute the following command:
-
-    `bin/ggvisorcmd.sh`
-
-On Windows, run the same commands with `.bat` extension.
-
-> **NOTE:** Visor GUI console has a much richer set of functionality over Visor command-based console.
-> You should always prefer Visor GUI console whenever possible.
-
-Here is an example of `Visor Data Grid Tab` which provides overall view on data grid.
-
-<img width="800" src="http://www.gridgain.com/images/visor_database2.png">
-
-Here is an example of `Visor Cache Tab` which provides view on individual cache.
-
-<img width="800" src="http://www.gridgain.com/images/visor_cache.png">
-
-## 6. Scala Integration
-GridGain provides a very nice and easy to use DSL for Scala users called `Scalar`. If you like Scala, take a look at
-Scalar examples located under `examples/src/main/scala` folder.
-
-## 7. Javadoc & Scaladoc
-All documentation is shipped with it and you can find it under `docs/javadoc` and `docs/scaladoc` sub-folder respectively.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/gridgain_readme.pdf
----------------------------------------------------------------------
diff --git a/docs/gridgain_readme.pdf b/docs/gridgain_readme.pdf
deleted file mode 100644
index 0e36849..0000000
Binary files a/docs/gridgain_readme.pdf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/hadoop_readme.md
----------------------------------------------------------------------
diff --git a/docs/hadoop_readme.md b/docs/hadoop_readme.md
index 95a04ed..1fda55d 100644
--- a/docs/hadoop_readme.md
+++ b/docs/hadoop_readme.md
@@ -1,22 +1,22 @@
 <center>
-![GridGain Logo](http://www.gridgain.com/images/logo/logo_mid.png "GridGain Logo")
+![Ignite Logo](https://ignite.incubator.apache.org/images/logo3.png "Ignite Logo")
 </center>
 
-## 1. GridGain In-Memory Accelerator For Apache Hadoop
+## 1. Apache Ignite In-Memory Accelerator For Apache Hadoop
 
-GridGain’s In-Memory Accelerator For Apache Hadoop is designed to deliver uncompromised performance for existing Apache 
+Ignite In-Memory Accelerator For Apache Hadoop is designed to deliver uncompromised performance for existing Apache 
 Hadoop 2.2 or above applications with zero code change as well as simplicity of installation and configuration across all the 
 supported platforms. 
 
 ## 2. Installation
 
-GridGain distribution comes in a ZIP file that simply needs to be unzipped. The Accelerator requires Apache Hadoop of 
+Ignite distribution comes in a ZIP file that simply needs to be unzipped. The Accelerator requires Apache Hadoop of 
 version 2.2 or above to be already installed on the system either using Apache Bigtop packages or manually (manual installation
 just means that Apache Hadoop binary distribution must be unpacked somewhere on the system). In case of manual 
 installation `HADOOP_HOME` environment variable must point to the installation directory of Apache Hadoop. 
 
 > **NOTE:** You do not need any Apache Hadoop processes to be started, you only need to deploy the Apache Hadoop 
-> distribution on your system. Nevertheless you can run Apache Hadoop jobs with GridGain's Accelerator over HDFS,
+> distribution on your system. Nevertheless you can run Apache Hadoop jobs with Ignite Accelerator over HDFS,
 > in this case up and running HDFS infrastructure will be needed.
 
 The Accelerator comes with command line setup tool `bin/setup-hadoop.sh` (`bin/setup-hadoop.bat` on Windows) which 
@@ -35,14 +35,14 @@ Installation requirements:
 > **NOTE:** On Windows platform Apache Hadoop client requires `JAVA_HOME` path to not contain space characters.
 > Java installed to `C:\\Program Files\` will not work, install JRE to correct location and point `JAVA_HOME` there.
 
-### 2.1 Check GridGain Installation
+### 2.1 Check Apache Ignite Installation
 
-After setup script successfully completed, you can execute the GridGain startup script.
-The following command will startup GridGain node with default configuration using multicast node discovery.
+After setup script successfully completed, you can execute the Ignite startup script.
+The following command will startup Ignite node with default configuration using multicast node discovery.
 
     bin/ggstart.{sh|bat}
 
-If GridGain was installed successfully, the output from above commands should produce no exceptions or errors.
+If Ignite was installed successfully, the output from above commands should produce no exceptions or errors.
 Note that you may see some other warnings during startup, but this is OK as they are meant to inform that certain
 functionality is turned on or off by default.
 
@@ -51,49 +51,39 @@ Here is an example of log printout when 2 nodes join topology:
 
     ... Topology snapshot [nodes=2, CPUs=8, hash=0xD551B245]
 
-You can also start GridGain Management Console, called Visor, and observe started nodes show up on Visor Dashboard.
-To startup Visor, you should execute the following script:
+You can also start Ignite Management Console, called Visor, and observe started nodes. To startup Visor, you should execute the following script:
 
-    bin/ggvisorcmd.{sh|bat} (Command-line, available in open source)
-    bin/ggvisorui.{sh|bat} (GUI mode, enterprise version only)
+    /bin/ggvisorcmd.{sh|bat} 
 
 ## 3. Configuration
 
-To configure GridGain nodes you can change configuration files at `config` directory of GridGain installation. Those
-are conventional Spring files. Please refer to shipped configuration files and GridGain javadocs for more details.
+To configure Ignite nodes you can change configuration files at `config` directory of Ignite installation. Those are conventional Spring files. Please refer to shipped configuration files and Ignite javadocs for more details.
 
 ### 3.1 Distributed File System Configuration
 
-GridGain has it's own distributed in-memory file system called GGFS. Hadoop jobs can use it instead of HDFS to achieve
-maximum performance and scalability. Setting up GGFS is much simpler than HDFS, it requires just few tweaks of 
-GridGain node configuration and does not require starting any additional processes. Default configuration shipped 
-with the Accelerator contains one configured instance named "ggfs" which can be used as reference.
+Ignite has it's own distributed in-memory file system called IgniteFS. Hadoop jobs can use it instead of HDFS to achieve maximum performance and scalability. Setting up GGFS is much simpler than HDFS, it requires just few tweaks of Ignite node configuration and does not require starting any additional processes. Default configuration shipped with the Accelerator contains one configured instance named "ignitefs" which can be used as reference.
 
-Generally URI for GGFS which will be used by Apache Hadoop looks like:
+Generally URI for IgniteFS which will be used by Apache Hadoop looks like:
 
-    ggfs://ggfs_name@host_name
+    ignitefs://ignitefs_name@host_name
 
-Where `ggfs_name` is GGFS instance name, `host_name` is any host running GridGain node with that GGFS instance configured. 
-For more details please refer to GGFS documentation.
+Where `ignitefs_name` is IgniteFS instance name, `host_name` is any host running Ignite node with that IgniteFS instance configured. 
+For more details please refer to IgniteFS documentation.
 
 ### 3.2 Apache Hadoop Client Configuration
 
-To run Apache Hadoop jobs with GridGain cluster you need to configure `core-site.xml` and `mapred-site.xml` at 
+To run Apache Hadoop jobs with Ignite cluster you need to configure `core-site.xml` and `mapred-site.xml` at 
 `$HADOOP_HOME/etc/hadoop` directory the same way as it is done in templates shipped with the Accelerator. 
-The setup tool `bin/setup-hadoop.{sh|bat}` will ask you to replace those files with GridGain's templates or 
-you can find these templates at `docs/core-site.gridgain.xml` and `docs/mapred-site.gridgain.xml` respectively
-and perform the needed configuration manually.
+The setup tool `bin/setup-hadoop.{sh|bat}` will ask you to replace those files with Ignite templates or 
+you can find these templates at `docs/core-site.ignite.xml` and `docs/mapred-site.ignite.xml` respectively and perform the needed configuration manually.
 
-Apache Hadoop client will need to have GridGain jar files in classpath, the setup tool will care of that as well.
+Apache Hadoop client will need to have Ignite jar files in classpath, the setup tool will care of that as well.
 
-## 4. Running Apache Hadoop Job With GridGain's In-Memory Accelerator
+## 4. Running Apache Hadoop Job With Ignite In-Memory Accelerator
 
-To run Apache Hadoop job with GridGain cluster you have to start one or multiple GridGain nodes and make sure 
-they successfully discovered each other.
+To run Apache Hadoop job with Ignite cluster you have to start one or multiple Ignite nodes and make sure they successfully discovered each other.
 
-When all the configuration is complete and GridGain nodes are started, running Apache Hadoop job will be the same as 
-with conventional Apache Hadoop distribution except that all GridGain nodes are equal and any of them can be treated 
-as Job Tracker and DFS Name Node.
+When all the configuration is complete and Ignite nodes are started, running Apache Hadoop job will be the same as with conventional Apache Hadoop distribution except that all Ignite nodes are equal and any of them can be treated as Job Tracker and DFS Name Node.
 
 To run "Word Count" example you can load some text files to GGFS using standard Apache Hadoop tools:
  
@@ -118,28 +108,10 @@ switch Apache Hadoop client to a cluster are to fix host in default DFS URI in `
 address in `mapred-site.xml`.
 
 ## 5. Management & Monitoring with Visor
-GridGain comes with GUI and CLI (command) based DevOps Managements Consoles delivering advance set of management and 
-monitoring capabilities. Visor GUI is based on a standalone Java application and CLI version is built on top of 
-Scala REPL providing fully scriptable and customizable DevOps environment.
-
-To start Visor GUI console (enterprise edition only), execute the following command:
-
-    `bin/ggvisorui.sh`
+Ignite comes with CLI (command) based DevOps Managements Console, called Visor, delivering advance set of management and monitoring capabilities. 
 
 To start Visor in console mode you should execute the following command:
 
     `bin/ggvisorcmd.sh`
 
-On Windows, run the same commands with `.bat` extension.
-
-> **NOTE:** Visor GUI console has a much richer set of functionality over Visor command-based console.
-> You should always prefer Visor GUI console whenever possible.
-
-Here is an example of `Visor Data Grid Tab` which provides overall view on data grid.
-
-<img width="800" src="http://www.gridgain.com/images/visor_database2.png">
-
-Here is an example of `Visor Cache Tab` which provides view on individual cache.
-
-<img width="800" src="http://www.gridgain.com/images/visor_cache.png">
-
+On Windows, run the same commands with `.bat` extension.
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/hadoop_readme.pdf
----------------------------------------------------------------------
diff --git a/docs/hadoop_readme.pdf b/docs/hadoop_readme.pdf
index cb5da28..d56d950 100644
Binary files a/docs/hadoop_readme.pdf and b/docs/hadoop_readme.pdf differ

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/hive-site.gridgain.xml
----------------------------------------------------------------------
diff --git a/docs/hive-site.gridgain.xml b/docs/hive-site.gridgain.xml
deleted file mode 100644
index e93be35..0000000
--- a/docs/hive-site.gridgain.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-  -->
-
-<!--
-    This template file contains settings needed to run Apache Hive queries
-    with GridGain In-Memory Accelerator.
-
-    You can replace '$HIVE_HOME/conf/hive-site.xml' file with this one or
-    run script '$GRIDGAIN_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop
-    and Hive client setup.
--->
-<configuration>
-    <!--
-        GridGain requires query plan to be passed not using local resource.
-    -->
-    <property>
-        <name>hive.rpc.query.plan</name>
-        <value>true</value>
-    </property>
-</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/hive-site.ignite.xml
----------------------------------------------------------------------
diff --git a/docs/hive-site.ignite.xml b/docs/hive-site.ignite.xml
new file mode 100644
index 0000000..e93be35
--- /dev/null
+++ b/docs/hive-site.ignite.xml
@@ -0,0 +1,36 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+  -->
+
+<!--
+    This template file contains settings needed to run Apache Hive queries
+    with GridGain In-Memory Accelerator.
+
+    You can replace '$HIVE_HOME/conf/hive-site.xml' file with this one or
+    run script '$GRIDGAIN_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop
+    and Hive client setup.
+-->
+<configuration>
+    <!--
+        GridGain requires query plan to be passed not using local resource.
+    -->
+    <property>
+        <name>hive.rpc.query.plan</name>
+        <value>true</value>
+    </property>
+</configuration>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/mapred-site.gridgain.xml
----------------------------------------------------------------------
diff --git a/docs/mapred-site.gridgain.xml b/docs/mapred-site.gridgain.xml
deleted file mode 100644
index c9b95c4..0000000
--- a/docs/mapred-site.gridgain.xml
+++ /dev/null
@@ -1,79 +0,0 @@
-<?xml version="1.0"?>
-<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-  -->
-
-<!--
-  Licensed under the Apache License, Version 2.0 (the "License");
-  you may not use this file except in compliance with the License.
-  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License. See accompanying LICENSE file.
--->
-
-<!--
-    This template file contains settings needed to run Apache Hadoop jobs
-    with GridGain In-Memory Accelerator.
-
-    You can replace '$HADOOP_HOME/etc/hadoop/mapred-site.xml' file with this one
-    to run jobs on localhost (local node can be a part of distributed cluster though).
-    To run jobs on remote host you have to change jobtracker address to the REST address
-    of any running GridGain node.
-
-    Note that GridGain jars must be in Apache Hadoop client classpath to work
-    with this configuration.
-
-    Run script '$GRIDGAIN_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop client setup.
--->
-
-<configuration>
-    <!-- 
-        Framework name must be set to 'gridgain'.
-    -->
-    <property>
-        <name>mapreduce.framework.name</name>
-        <value>gridgain</value>
-    </property>
-
-    <!--
-        Job tracker address must be set to the REST address of any running GridGain node.
-    -->
-    <property>
-        <name>mapreduce.jobtracker.address</name>
-        <value>localhost:11211</value>
-    </property>
-
-    <!-- Parameters for job tuning. -->
-    <!--
-    <property>
-        <name>mapreduce.job.reduces</name>
-        <value>1</value>
-    </property>
-
-    <property>
-        <name>mapreduce.job.maps</name>
-        <value>4</value>
-    </property>
-    -->
-
-</configuration>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/f5d64b2e/docs/mapred-site.ignite.xml
----------------------------------------------------------------------
diff --git a/docs/mapred-site.ignite.xml b/docs/mapred-site.ignite.xml
new file mode 100644
index 0000000..c9b95c4
--- /dev/null
+++ b/docs/mapred-site.ignite.xml
@@ -0,0 +1,79 @@
+<?xml version="1.0"?>
+<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
+<!--
+  Licensed to the Apache Software Foundation (ASF) under one or more
+  contributor license agreements.  See the NOTICE file distributed with
+  this work for additional information regarding copyright ownership.
+  The ASF licenses this file to You under the Apache License, Version 2.0
+  (the "License"); you may not use this file except in compliance with
+  the License.  You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License.
+  -->
+
+<!--
+  Licensed under the Apache License, Version 2.0 (the "License");
+  you may not use this file except in compliance with the License.
+  You may obtain a copy of the License at
+
+    http://www.apache.org/licenses/LICENSE-2.0
+
+  Unless required by applicable law or agreed to in writing, software
+  distributed under the License is distributed on an "AS IS" BASIS,
+  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+  See the License for the specific language governing permissions and
+  limitations under the License. See accompanying LICENSE file.
+-->
+
+<!--
+    This template file contains settings needed to run Apache Hadoop jobs
+    with GridGain In-Memory Accelerator.
+
+    You can replace '$HADOOP_HOME/etc/hadoop/mapred-site.xml' file with this one
+    to run jobs on localhost (local node can be a part of distributed cluster though).
+    To run jobs on remote host you have to change jobtracker address to the REST address
+    of any running GridGain node.
+
+    Note that GridGain jars must be in Apache Hadoop client classpath to work
+    with this configuration.
+
+    Run script '$GRIDGAIN_HOME/bin/setup-hadoop.{sh|bat}' for Apache Hadoop client setup.
+-->
+
+<configuration>
+    <!-- 
+        Framework name must be set to 'gridgain'.
+    -->
+    <property>
+        <name>mapreduce.framework.name</name>
+        <value>gridgain</value>
+    </property>
+
+    <!--
+        Job tracker address must be set to the REST address of any running GridGain node.
+    -->
+    <property>
+        <name>mapreduce.jobtracker.address</name>
+        <value>localhost:11211</value>
+    </property>
+
+    <!-- Parameters for job tuning. -->
+    <!--
+    <property>
+        <name>mapreduce.job.reduces</name>
+        <value>1</value>
+    </property>
+
+    <property>
+        <name>mapreduce.job.maps</name>
+        <value>4</value>
+    </property>
+    -->
+
+</configuration>


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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
index 171573b..1e68763 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheMockEntry.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.eviction;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -164,7 +165,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<V> setAsync(V val,
+    @Nullable @Override public IgniteInternalFuture<V> setAsync(V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         // No-op.
         return null;
@@ -177,7 +178,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<V> setIfAbsentAsync(V val) {
+    @Nullable @Override public IgniteInternalFuture<V> setIfAbsentAsync(V val) {
         // No-op.
         return null;
     }
@@ -190,7 +191,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<Boolean> setxAsync(V val,
+    @Nullable @Override public IgniteInternalFuture<Boolean> setxAsync(V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         // No-op.
         return null;
@@ -203,7 +204,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<Boolean> setxIfAbsentAsync(V val) {
+    @Nullable @Override public IgniteInternalFuture<Boolean> setxIfAbsentAsync(V val) {
         // No-op.
         return null;
     }
@@ -215,7 +216,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<V> replaceAsync(V val) {
+    @Nullable @Override public IgniteInternalFuture<V> replaceAsync(V val) {
         // No-op.
         return null;
     }
@@ -227,7 +228,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<Boolean> replacexAsync(V val) {
+    @Nullable @Override public IgniteInternalFuture<Boolean> replacexAsync(V val) {
         // No-op.
         return null;
     }
@@ -239,7 +240,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<Boolean> replaceAsync(V oldVal, V newVal) {
+    @Nullable @Override public IgniteInternalFuture<Boolean> replaceAsync(V oldVal, V newVal) {
         // No-op.
         return null;
     }
@@ -252,7 +253,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<V> removeAsync(
+    @Nullable @Override public IgniteInternalFuture<V> removeAsync(
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         // No-op.
         return null;
@@ -265,7 +266,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<Boolean> removexAsync(
+    @Nullable @Override public IgniteInternalFuture<Boolean> removexAsync(
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         // No-op.
         return null;
@@ -278,7 +279,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<Boolean> removeAsync(V val) {
+    @Nullable @Override public IgniteInternalFuture<Boolean> removeAsync(V val) {
         // No-op.
         return null;
     }
@@ -295,7 +296,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAsync(long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAsync(long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         return new GridFinishedFuture<>(null, false);
     }
@@ -326,7 +327,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> reloadAsync() {
+    @Override public IgniteInternalFuture<V> reloadAsync() {
         return new GridFinishedFuture<>();
     }
 
@@ -336,7 +337,7 @@ public class GridCacheMockEntry<K, V> extends GridMetadataAwareAdapter implement
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getAsync() {
+    @Override public IgniteInternalFuture<V> getAsync() {
         return new GridFinishedFuture<>();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java
index b4f53dd..a90e7c0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/eviction/GridCacheSynchronousEvictionsFailoverSelfTest.java
@@ -22,8 +22,8 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
 
@@ -81,7 +81,7 @@ public class GridCacheSynchronousEvictionsFailoverSelfTest extends GridCacheAbst
 
         final AtomicBoolean stop = new AtomicBoolean();
 
-        IgniteFuture<?> fut = null;
+        IgniteInternalFuture<?> fut = null;
 
         try {
             Map<String, Integer> data = new HashMap<>();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java
index 964fdc9..f5e8e26 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/closure/GridClosureProcessorSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 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.marshaller.optimized.*;
 import org.apache.ignite.resources.*;
@@ -189,7 +190,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @return Future object.
      * @throws IgniteCheckedException If failed.
      */
-    private IgniteFuture<?> runAsync(int idx, Runnable job, @Nullable IgnitePredicate<ClusterNode> p)
+    private IgniteInternalFuture<?> runAsync(int idx, Runnable job, @Nullable IgnitePredicate<ClusterNode> p)
         throws IgniteCheckedException {
         assert idx >= 0 && idx < NODES_CNT;
         assert job != null;
@@ -212,7 +213,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @return Future object.
      * @throws IgniteCheckedException If failed.
      */
-    private IgniteFuture<?> broadcast(int idx, Runnable job, @Nullable IgnitePredicate<ClusterNode> p)
+    private IgniteInternalFuture<?> broadcast(int idx, Runnable job, @Nullable IgnitePredicate<ClusterNode> p)
         throws IgniteCheckedException {
         assert idx >= 0 && idx < NODES_CNT;
         assert job != null;
@@ -238,7 +239,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @return Future object.
      * @throws IgniteCheckedException If failed.
      */
-    private IgniteFuture<?> runAsync(int idx, Collection<TestRunnable> jobs, @Nullable IgnitePredicate<ClusterNode> p)
+    private IgniteInternalFuture<?> runAsync(int idx, Collection<TestRunnable> jobs, @Nullable IgnitePredicate<ClusterNode> p)
         throws IgniteCheckedException {
         assert idx >= 0 && idx < NODES_CNT;
         assert !F.isEmpty(jobs);
@@ -261,7 +262,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @return Future object.
      * @throws IgniteCheckedException If failed.
      */
-    private IgniteFuture<Integer> callAsync(int idx, Callable<Integer> job, @Nullable IgnitePredicate<ClusterNode> p)
+    private IgniteInternalFuture<Integer> callAsync(int idx, Callable<Integer> job, @Nullable IgnitePredicate<ClusterNode> p)
         throws IgniteCheckedException {
         assert idx >= 0 && idx < NODES_CNT;
         assert job != null;
@@ -284,7 +285,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @return Future object.
      * @throws IgniteCheckedException If failed.
      */
-    private IgniteFuture<Collection<Integer>> broadcast(int idx, Callable<Integer> job,
+    private IgniteInternalFuture<Collection<Integer>> broadcast(int idx, Callable<Integer> job,
         @Nullable IgnitePredicate<ClusterNode> p) throws IgniteCheckedException {
         assert idx >= 0 && idx < NODES_CNT;
         assert job != null;
@@ -307,7 +308,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
      * @return Future object.
      * @throws IgniteCheckedException If failed.
      */
-    private IgniteFuture<Collection<Integer>> callAsync(int idx, Collection<TestCallable> jobs,
+    private IgniteInternalFuture<Collection<Integer>> callAsync(int idx, Collection<TestCallable> jobs,
         @Nullable IgnitePredicate<ClusterNode> p) throws IgniteCheckedException {
         assert idx >= 0 && idx < NODES_CNT;
         assert !F.isEmpty(jobs);
@@ -341,7 +342,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
     public void testRunAsyncSingle() throws Exception {
         Runnable job = new TestRunnable();
 
-        IgniteFuture<?> fut = broadcast(0, job, null);
+        IgniteInternalFuture<?> fut = broadcast(0, job, null);
 
         assert fut.get() == null;
 
@@ -367,7 +368,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
     public void testRunAsyncMultiple() throws Exception {
         Collection<TestRunnable> jobs = F.asList(new TestRunnable(), new TestRunnable());
 
-        IgniteFuture<?> fut = runAsync(0, jobs, null);
+        IgniteInternalFuture<?> fut = runAsync(0, jobs, null);
 
         assert fut.get() == null : "Execution result must be null.";
 
@@ -381,7 +382,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
     public void testCallAsyncSingle() throws Exception {
         Callable<Integer> job = new TestCallable();
 
-        IgniteFuture<Collection<Integer>> fut1 = broadcast(0, job, null);
+        IgniteInternalFuture<Collection<Integer>> fut1 = broadcast(0, job, null);
 
         assert fut1.get() != null;
 
@@ -394,7 +395,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
 
         assertEquals(1, execCntr.get());
 
-        IgniteFuture<Integer> fut2 = callAsync(0, job, null);
+        IgniteInternalFuture<Integer> fut2 = callAsync(0, job, null);
 
         assert fut2.get() == 1 :
             "Execution result must be equal to 1, actual: " + fut2.get();
@@ -411,7 +412,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
 
         comp.withNoFailover().call(new TestCallableError());
 
-        IgniteFuture<Integer> fut = comp.future();
+        IgniteInternalFuture<Integer> fut = comp.future();
 
         try {
             fut.get();
@@ -467,7 +468,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
     public void testCallAsyncMultiple() throws Exception {
         Collection<TestCallable> jobs = F.asList(new TestCallable(), new TestCallable());
 
-        IgniteFuture<Collection<Integer>> fut = callAsync(0, jobs, null);
+        IgniteInternalFuture<Collection<Integer>> fut = callAsync(0, jobs, null);
 
         Collection<Integer> results = fut.get();
 
@@ -490,7 +491,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
 
         comp.call(jobs, F.sumIntReducer());
 
-        IgniteFuture<Integer> fut = comp.future();
+        IgniteInternalFuture<Integer> fut = comp.future();
 
         // Sum of arithmetic progression.
         int exp = (1 + jobs.size()) * jobs.size() / 2;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
index a67d1cd..8109f5e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/continuous/GridEventConsumeSelfTest.java
@@ -958,7 +958,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
         final Random rnd = new Random();
 
-        IgniteFuture<?> starterFut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> starterFut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 for (int i = 0; i < CONSUME_CNT; i++) {
                     int idx = rnd.nextInt(GRID_CNT);
@@ -991,7 +991,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
             }
         }, 8, "consume-starter");
 
-        IgniteFuture<?> stopperFut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> stopperFut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!stop.get()) {
                     IgniteBiTuple<Integer, UUID> t = queue.poll(1, SECONDS);
@@ -1020,7 +1020,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
             }
         }, 4, "consume-stopper");
 
-        IgniteFuture<?> nodeRestarterFut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> nodeRestarterFut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!stop.get()) {
                     startGrid("anotherGrid");
@@ -1031,7 +1031,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
             }
         }, 1, "node-restarter");
 
-        IgniteFuture<?> jobRunnerFut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> jobRunnerFut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!stop.get()) {
                     int idx = rnd.nextInt(GRID_CNT);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessorSelfTest.java
index 6910310..e0af999 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/dataload/GridDataLoaderProcessorSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.cache.eviction.fifo.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -208,9 +209,9 @@ public class GridDataLoaderProcessorSelfTest extends GridCommonAbstractTest {
 
             final CountDownLatch l1 = new CountDownLatch(threads);
 
-            IgniteFuture<?> f1 = multithreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> f1 = multithreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    Collection<IgniteFuture<?>> futs = new ArrayList<>(cnt);
+                    Collection<IgniteInternalFuture<?>> futs = new ArrayList<>(cnt);
 
                     for (int i = 0; i < cnt; i++) {
                         int idx = idxGen.getAndIncrement();
@@ -220,7 +221,7 @@ public class GridDataLoaderProcessorSelfTest extends GridCommonAbstractTest {
 
                     l1.countDown();
 
-                    for (IgniteFuture<?> fut : futs)
+                    for (IgniteInternalFuture<?> fut : futs)
                         fut.get();
 
                     return null;
@@ -247,9 +248,9 @@ public class GridDataLoaderProcessorSelfTest extends GridCommonAbstractTest {
 
             final CountDownLatch l2 = new CountDownLatch(threads);
 
-            IgniteFuture<?> f2 = multithreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> f2 = multithreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
-                    Collection<IgniteFuture<?>> futs = new ArrayList<>(cnt);
+                    Collection<IgniteInternalFuture<?>> futs = new ArrayList<>(cnt);
 
                     for (int i = 0; i < cnt; i++) {
                         final int key = idxGen.decrementAndGet();
@@ -259,7 +260,7 @@ public class GridDataLoaderProcessorSelfTest extends GridCommonAbstractTest {
 
                     l2.countDown();
 
-                    for (IgniteFuture<?> fut : futs)
+                    for (IgniteInternalFuture<?> fut : futs)
                         fut.get();
 
                     return null;
@@ -391,9 +392,9 @@ public class GridDataLoaderProcessorSelfTest extends GridCommonAbstractTest {
             try {
                 final int totalPutCnt = 50000;
 
-                IgniteFuture<?> fut1 = multithreadedAsync(new Callable<Object>() {
+                IgniteInternalFuture<?> fut1 = multithreadedAsync(new Callable<Object>() {
                     @Override public Object call() throws Exception {
-                        Collection<IgniteFuture<?>> futs = new ArrayList<>();
+                        Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
                         while (!done.get()) {
                             int idx = idxGen.getAndIncrement();
@@ -409,14 +410,14 @@ public class GridDataLoaderProcessorSelfTest extends GridCommonAbstractTest {
 
                         ldr.flush();
 
-                        for (IgniteFuture<?> fut : futs)
+                        for (IgniteInternalFuture<?> fut : futs)
                             fut.get();
 
                         return null;
                     }
                 }, 5, "producer");
 
-                IgniteFuture<?> fut2 = multithreadedAsync(new Callable<Object>() {
+                IgniteInternalFuture<?> fut2 = multithreadedAsync(new Callable<Object>() {
                     @Override public Object call() throws Exception {
                         while (!done.get()) {
                             ldr.flush();
@@ -431,7 +432,7 @@ public class GridDataLoaderProcessorSelfTest extends GridCommonAbstractTest {
                 // Define index of node being restarted.
                 final int restartNodeIdx = nodesCntCache + nodesCntNoCache + 1;
 
-                IgniteFuture<?> fut3 = multithreadedAsync(new Callable<Object>() {
+                IgniteInternalFuture<?> fut3 = multithreadedAsync(new Callable<Object>() {
                     @Override public Object call() throws Exception {
                         try {
                             for (int i = 0; i < 5; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java
index d6b7583..8fa33d2 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsAbstractSelfTest.java
@@ -1121,7 +1121,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
         int threadCnt = 5;
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 while (!stop.get()) {
                     IgniteFsOutputStream os = null;
@@ -1440,7 +1440,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
         int threadCnt = 5;
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 while (!stop.get()) {
                     IgniteFsOutputStream os = null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java
index 0ec979d..92d18ab 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManagerSelfTest.java
@@ -21,6 +21,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.fs.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -166,7 +167,7 @@ public class GridGgfsDataManagerSelfTest extends GridGgfsCommonAbstractTest {
 
             rnd.nextBytes(data);
 
-            IgniteFuture<Boolean> fut = mgr.writeStart(info);
+            IgniteInternalFuture<Boolean> fut = mgr.writeStart(info);
 
             expectsStoreFail(info, data, "Not enough space reserved to store data");
 
@@ -253,7 +254,7 @@ public class GridGgfsDataManagerSelfTest extends GridGgfsCommonAbstractTest {
 
             info = new GridGgfsFileInfo(info, info.length() + data.length + remainder.length);
 
-            IgniteFuture<Boolean> fut = mgr.writeStart(info);
+            IgniteInternalFuture<Boolean> fut = mgr.writeStart(info);
 
             GridGgfsFileAffinityRange range = new GridGgfsFileAffinityRange();
 
@@ -338,7 +339,7 @@ public class GridGgfsDataManagerSelfTest extends GridGgfsCommonAbstractTest {
 
             info = new GridGgfsFileInfo(info, info.length() + data.length * writesCnt);
 
-            IgniteFuture<Boolean> fut = mgr.writeStart(info);
+            IgniteInternalFuture<Boolean> fut = mgr.writeStart(info);
 
             for (int j = 0; j < 64; j++) {
                 Arrays.fill(data, (byte)(j / 4));
@@ -375,7 +376,7 @@ public class GridGgfsDataManagerSelfTest extends GridGgfsCommonAbstractTest {
                 pos += stored.length;
             }
 
-            IgniteFuture<Object> delFut = mgr.delete(info);
+            IgniteInternalFuture<Object> delFut = mgr.delete(info);
 
             delFut.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsTaskSelfTest.java
index 1211797..d003d9b 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsTaskSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/fs/GridGgfsTaskSelfTest.java
@@ -24,6 +24,7 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.fs.*;
 import org.apache.ignite.fs.mapreduce.*;
 import org.apache.ignite.fs.mapreduce.records.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -185,7 +186,7 @@ public class GridGgfsTaskSelfTest extends GridGgfsCommonAbstractTest {
             assertNull(ggfsAsync.execute(
                 new Task(), new IgniteFsStringDelimiterRecordResolver(" "), Collections.singleton(FILE), arg));
 
-            IgniteFuture<IgniteBiTuple<Long, Integer>> fut = ggfsAsync.future();
+            IgniteInternalFuture<IgniteBiTuple<Long, Integer>> fut = ggfsAsync.future();
 
             assertNotNull(fut);
 
@@ -197,7 +198,7 @@ public class GridGgfsTaskSelfTest extends GridGgfsCommonAbstractTest {
 
         ggfsAsync.format();
 
-        IgniteFuture<?> fut = ggfsAsync.future();
+        IgniteInternalFuture<?> fut = ggfsAsync.future();
 
         assertNotNull(fut);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
index b6836e8..dd2f623 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
@@ -22,7 +22,6 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.internal.processors.rest.*;
@@ -242,7 +241,7 @@ public class GridCacheCommandHandlerSelfTest extends GridCommonAbstractTest {
                 new InvocationHandler() {
                     @Override public Object invoke(Object proxy, Method mtd, Object[] args) throws Throwable {
                         if (failMtd.equals(mtd.getName())) {
-                            IgniteFuture<Object> fut = new GridFinishedFuture<>(ctx,
+                            IgniteInternalFuture<Object> fut = new GridFinishedFuture<>(ctx,
                                 new IgniteCheckedException("Operation failed"));
 
                             fut.syncNotify(sync);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java
index a0469b3..c36edff 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorAbstractSelfTest.java
@@ -21,7 +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.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.managed.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.internal.processors.affinity.*;
@@ -146,11 +146,11 @@ public abstract class GridServiceProcessorAbstractSelfTest extends GridCommonAbs
 
         svcs1.deployClusterSingleton(name, new DummyService());
 
-        IgniteFuture<?> fut1 = svcs1.future();
+        IgniteInternalFuture<?> fut1 = svcs1.future();
 
         svcs2.deployClusterSingleton(name, new DummyService());
 
-        IgniteFuture<?> fut2 = svcs2.future();
+        IgniteInternalFuture<?> fut2 = svcs2.future();
 
         info("Deployed service: " + name);
 
@@ -175,11 +175,11 @@ public abstract class GridServiceProcessorAbstractSelfTest extends GridCommonAbs
 
         svcs1.deployClusterSingleton(name, new DummyService());
 
-        IgniteFuture<?> fut1 = svcs1.future();
+        IgniteInternalFuture<?> fut1 = svcs1.future();
 
         svcs2.deployNodeSingleton(name, new DummyService());
 
-        IgniteFuture<?> fut2 = svcs2.future();
+        IgniteInternalFuture<?> fut2 = svcs2.future();
 
         info("Deployed service: " + name);
 
@@ -259,7 +259,7 @@ public abstract class GridServiceProcessorAbstractSelfTest extends GridCommonAbs
 
         svcs.deployNodeSingleton(name, new DummyService());
 
-        IgniteFuture<?> fut = svcs.future();
+        IgniteInternalFuture<?> fut = svcs.future();
 
         info("Deployed service: " + name);
 
@@ -291,7 +291,7 @@ public abstract class GridServiceProcessorAbstractSelfTest extends GridCommonAbs
 
         svcs.deployClusterSingleton(name, new DummyService());
 
-        IgniteFuture<?> fut = svcs.future();
+        IgniteInternalFuture<?> fut = svcs.future();
 
         info("Deployed service: " + name);
 
@@ -325,7 +325,7 @@ public abstract class GridServiceProcessorAbstractSelfTest extends GridCommonAbs
         svcs.deployKeyAffinitySingleton(name, new AffinityService(affKey),
                 CACHE_NAME, affKey);
 
-        IgniteFuture<?> fut = svcs.future();
+        IgniteInternalFuture<?> fut = svcs.future();
 
         info("Deployed service: " + name);
 
@@ -352,7 +352,7 @@ public abstract class GridServiceProcessorAbstractSelfTest extends GridCommonAbs
 
         svcs.deployMultiple(name, new DummyService(), nodeCount() * 2, 3);
 
-        IgniteFuture<?> fut = svcs.future();
+        IgniteInternalFuture<?> fut = svcs.future();
 
         info("Deployed service: " + name);
 
@@ -386,7 +386,7 @@ public abstract class GridServiceProcessorAbstractSelfTest extends GridCommonAbs
 
         svcs.deployMultiple(name, new DummyService(), cnt, 3);
 
-        IgniteFuture<?> fut = svcs.future();
+        IgniteInternalFuture<?> fut = svcs.future();
 
         info("Deployed service: " + name);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java
index fccbf85..355aea4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/service/GridServiceProcessorMultiNodeSelfTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.processors.service;
 
 import junit.framework.*;
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 
 import java.util.concurrent.*;
 
@@ -48,7 +48,7 @@ public class GridServiceProcessorMultiNodeSelfTest extends GridServiceProcessorA
 
         svcs.deployClusterSingleton(name, new DummyService());
 
-        IgniteFuture<?> fut = svcs.future();
+        IgniteInternalFuture<?> fut = svcs.future();
 
         info("Deployed service: " + name);
 
@@ -96,7 +96,7 @@ public class GridServiceProcessorMultiNodeSelfTest extends GridServiceProcessorA
         svcs.deployKeyAffinitySingleton(name, new AffinityService(affKey),
             CACHE_NAME, affKey);
 
-        IgniteFuture<?> fut = svcs.future();
+        IgniteInternalFuture<?> fut = svcs.future();
 
         info("Deployed service: " + name);
 
@@ -134,7 +134,7 @@ public class GridServiceProcessorMultiNodeSelfTest extends GridServiceProcessorA
 
         svcs.deployNodeSingleton(name, new DummyService());
 
-        IgniteFuture<?> fut = svcs.future();
+        IgniteInternalFuture<?> fut = svcs.future();
 
         info("Deployed service: " + name);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/processors/streamer/GridStreamerFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/streamer/GridStreamerFailoverSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/streamer/GridStreamerFailoverSelfTest.java
index e28f927..90fbe20 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/streamer/GridStreamerFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/streamer/GridStreamerFailoverSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.streamer;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.streamer.*;
 import org.apache.ignite.streamer.window.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -139,7 +139,7 @@ public class GridStreamerFailoverSelfTest extends GridCommonAbstractTest {
 
             final AtomicBoolean done = new AtomicBoolean(false);
 
-            IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     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/timeout/GridTimeoutProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java
index b9cfbc5..aafe93c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/timeout/GridTimeoutProcessorSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.internal.processors.timeout;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.*;
@@ -382,7 +383,7 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
     public void testAddRemoveInterleaving() throws Exception {
         final AtomicInteger callCnt = new AtomicInteger(0);
 
-        IgniteFuture<?> rmv = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> rmv = GridTestUtils.runMultiThreadedAsync(new Runnable() {
             /** {@inheritDoc} */
             @SuppressWarnings("CallToThreadYield")
             @Override public void run() {
@@ -440,7 +441,7 @@ public class GridTimeoutProcessorSelfTest extends GridCommonAbstractTest {
 
         final CountDownLatch latch = new CountDownLatch(max * threads);
 
-        IgniteFuture<?> called = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> called = GridTestUtils.runMultiThreadedAsync(new Runnable() {
             /** {@inheritDoc} */
             @SuppressWarnings("CallToThreadYield")
             @Override public void run() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridCompoundFutureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridCompoundFutureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridCompoundFutureSelfTest.java
index d31cf82..550e8ee 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridCompoundFutureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridCompoundFutureSelfTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.util.future;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.junits.*;
 import org.apache.ignite.testframework.junits.common.*;
@@ -39,7 +39,7 @@ public class GridCompoundFutureSelfTest extends GridCommonAbstractTest {
         GridCompoundFuture<Boolean, Boolean> fut = new GridCompoundFuture<>();
 
         for (int i = 0; i < 5; i++) {
-            IgniteFuture<Boolean> part = new GridFinishedFuture<>(ctx, true);
+            IgniteInternalFuture<Boolean> part = new GridFinishedFuture<>(ctx, true);
 
             part.syncNotify(true);
 
@@ -143,7 +143,7 @@ public class GridCompoundFutureSelfTest extends GridCommonAbstractTest {
 
         fut.markInitialized();
 
-        IgniteFuture<?> complete = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> complete = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 GridFutureAdapter<Boolean> part;
 
@@ -178,7 +178,7 @@ public class GridCompoundFutureSelfTest extends GridCommonAbstractTest {
 
         fut.markInitialized();
 
-        IgniteFuture<?> complete = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> complete = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 GridFutureAdapter<Boolean> part;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridEmbeddedFutureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridEmbeddedFutureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridEmbeddedFutureSelfTest.java
index d94a4e0..62d5c1a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridEmbeddedFutureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridEmbeddedFutureSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.future;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.junits.*;
@@ -48,12 +49,12 @@ public class GridEmbeddedFutureSelfTest extends GridCommonAbstractTest {
     public void testFutureChain() throws Exception {
         GridFutureAdapter<Integer> fut = new GridFutureAdapter<>(ctx);
 
-        IgniteFuture<Integer> cur = fut;
+        IgniteInternalFuture<Integer> cur = fut;
 
         for (int i = 0; i < DFLT_MAX_CONCURRENT_ASYNC_OPS; i++) {
             cur = new GridEmbeddedFuture<>(cur,
-                new IgniteBiClosure<Integer, Exception, IgniteFuture<Integer>>() {
-                    @Override public IgniteFuture<Integer> apply(Integer o, Exception e) {
+                new IgniteBiClosure<Integer, Exception, IgniteInternalFuture<Integer>>() {
+                    @Override public IgniteInternalFuture<Integer> apply(Integer o, Exception e) {
                         return new GridFinishedFuture<>(ctx, o);
                     }
                 }, ctx);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java
index 3a15be5..43a8cf0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFinishedFutureSelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.util.future;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.optimized.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -69,7 +68,7 @@ public class GridFinishedFutureSelfTest extends GridCommonAbstractTest {
         IgniteMarshaller m = new IgniteOptimizedMarshaller();
         ClassLoader clsLdr = getClass().getClassLoader();
 
-        IgniteFuture<Object> orig = t == null ? new GridFinishedFuture<>(ctx, ex) :
+        IgniteInternalFuture<Object> orig = t == null ? new GridFinishedFuture<>(ctx, ex) :
             new GridFinishedFuture<>(ctx, t);
 
         orig.syncNotify(syncNotify);
@@ -90,8 +89,8 @@ public class GridFinishedFutureSelfTest extends GridCommonAbstractTest {
 
         final CountDownLatch done = new CountDownLatch(1);
 
-        fut.listenAsync(new CI1<IgniteFuture<Object>>() {
-            @Override public void apply(IgniteFuture<Object> t) {
+        fut.listenAsync(new CI1<IgniteInternalFuture<Object>>() {
+            @Override public void apply(IgniteInternalFuture<Object> t) {
                 done.countDown();
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
index 393a054..9acdb19 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureAdapterSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.util.future;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.closure.*;
 import org.apache.ignite.internal.util.io.*;
@@ -126,8 +127,8 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
         final AtomicReference<Exception> err = new AtomicReference<>();
 
         for (int i = 0; i < lsnrCnt; i++) {
-            fut.listenAsync(new CI1<IgniteFuture<String>>() {
-                @Override public void apply(IgniteFuture<String> t) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<String>>() {
+                @Override public void apply(IgniteInternalFuture<String> t) {
                     if (Thread.currentThread() != runThread)
                         err.compareAndSet(null, new Exception("Wrong notification thread: " + Thread.currentThread()));
 
@@ -147,8 +148,8 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
 
         err.set(null);
 
-        fut.listenAsync(new CI1<IgniteFuture<String>>() {
-            @Override public void apply(IgniteFuture<String> t) {
+        fut.listenAsync(new CI1<IgniteInternalFuture<String>>() {
+            @Override public void apply(IgniteInternalFuture<String> t) {
                 if (Thread.currentThread() != runThread)
                     err.compareAndSet(null, new Exception("Wrong notification thread: " + Thread.currentThread()));
 
@@ -187,8 +188,8 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
             final AtomicReference<Exception> err = new AtomicReference<>();
 
             for (int i = 0; i < lsnrCnt; i++) {
-                fut.listenAsync(new CI1<IgniteFuture<String>>() {
-                    @Override public void apply(IgniteFuture<String> t) {
+                fut.listenAsync(new CI1<IgniteInternalFuture<String>>() {
+                    @Override public void apply(IgniteInternalFuture<String> t) {
                         if (Thread.currentThread() == runThread)
                             err.compareAndSet(null, new Exception("Wrong notification thread: " +
                                 Thread.currentThread()));
@@ -209,8 +210,8 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
 
             err.set(null);
 
-            fut.listenAsync(new CI1<IgniteFuture<String>>() {
-                @Override public void apply(IgniteFuture<String> t) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<String>>() {
+                @Override public void apply(IgniteInternalFuture<String> t) {
                     if (Thread.currentThread() == runThread)
                         err.compareAndSet(null, new Exception("Wrong notification thread: " + Thread.currentThread()));
 
@@ -235,8 +236,8 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
      */
     @SuppressWarnings("ErrorNotRethrown")
     public void testChaining() throws Exception {
-        final CX1<IgniteFuture<Object>, Object> passThrough = new CX1<IgniteFuture<Object>, Object>() {
-            @Override public Object applyx(IgniteFuture<Object> f) throws IgniteCheckedException {
+        final CX1<IgniteInternalFuture<Object>, Object> passThrough = new CX1<IgniteInternalFuture<Object>, Object>() {
+            @Override public Object applyx(IgniteInternalFuture<Object> f) throws IgniteCheckedException {
                 return f.get();
             }
         };
@@ -254,7 +255,7 @@ public class GridFutureAdapterSelfTest extends GridCommonAbstractTest {
             // Test result returned.
 
             GridFutureAdapter<Object> fut = new GridFutureAdapter<>(ctx);
-            IgniteFuture<Object> chain = fut.chain(passThrough);
+            IgniteInternalFuture<Object> chain = fut.chain(passThrough);
 
             assertFalse(fut.isDone());
             assertFalse(chain.isDone());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureListenPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureListenPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureListenPerformanceTest.java
index 39fc283..f87e12d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureListenPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/future/GridFutureListenPerformanceTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.future;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.jdk8.backport.*;
 
@@ -74,8 +75,8 @@ public class GridFutureListenPerformanceTest {
                             futs.add(fut);
 
                             for (int k = 1; k < rnd.nextInt(3); k++) {
-                                    fut.listenAsync(new IgniteInClosure<IgniteFuture<Object>>() {
-                                    @Override public void apply(IgniteFuture<Object> t) {
+                                    fut.listenAsync(new IgniteInClosure<IgniteInternalFuture<Object>>() {
+                                    @Override public void apply(IgniteInternalFuture<Object> t) {
                                         try {
                                             t.get();
                                         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java
index fac8c9b..6d08898 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/ipc/shmem/GridIpcSharedMemorySpaceSelfTest.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.util.ipc.shmem;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.lang.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jdk8.backport.*;
@@ -68,7 +68,7 @@ public class GridIpcSharedMemorySpaceSelfTest extends GridCommonAbstractTest {
 
         final AtomicReference<GridIpcSharedMemorySpace> spaceRef = new AtomicReference<>();
 
-        IgniteFuture<?> fut1 = multithreadedAsync(
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(
             new Callable<Object>() {
                 @SuppressWarnings("TooBroadScope")
                 @Override public Object call() throws Exception {
@@ -99,7 +99,7 @@ public class GridIpcSharedMemorySpaceSelfTest extends GridCommonAbstractTest {
             1,
             "writer");
 
-        IgniteFuture<?> fut2 = multithreadedAsync(
+        IgniteInternalFuture<?> fut2 = multithreadedAsync(
             new Callable<Object>() {
                 @SuppressWarnings({"TooBroadScope", "StatementWithEmptyBody"})
                 @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
index f5f061a..83729fc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/nio/GridNioSelfTest.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.util.nio;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -102,7 +102,7 @@ public class GridNioSelfTest extends GridCommonAbstractTest {
         GridNioServer<?> srvr = startServer(PORT, new GridPlainParser(), lsnr);
 
         try {
-            IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
                 @Override
                 public void run() {
                     byte[] msg = new byte[MSG_SIZE];
@@ -256,7 +256,7 @@ public class GridNioSelfTest extends GridCommonAbstractTest {
         final AtomicBoolean running = new AtomicBoolean(true);
 
         try {
-            IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
                 @Override
                 public void run() {
                     try {
@@ -628,7 +628,7 @@ public class GridNioSelfTest extends GridCommonAbstractTest {
             new EchoListener());
 
         try {
-            IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
                 @SuppressWarnings("BusyWait")
                 @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/util/offheap/GridOffHeapMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
index 2fc7db0..f271512 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapMapAbstractSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.offheap;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.offheap.unsafe.*;
@@ -662,7 +663,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
 
         final AtomicBoolean run = new AtomicBoolean(true);
 
-        IgniteFuture<?> itFut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> itFut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     startLatch.await();
@@ -687,7 +688,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
             }
         }, 1);
 
-        IgniteFuture<?> putFut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> putFut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     startLatch.await();
@@ -746,7 +747,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
             keys[i][0] = (byte)i; // hash
         }
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Void>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Void>() {
             @Override
             public Void call() throws IgniteCheckedException {
                 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/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
index c7c81c4..61bcae0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.util.offheap;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.testframework.junits.common.*;
@@ -595,7 +596,7 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
 
         final AtomicBoolean running = new AtomicBoolean(true);
 
-        IgniteFuture<?> iterFut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> iterFut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     while (running.get()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
index f8c469e..1f1f806 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.util.offheap;
 
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
-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.testframework.junits.common.*;
@@ -167,7 +167,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 
@@ -222,7 +222,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 
@@ -293,7 +293,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 
@@ -348,7 +348,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 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/util/offheap/unsafe/GridUnsafeMemorySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMemorySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMemorySelfTest.java
index 1f23c9b..9048eb4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMemorySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/util/offheap/unsafe/GridUnsafeMemorySelfTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.util.offheap.unsafe;
 
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jdk8.backport.*;
@@ -264,7 +264,7 @@ public class GridUnsafeMemorySelfTest extends GridCommonAbstractTest {
 
         final AtomicBoolean run = new AtomicBoolean(true);
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 int x = 0;
 
@@ -317,7 +317,7 @@ public class GridUnsafeMemorySelfTest extends GridCommonAbstractTest {
 
             final LongAdder locAdder = new LongAdder();
 
-            IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     Random rnd = new GridRandom();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/jvmtest/ConcurrentMapTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/jvmtest/ConcurrentMapTest.java b/modules/core/src/test/java/org/apache/ignite/jvmtest/ConcurrentMapTest.java
index 368dd72..aec32f5 100644
--- a/modules/core/src/test/java/org/apache/ignite/jvmtest/ConcurrentMapTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/jvmtest/ConcurrentMapTest.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.jvmtest;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.testframework.*;
@@ -84,7 +84,7 @@ public class ConcurrentMapTest {
 
         final AtomicInteger writes = new AtomicInteger();
 
-        IgniteFuture fut1 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture fut1 = GridTestUtils.runMultiThreadedAsync(
                 new Runnable() {
                     @Override public void run() {
                         while (!done.get()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/jvmtest/NetworkFailureTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/jvmtest/NetworkFailureTest.java b/modules/core/src/test/java/org/apache/ignite/jvmtest/NetworkFailureTest.java
index 8aa9081..5a8b1da 100644
--- a/modules/core/src/test/java/org/apache/ignite/jvmtest/NetworkFailureTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/jvmtest/NetworkFailureTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.jvmtest;
 
 import junit.framework.*;
-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.testframework.*;
@@ -42,7 +42,7 @@ public class NetworkFailureTest extends TestCase {
 
         final InetAddress addr = InetAddress.getByName("192.168.0.100");
 
-        IgniteFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     ServerSocket srvSock = null;
@@ -80,7 +80,7 @@ public class NetworkFailureTest extends TestCase {
             "server"
         );
 
-        IgniteFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     Socket sock = null;
@@ -141,7 +141,7 @@ public class NetworkFailureTest extends TestCase {
     public void testReadTimeout() throws Exception {
         final InetAddress addr = InetAddress.getByName("192.168.3.10");
 
-        IgniteFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     ServerSocket srvSock = null;
@@ -182,7 +182,7 @@ public class NetworkFailureTest extends TestCase {
             "server"
         );
 
-        IgniteFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     Socket sock = null;
@@ -225,7 +225,7 @@ public class NetworkFailureTest extends TestCase {
 
         final InetAddress addr = InetAddress.getByName("192.168.0.100");
 
-        IgniteFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     ServerSocket srvSock = null;
@@ -253,7 +253,7 @@ public class NetworkFailureTest extends TestCase {
 
         final AtomicReference<Socket> sockRef = new AtomicReference<>();
 
-        IgniteFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     Socket sock = null;
@@ -287,7 +287,7 @@ public class NetworkFailureTest extends TestCase {
             "client"
         );
 
-        IgniteFuture<?> fut3 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut3 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     while (true) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/jvmtest/QueueSizeCounterMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/jvmtest/QueueSizeCounterMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/jvmtest/QueueSizeCounterMultiThreadedTest.java
index ecbcdb5..edade0a 100644
--- a/modules/core/src/test/java/org/apache/ignite/jvmtest/QueueSizeCounterMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/jvmtest/QueueSizeCounterMultiThreadedTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.jvmtest;
 
 import junit.framework.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
 import org.jetbrains.annotations.*;
@@ -46,7 +46,7 @@ public class QueueSizeCounterMultiThreadedTest extends TestCase {
 
         final ReadWriteLock lock = new ReentrantReadWriteLock();
 
-        IgniteFuture fut1 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture fut1 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @SuppressWarnings( {"BusyWait"})
                 @Nullable @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/jvmtest/ReadWriteLockMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/jvmtest/ReadWriteLockMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/jvmtest/ReadWriteLockMultiThreadedTest.java
index 416f040..b484c97 100644
--- a/modules/core/src/test/java/org/apache/ignite/jvmtest/ReadWriteLockMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/jvmtest/ReadWriteLockMultiThreadedTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.jvmtest;
 
 import junit.framework.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
 import org.jetbrains.annotations.*;
@@ -62,7 +62,7 @@ public class ReadWriteLockMultiThreadedTest extends TestCase {
 
         X.println("Read lock acquired.");
 
-        IgniteFuture fut1 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture fut1 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     X.println("Attempting to acquire write lock: " + lock);
@@ -85,7 +85,7 @@ public class ReadWriteLockMultiThreadedTest extends TestCase {
 
         Thread.sleep(2000);
 
-        IgniteFuture fut2 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture fut2 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     X.println("Attempting to acquire read lock: " + lock);
@@ -131,7 +131,7 @@ public class ReadWriteLockMultiThreadedTest extends TestCase {
 
         X.println("Write lock acquired: " + lock);
 
-        IgniteFuture fut = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     X.println("Attempting to acquire read lock: " + lock);
@@ -176,7 +176,7 @@ public class ReadWriteLockMultiThreadedTest extends TestCase {
 
         X.println("Read lock acquired.");
 
-        IgniteFuture fut = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture fut = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     boolean res = lock.writeLock().tryLock();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java
index 688eebf..3e2fa19 100644
--- a/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/lang/GridBasicPerformanceTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.lang;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -821,7 +822,7 @@ public class GridBasicPerformanceTest {
 
         final CountDownLatch latch1 = new CountDownLatch(1);
 
-        IgniteFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut1 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     latch1.await();
@@ -854,7 +855,7 @@ public class GridBasicPerformanceTest {
 
         final CountDownLatch latch2 = new CountDownLatch(1);
 
-        IgniteFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture<?> fut2 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @SuppressWarnings("StatementWithEmptyBody")
                 @Nullable @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/lang/GridFuncSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/lang/GridFuncSelfTest.java b/modules/core/src/test/java/org/apache/ignite/lang/GridFuncSelfTest.java
index fe3f031..7e9d618 100644
--- a/modules/core/src/test/java/org/apache/ignite/lang/GridFuncSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/lang/GridFuncSelfTest.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
+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.internal.util.typedef.*;
@@ -3187,7 +3188,7 @@ public class GridFuncSelfTest extends GridCommonAbstractTest {
             new GridFutureAdapter<>(), new GridFutureAdapter<>(), new GridFutureAdapter<>()
         };
 
-        for (IgniteFuture fut : futs) {
+        for (IgniteInternalFuture fut : futs) {
             assert !fut.isDone();
         }
 
@@ -3208,7 +3209,7 @@ public class GridFuncSelfTest extends GridCommonAbstractTest {
 
         F.<Object>awaitAll(futs);
 
-        for (IgniteFuture fut : futs) {
+        for (IgniteInternalFuture fut : futs) {
             assert fut.isDone();
         }
     }
@@ -3220,7 +3221,7 @@ public class GridFuncSelfTest extends GridCommonAbstractTest {
     public void testAwaitOne() throws Exception {
         final GridFutureAdapter<?>[] futs = {new GridFutureAdapter(), new GridFutureAdapter(), new GridFutureAdapter()};
 
-        for (IgniteFuture fut : futs) {
+        for (IgniteInternalFuture fut : futs) {
             assert !fut.isDone();
         }
 
@@ -3237,16 +3238,16 @@ public class GridFuncSelfTest extends GridCommonAbstractTest {
             }
         }.start();
 
-        IgniteFuture doneFut = F.awaitOne((IgniteFuture[])futs);
+        IgniteInternalFuture doneFut = F.awaitOne((IgniteInternalFuture[])futs);
 
         assert doneFut.isDone();
 
-        for (IgniteFuture fut : futs) {
+        for (IgniteInternalFuture fut : futs) {
             assert doneFut == fut ? fut.isDone() : !fut.isDone();
         }
 
         // Check only NULLs.
-        IgniteFuture<Object> fut = F.awaitOne(Arrays.asList((IgniteFuture<Object>)null, null, null));
+        IgniteInternalFuture<Object> fut = F.awaitOne(Arrays.asList((IgniteInternalFuture<Object>)null, null, null));
 
         assert fut.isDone();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/lang/GridFutureListenPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/lang/GridFutureListenPerformanceTest.java b/modules/core/src/test/java/org/apache/ignite/lang/GridFutureListenPerformanceTest.java
index 6929be8..a139c4f 100644
--- a/modules/core/src/test/java/org/apache/ignite/lang/GridFutureListenPerformanceTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/lang/GridFutureListenPerformanceTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.lang;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.future.*;
 import org.jdk8.backport.*;
 
@@ -74,8 +75,8 @@ public class GridFutureListenPerformanceTest {
                             futs.add(fut);
 
                             for (int k = 1; k < rnd.nextInt(3); k++) {
-                                fut.listenAsync(new IgniteInClosure<IgniteFuture<Object>>() {
-                                    @Override public void apply(IgniteFuture<Object> t) {
+                                fut.listenAsync(new IgniteInClosure<IgniteInternalFuture<Object>>() {
+                                    @Override public void apply(IgniteInternalFuture<Object> t) {
                                         try {
                                             t.get();
                                         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
index 51f243f..e7412c3 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.loadtest.swapspace;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.swapspace.*;
 import org.apache.ignite.spi.swapspace.file.*;
@@ -147,7 +147,7 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
             }
         });
 
-        IgniteFuture<?> evictFut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> evictFut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();
@@ -209,7 +209,7 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
             }
         });
 
-        IgniteFuture<?> unswapFut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> unswapFut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheAbstractLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheAbstractLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheAbstractLoadTest.java
index f6b5904..077beed 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheAbstractLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheAbstractLoadTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.loadtests.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.transactions.*;
 import org.apache.log4j.*;
 import org.apache.log4j.varia.*;
@@ -136,7 +136,7 @@ abstract class GridCacheAbstractLoadTest {
         assert cache != null;
 
         try {
-            IgniteFuture<?> f1 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> f1 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     long start = System.currentTimeMillis();
 
@@ -158,7 +158,7 @@ abstract class GridCacheAbstractLoadTest {
                 }
             }, writeThreads(), "cache-load-test-worker");
 
-            IgniteFuture<?> f2 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> f2 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     long start = System.currentTimeMillis();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheSwapLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheSwapLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheSwapLoadTest.java
index 66e99f8..cb91263 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheSwapLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheSwapLoadTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.loadtests.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
@@ -123,7 +124,7 @@ public class GridCacheSwapLoadTest {
                 }
             }, EVT_CACHE_OBJECT_SWAPPED);
 
-            Collection<IgniteFuture<?>> futs = new ArrayList<>(3);
+            Collection<IgniteInternalFuture<?>> futs = new ArrayList<>(3);
 
             long start = System.currentTimeMillis();
 
@@ -197,7 +198,7 @@ public class GridCacheSwapLoadTest {
     /**
      * @return Future.
      */
-    private static IgniteFuture<?> doPut(final Ignite g) {
+    private static IgniteInternalFuture<?> doPut(final Ignite g) {
         final AtomicInteger putKey = new AtomicInteger(0);
 
         return GridTestUtils.runMultiThreadedAsync(new CAX() {
@@ -226,7 +227,7 @@ public class GridCacheSwapLoadTest {
     /**
      * @return Futures.
      */
-    private static Collection<IgniteFuture<Long>> doGetRemove(final Ignite g) {
+    private static Collection<IgniteInternalFuture<Long>> doGetRemove(final Ignite g) {
         final AtomicBoolean stop = new AtomicBoolean(false);
 
         return F.asList(
@@ -300,9 +301,9 @@ public class GridCacheSwapLoadTest {
     /**
      * @param futs Futures.
      */
-    private static void wait(Iterable<IgniteFuture<?>> futs) {
-        F.forEach(futs, new CIX1<IgniteFuture<?>>() {
-            @Override public void applyx(IgniteFuture<?> fut) throws IgniteCheckedException {
+    private static void wait(Iterable<IgniteInternalFuture<?>> futs) {
+        F.forEach(futs, new CIX1<IgniteInternalFuture<?>>() {
+            @Override public void applyx(IgniteInternalFuture<?> fut) throws IgniteCheckedException {
                 fut.get();
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheWriteBehindStoreLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheWriteBehindStoreLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheWriteBehindStoreLoadTest.java
index 213b634..e8b5a30 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheWriteBehindStoreLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/cache/GridCacheWriteBehindStoreLoadTest.java
@@ -22,7 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.GridCache;
 import org.apache.ignite.cache.store.*;
 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.internal.util.typedef.internal.*;
@@ -157,7 +157,7 @@ public class GridCacheWriteBehindStoreLoadTest extends GridCommonAbstractTest {
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @SuppressWarnings({"NullableProblems"})
             @Override public void run() {
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestMain.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestMain.java b/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestMain.java
index 0aed81d..5207af6 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestMain.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/colocation/GridTestMain.java
@@ -20,7 +20,7 @@ package org.apache.ignite.loadtests.colocation;
 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.thread.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.springframework.beans.factory.*;
@@ -73,7 +73,7 @@ public class GridTestMain {
 
         final GridCache<GridTestKey, Long> cache = g.cache("partitioned");
 
-        final BlockingQueue<IgniteFuture> q = new ArrayBlockingQueue<>(400);
+        final BlockingQueue<IgniteInternalFuture> q = new ArrayBlockingQueue<>(400);
 
         long start = System.currentTimeMillis();
 
@@ -93,12 +93,12 @@ public class GridTestMain {
                 }
             });
 
-            final IgniteFuture<?> f = comp.future();
+            final IgniteInternalFuture<?> f = comp.future();
 
             q.put(f);
 
-            f.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> o) {
+            f.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> o) {
                     q.poll();
                 }
             });


[04/50] [abbrv] incubator-ignite git commit: ignite-91

Posted by vo...@apache.org.
ignite-91


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

Branch: refs/heads/ignite-16
Commit: 27827c8217545686e8c5132e0f29dd1aecc19f3b
Parents: 9996140
Author: sboikov <sb...@gridgain.com>
Authored: Wed Jan 28 18:09:48 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Wed Jan 28 18:09:48 2015 +0300

----------------------------------------------------------------------
 .../processors/cache/GridCacheAdapter.java      | 26 +++++++++++++-------
 .../dht/GridPartitionedGetFuture.java           | 11 +++++++--
 .../dht/atomic/GridDhtAtomicCache.java          | 21 ++++++++++++++++
 .../dht/atomic/GridNearAtomicUpdateFuture.java  | 17 +++++++++++--
 .../dht/colocated/GridDhtColocatedCache.java    |  3 +++
 .../distributed/near/GridNearGetFuture.java     | 11 +++++++--
 .../local/atomic/GridLocalAtomicCache.java      | 12 ++++++---
 .../cache/GridCacheAbstractFullApiSelfTest.java | 12 ++++++---
 8 files changed, 91 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27827c82/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 14ddae1..4f401c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -1681,12 +1681,14 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Override public void evictAll(@Nullable Collection<? extends K> keys) {
+    @Override public void evictAll(Collection<? extends K> keys) {
         evictAll(keys, (IgnitePredicate<CacheEntry<K, V>>[])null);
     }
 
     /** {@inheritDoc} */
     @Nullable @Override public V get(K key) throws IgniteCheckedException {
+        A.notNull(key, "key");
+
         boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         long start = statsEnabled ? System.nanoTime() : 0L;
@@ -1704,6 +1706,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<V> getAsync(final K key) {
+        A.notNull(key, "key");
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -1725,6 +1729,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
     /** {@inheritDoc} */
     @Override public Map<K, V> getAll(@Nullable Collection<? extends K> keys) throws IgniteCheckedException {
+        A.notNull(keys, "keys");
+
         boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         long start = statsEnabled ? System.nanoTime() : 0L;
@@ -1742,6 +1748,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Map<K, V>> getAllAsync(@Nullable final Collection<? extends K> keys) {
+        A.notNull(keys, "keys");
+
         final boolean statsEnabled = ctx.config().isStatisticsEnabled();
 
         final long start = statsEnabled ? System.nanoTime() : 0L;
@@ -1879,9 +1887,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
                 Map<K, GridCacheVersion> misses = null;
 
                 for (K key : keys) {
-                    // Ignore null keys.
                     if (key == null)
-                        continue;
+                        throw new NullPointerException("Null key.");
 
                     while (true) {
                         GridCacheEntryEx<K, V> entry;
@@ -2206,14 +2213,12 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
         ctx.denyOnLocalRead();
 
         return ctx.wrapClone(asyncOp(new AsyncOp<V>(key) {
-            @Override
-            public IgniteFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
+            @Override public IgniteFuture<V> op(IgniteTxLocalAdapter<K, V> tx) {
                 return tx.putAllAsync(ctx, F.t(key, val), true, entry, ttl, filter)
                     .chain((IgniteClosure<IgniteFuture<GridCacheReturn<V>>, V>)RET2VAL);
             }
 
-            @Override
-            public String toString() {
+            @Override public String toString() {
                 return "putAsync [key=" + key + ", val=" + val + ", filter=" + Arrays.toString(filter) + ']';
             }
         }));
@@ -2664,8 +2669,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
     }
 
     /** {@inheritDoc} */
-    @Nullable
-    @Override public V replace(final K key, final V val) throws IgniteCheckedException {
+    @Nullable @Override public V replace(final K key, final V val) throws IgniteCheckedException {
         A.notNull(key, "key", val, "val");
 
         if (keyCheck)
@@ -3098,6 +3102,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> removexAsync(K key, IgnitePredicate<CacheEntry<K, V>>... filter) {
+        A.notNull(key, "key");
+
         return removexAsync(key, null, filter);
     }
 
@@ -4571,6 +4577,8 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
      */
     public void evictAll(Collection<? extends K> keys,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+        A.notNull(keys, "keys");
+
         ctx.denyOnFlag(READ);
 
         if (F.isEmpty(keys))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27827c82/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
index 23e487f..7227026 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridPartitionedGetFuture.java
@@ -290,8 +290,15 @@ public class GridPartitionedGetFuture<K, V> extends GridCompoundIdentityFuture<M
 
         // Assign keys to primary nodes.
         for (K key : keys) {
-            if (key != null)
-                hasRmtNodes |= map(key, mappings, locVals, topVer, mapped);
+            if (key == null) {
+                NullPointerException err = new NullPointerException("Null key");
+
+                onDone(err);
+
+                throw err;
+            }
+
+            hasRmtNodes |= map(key, mappings, locVals, topVer, mapped);
         }
 
         if (isDone())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27827c82/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index c993397..e35c6d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -309,6 +309,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @SuppressWarnings("unchecked")
     @Override public IgniteFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry,
         long ttl, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+        A.notNull(key, "key");
+
         return updateAllAsync0(F0.asMap(key, val),
             null,
             null,
@@ -324,6 +326,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @SuppressWarnings("unchecked")
     @Override public IgniteFuture<Boolean> putxAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+        A.notNull(key, "key");
+
         return updateAllAsync0(F0.asMap(key, val),
             null,
             null,
@@ -342,6 +346,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<V> putIfAbsentAsync(K key, V val) {
+        A.notNull(key, "key", val, "val");
+
         return putAsync(key, val, ctx.noPeekArray());
     }
 
@@ -352,6 +358,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> putxIfAbsentAsync(K key, V val) {
+        A.notNull(key, "key", val, "val");
+
         return putxAsync(key, val, ctx.noPeekArray());
     }
 
@@ -362,6 +370,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<V> replaceAsync(K key, V val) {
+        A.notNull(key, "key", val, "val");
+
         return putAsync(key, val, ctx.hasPeekArray());
     }
 
@@ -372,6 +382,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> replacexAsync(K key, V val) {
+        A.notNull(key, "key", val, "val");
+
         return putxAsync(key, val, ctx.hasPeekArray());
     }
 
@@ -382,6 +394,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
+        A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
+
         return putxAsync(key, newVal, ctx.equalsPeekArray(oldVal));
     }
 
@@ -398,6 +412,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
+        A.notNull(key, "key", val, "val");
+
         return removeAllAsync0(F.asList(key), null, null, true, true, ctx.equalsPeekArray(val));
     }
 
@@ -492,6 +508,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @SuppressWarnings("unchecked")
     @Override public IgniteFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+        A.notNull(key, "key");
+
         return removeAllAsync0(Collections.singletonList(key), null, entry, false, false, filter);
     }
 
@@ -874,6 +892,9 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
             // Optimistically expect that all keys are available locally (avoid creation of get future).
             for (K key : keys) {
+                if (key == null)
+                    throw new NullPointerException("Null key.");
+
                 GridCacheEntryEx<K, V> entry = null;
 
                 while (true) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27827c82/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index d55c001..6f6f03b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -626,8 +626,13 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
 
             // Create mappings first, then send messages.
             for (K key : keys) {
-                if (key == null)
-                    continue;
+                if (key == null) {
+                    NullPointerException err = new NullPointerException("Null key.");
+
+                    onDone(err);
+
+                    throw err;
+                }
 
                 Object val;
                 long drTtl;
@@ -639,6 +644,14 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
                     drTtl = -1;
                     drExpireTime = -1;
                     drVer = null;
+
+                    if (val == null) {
+                        NullPointerException err = new NullPointerException("Null value.");
+
+                        onDone(err);
+
+                        throw err;
+                    }
                 }
                 else if (drPutVals != null) {
                     GridCacheDrInfo<V> drPutVal =  drPutValsIt.next();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27827c82/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
index ae582cc..668f6fe 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/colocated/GridDhtColocatedCache.java
@@ -265,6 +265,9 @@ public class GridDhtColocatedCache<K, V> extends GridDhtTransactionalCacheAdapte
 
             // Optimistically expect that all keys are available locally (avoid creation of get future).
             for (K key : keys) {
+                if (key == null)
+                    throw new NullPointerException("Null key.");
+
                 GridCacheEntryEx<K, V> entry = null;
 
                 while (true) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27827c82/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
index 5500ba3..7eb5013 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/near/GridNearGetFuture.java
@@ -295,8 +295,15 @@ public final class GridNearGetFuture<K, V> extends GridCompoundIdentityFuture<Ma
 
         // Assign keys to primary nodes.
         for (K key : keys) {
-            if (key != null)
-                savedVers = map(key, mappings, topVer, mapped, savedVers);
+            if (key == null) {
+                NullPointerException err = new NullPointerException("Null key.");
+
+                onDone(err);
+
+                throw err;
+            }
+
+            savedVers = map(key, mappings, topVer, mapped, savedVers);
         }
 
         if (isDone())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27827c82/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 2c4a774..2c360ba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -259,7 +259,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public GridCacheReturn<V> replacex(K key, V oldVal, V newVal) throws IgniteCheckedException {
-        A.notNull(key, "key");
+        A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
 
         ctx.denyOnLocalRead();
 
@@ -277,7 +277,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public GridCacheReturn<V> removex(K key, V val) throws IgniteCheckedException {
-        A.notNull(key, "key");
+        A.notNull(key, "key", val, "val");
 
         ctx.denyOnLocalRead();
 
@@ -295,7 +295,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
-        A.notNull(key, "key");
+        A.notNull(key, "key", val, "val");
 
         ctx.denyOnLocalRead();
 
@@ -305,7 +305,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public IgniteFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
-        A.notNull(key, "key");
+        A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
 
         ctx.denyOnLocalRead();
 
@@ -500,6 +500,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         throws IgniteCheckedException {
         ctx.denyOnFlag(LOCAL);
 
+        A.notNull(keys, "keys");
+
         String taskName = ctx.kernalContext().job().currentTaskName();
 
         return getAllInternal(keys,
@@ -526,6 +528,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     ) {
         ctx.denyOnFlag(LOCAL);
 
+        A.notNull(keys, "keys");
+
         final boolean swapOrOffheap = ctx.isSwapOrOffheapEnabled();
         final boolean storeEnabled = ctx.readThrough();
         final boolean clone = ctx.hasFlag(CLONE);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/27827c82/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 e7b93bf..905f6b4 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
@@ -525,14 +525,20 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception In case of error.
      */
     public void testGetAllWithNulls() throws Exception {
-        GridCache<String, Integer> cache = cache();
+        final GridCache<String, Integer> cache = cache();
 
-        Collection<String> c = new LinkedList<>();
+        final Collection<String> c = new LinkedList<>();
 
         c.add("key1");
         c.add(null);
 
-        cache.getAll(c);
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.getAll(c);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
     }
 
     /**


[38/50] [abbrv] incubator-ignite git commit: GridEx -> IgniteEx GridKernal -> IgniteKernal

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
new file mode 100644
index 0000000..7854bd9
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -0,0 +1,3322 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cache.*;
+import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.processors.*;
+import org.apache.ignite.internal.processors.cache.*;
+import org.apache.ignite.internal.processors.portable.*;
+import org.apache.ignite.internal.util.*;
+import org.apache.ignite.lang.*;
+import org.apache.ignite.lifecycle.*;
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.mxbean.*;
+import org.apache.ignite.plugin.*;
+import org.apache.ignite.internal.product.*;
+import org.apache.ignite.spi.*;
+import org.apache.ignite.spi.authentication.*;
+import org.apache.ignite.spi.authentication.noop.*;
+import org.apache.ignite.hadoop.*;
+import org.apache.ignite.internal.managers.*;
+import org.apache.ignite.internal.managers.checkpoint.*;
+import org.apache.ignite.internal.managers.collision.*;
+import org.apache.ignite.internal.managers.communication.*;
+import org.apache.ignite.internal.managers.deployment.*;
+import org.apache.ignite.internal.managers.discovery.*;
+import org.apache.ignite.internal.managers.eventstorage.*;
+import org.apache.ignite.internal.managers.failover.*;
+import org.apache.ignite.internal.managers.indexing.*;
+import org.apache.ignite.internal.managers.loadbalancer.*;
+import org.apache.ignite.internal.managers.securesession.*;
+import org.apache.ignite.internal.managers.security.*;
+import org.apache.ignite.internal.managers.swapspace.*;
+import org.apache.ignite.internal.processors.affinity.*;
+import org.apache.ignite.internal.processors.clock.*;
+import org.apache.ignite.internal.processors.closure.*;
+import org.apache.ignite.internal.processors.continuous.*;
+import org.apache.ignite.internal.processors.dataload.*;
+import org.apache.ignite.internal.processors.email.*;
+import org.apache.ignite.internal.processors.interop.*;
+import org.apache.ignite.internal.processors.job.*;
+import org.apache.ignite.internal.processors.jobmetrics.*;
+import org.apache.ignite.internal.processors.license.*;
+import org.apache.ignite.internal.processors.offheap.*;
+import org.apache.ignite.internal.processors.plugin.*;
+import org.apache.ignite.internal.processors.port.*;
+import org.apache.ignite.internal.processors.query.*;
+import org.apache.ignite.internal.processors.resource.*;
+import org.apache.ignite.internal.processors.rest.*;
+import org.apache.ignite.internal.processors.segmentation.*;
+import org.apache.ignite.internal.processors.service.*;
+import org.apache.ignite.internal.processors.session.*;
+import org.apache.ignite.internal.processors.streamer.*;
+import org.apache.ignite.internal.processors.task.*;
+import org.apache.ignite.internal.processors.timeout.*;
+import org.apache.ignite.plugin.security.*;
+import org.apache.ignite.spi.securesession.noop.*;
+import org.apache.ignite.internal.util.future.*;
+import org.apache.ignite.internal.util.lang.*;
+import org.apache.ignite.internal.util.nodestart.*;
+import org.apache.ignite.internal.util.tostring.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.jetbrains.annotations.*;
+
+import javax.management.*;
+import java.io.*;
+import java.lang.management.*;
+import java.lang.reflect.*;
+import java.net.*;
+import java.text.*;
+import java.util.*;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.*;
+
+import static org.apache.ignite.internal.GridKernalState.*;
+import static org.apache.ignite.lifecycle.LifecycleEventType.*;
+import static org.apache.ignite.IgniteSystemProperties.*;
+import static org.apache.ignite.internal.IgniteComponentType.*;
+import static org.apache.ignite.internal.GridNodeAttributes.*;
+import static org.apache.ignite.internal.GridProductImpl.*;
+import static org.apache.ignite.internal.processors.license.GridLicenseSubsystem.*;
+import static org.apache.ignite.internal.util.nodestart.GridNodeStartUtils.*;
+
+/**
+ * GridGain kernal.
+ * <p/>
+ * See <a href="http://en.wikipedia.org/wiki/Kernal">http://en.wikipedia.org/wiki/Kernal</a> for information on the
+ * misspelling.
+ */
+public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, IgniteMXBean {
+    /** */
+    private static final long serialVersionUID = 0L;
+
+    /** Compatible versions. */
+    private static final String COMPATIBLE_VERS = GridProperties.get("gridgain.compatible.vers");
+
+    /** GridGain site that is shown in log messages. */
+    static final String SITE = "www.gridgain.com";
+
+    /** System line separator. */
+    private static final String NL = U.nl();
+
+    /** Periodic version check delay. */
+    private static final long PERIODIC_VER_CHECK_DELAY = 1000 * 60 * 60; // Every hour.
+
+    /** Periodic version check delay. */
+    private static final long PERIODIC_VER_CHECK_CONN_TIMEOUT = 10 * 1000; // 10 seconds.
+
+    /** Periodic version check delay. */
+    private static final long PERIODIC_LIC_CHECK_DELAY = 1000 * 60; // Every minute.
+
+    /** Periodic starvation check interval. */
+    private static final long PERIODIC_STARVATION_CHECK_FREQ = 1000 * 30;
+
+    /** Shutdown delay in msec. when license violation detected. */
+    private static final int SHUTDOWN_DELAY = 60 * 1000;
+
+    /** */
+    private IgniteConfiguration cfg;
+
+    /** */
+    @SuppressWarnings({"FieldAccessedSynchronizedAndUnsynchronized"})
+    @GridToStringExclude
+    private GridLoggerProxy log;
+
+    /** */
+    private String gridName;
+
+    /** */
+    @GridToStringExclude
+    private ObjectName kernalMBean;
+
+    /** */
+    @GridToStringExclude
+    private ObjectName locNodeMBean;
+
+    /** */
+    @GridToStringExclude
+    private ObjectName pubExecSvcMBean;
+
+    /** */
+    @GridToStringExclude
+    private ObjectName sysExecSvcMBean;
+
+    /** */
+    @GridToStringExclude
+    private ObjectName mgmtExecSvcMBean;
+
+    /** */
+    @GridToStringExclude
+    private ObjectName p2PExecSvcMBean;
+
+    /** */
+    @GridToStringExclude
+    private ObjectName restExecSvcMBean;
+
+    /** Kernal start timestamp. */
+    private long startTime = U.currentTimeMillis();
+
+    /** Spring context, potentially {@code null}. */
+    private GridSpringResourceContext rsrcCtx;
+
+    /** */
+    @GridToStringExclude
+    private Timer updateNtfTimer;
+
+    /** */
+    @GridToStringExclude
+    private Timer starveTimer;
+
+    /** */
+    @GridToStringExclude
+    private Timer licTimer;
+
+    /** */
+    @GridToStringExclude
+    private Timer metricsLogTimer;
+
+    /** Indicate error on grid stop. */
+    @GridToStringExclude
+    private boolean errOnStop;
+
+    /** Node local store. */
+    @GridToStringExclude
+    private ClusterNodeLocalMap nodeLoc;
+
+    /** Scheduler. */
+    @GridToStringExclude
+    private IgniteScheduler scheduler;
+
+    /** Grid security instance. */
+    @GridToStringExclude
+    private GridSecurity security;
+
+    /** Portables instance. */
+    @GridToStringExclude
+    private IgnitePortables portables;
+
+    /** Kernal gateway. */
+    @GridToStringExclude
+    private final AtomicReference<GridKernalGateway> gw = new AtomicReference<>();
+
+    /** Data Grid edition usage registered flag. */
+    @GridToStringExclude
+    private volatile boolean dbUsageRegistered;
+
+    /** */
+    @GridToStringExclude
+    private final Collection<String> compatibleVers;
+
+    /** Stop guard. */
+    @GridToStringExclude
+    private final AtomicBoolean stopGuard = new AtomicBoolean();
+
+    /**
+     * No-arg constructor is required by externalization.
+     */
+    public IgniteKernal() {
+        this(null);
+    }
+
+    /**
+     * @param rsrcCtx Optional Spring application context.
+     */
+    public IgniteKernal(@Nullable GridSpringResourceContext rsrcCtx) {
+        super(null, null, null, (IgnitePredicate<ClusterNode>)null);
+
+        this.rsrcCtx = rsrcCtx;
+
+        String[] compatibleVers = COMPATIBLE_VERS.split(",");
+
+        for (int i = 0; i < compatibleVers.length; i++)
+            compatibleVers[i] = compatibleVers[i].trim();
+
+        this.compatibleVers = Collections.unmodifiableList(Arrays.asList(compatibleVers));
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteCluster cluster() {
+        return this;
+    }
+
+    /** {@inheritDoc} */
+    @Override public final IgniteCompute compute(ClusterGroup prj) {
+        return ((ClusterGroupAdapter)prj).compute();
+    }
+
+    /** {@inheritDoc} */
+    @Override public final IgniteMessaging message(ClusterGroup prj) {
+        return ((ClusterGroupAdapter)prj).message();
+    }
+
+    /** {@inheritDoc} */
+    @Override public final IgniteEvents events(ClusterGroup prj) {
+        return ((ClusterGroupAdapter)prj).events();
+    }
+
+    /** {@inheritDoc} */
+    @Override public IgniteManaged managed(ClusterGroup prj) {
+        return ((ClusterGroupAdapter)prj).managed();
+    }
+
+    /** {@inheritDoc} */
+    @Override public ExecutorService executorService(ClusterGroup prj) {
+        return ((ClusterGroupAdapter)prj).executorService();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String name() {
+        return gridName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCopyright() {
+        return ctx.product().copyright();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getLicenseFilePath() {
+        assert cfg != null;
+
+        return cfg.getLicenseUrl();
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getStartTimestamp() {
+        return startTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getStartTimestampFormatted() {
+        return DateFormat.getDateTimeInstance().format(new Date(startTime));
+    }
+
+    /** {@inheritDoc} */
+    @Override public long getUpTime() {
+        return U.currentTimeMillis() - startTime;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getUpTimeFormatted() {
+        return X.timeSpan2HMSM(U.currentTimeMillis() - startTime);
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getFullVersion() {
+        return COMPOUND_VER + '-' + BUILD_TSTAMP_STR;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCheckpointSpiFormatted() {
+        assert cfg != null;
+
+        return Arrays.toString(cfg.getCheckpointSpi());
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSwapSpaceSpiFormatted() {
+        assert cfg != null;
+
+        return cfg.getSwapSpaceSpi().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCommunicationSpiFormatted() {
+        assert cfg != null;
+
+        return cfg.getCommunicationSpi().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDeploymentSpiFormatted() {
+        assert cfg != null;
+
+        return cfg.getDeploymentSpi().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getDiscoverySpiFormatted() {
+        assert cfg != null;
+
+        return cfg.getDiscoverySpi().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getEventStorageSpiFormatted() {
+        assert cfg != null;
+
+        return cfg.getEventStorageSpi().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getCollisionSpiFormatted() {
+        assert cfg != null;
+
+        return cfg.getCollisionSpi().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getFailoverSpiFormatted() {
+        assert cfg != null;
+
+        return Arrays.toString(cfg.getFailoverSpi());
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getLoadBalancingSpiFormatted() {
+        assert cfg != null;
+
+        return Arrays.toString(cfg.getLoadBalancingSpi());
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getAuthenticationSpiFormatted() {
+        assert cfg != null;
+
+        return cfg.getAuthenticationSpi().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getSecureSessionSpiFormatted() {
+        assert cfg != null;
+
+        return cfg.getSecureSessionSpi().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getOsInformation() {
+        return U.osString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getJdkInformation() {
+        return U.jdkString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getOsUser() {
+        return System.getProperty("user.name");
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getVmName() {
+        return ManagementFactory.getRuntimeMXBean().getName();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getInstanceName() {
+        return gridName;
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getExecutorServiceFormatted() {
+        assert cfg != null;
+
+        return cfg.getExecutorService().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getGridGainHome() {
+        assert cfg != null;
+
+        return cfg.getGridGainHome();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getGridLoggerFormatted() {
+        assert cfg != null;
+
+        return cfg.getGridLogger().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public String getMBeanServerFormatted() {
+        assert cfg != null;
+
+        return cfg.getMBeanServer().toString();
+    }
+
+    /** {@inheritDoc} */
+    @Override public UUID getLocalNodeId() {
+        assert cfg != null;
+
+        return cfg.getNodeId();
+    }
+
+    /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
+    @Override public Collection<String> getUserAttributesFormatted() {
+        assert cfg != null;
+
+        return F.transform(cfg.getUserAttributes().entrySet(), new C1<Map.Entry<String, ?>, String>() {
+            @Override public String apply(Map.Entry<String, ?> e) {
+                return e.getKey() + ", " + e.getValue().toString();
+            }
+        });
+    }
+
+    /** {@inheritDoc} */
+    @Override public boolean isPeerClassLoadingEnabled() {
+        assert cfg != null;
+
+        return cfg.isPeerClassLoadingEnabled();
+    }
+
+    /** {@inheritDoc} */
+    @Override public Collection<String> getLifecycleBeansFormatted() {
+        LifecycleBean[] beans = cfg.getLifecycleBeans();
+
+        return F.isEmpty(beans) ? Collections.<String>emptyList() : F.transform(beans, F.<LifecycleBean>string());
+    }
+
+    /**
+     * @param attrs Current attributes.
+     * @param name  New attribute name.
+     * @param val New attribute value.
+     * @throws IgniteCheckedException If duplicated SPI name found.
+     */
+    private void add(Map<String, Object> attrs, String name, @Nullable Serializable val) throws IgniteCheckedException {
+        assert attrs != null;
+        assert name != null;
+
+        if (attrs.put(name, val) != null) {
+            if (name.endsWith(ATTR_SPI_CLASS))
+                // User defined duplicated names for the different SPIs.
+                throw new IgniteCheckedException("Failed to set SPI attribute. Duplicated SPI name found: " +
+                    name.substring(0, name.length() - ATTR_SPI_CLASS.length()));
+
+            // Otherwise it's a mistake of setting up duplicated attribute.
+            assert false : "Duplicate attribute: " + name;
+        }
+    }
+
+    /**
+     * Notifies life-cycle beans of grid event.
+     *
+     * @param evt Grid event.
+     * @throws IgniteCheckedException If user threw exception during start.
+     */
+    @SuppressWarnings({"CatchGenericClass"})
+    private void notifyLifecycleBeans(LifecycleEventType evt) throws IgniteCheckedException {
+        if (!cfg.isDaemon() && cfg.getLifecycleBeans() != null)
+            for (LifecycleBean bean : cfg.getLifecycleBeans())
+                if (bean != null)
+                    bean.onLifecycleEvent(evt);
+    }
+
+    /**
+     * Notifies life-cycle beans of grid event.
+     *
+     * @param evt Grid event.
+     */
+    @SuppressWarnings({"CatchGenericClass"})
+    private void notifyLifecycleBeansEx(LifecycleEventType evt) {
+        try {
+            notifyLifecycleBeans(evt);
+        }
+        // Catch generic throwable to secure against user assertions.
+        catch (Throwable e) {
+            U.error(log, "Failed to notify lifecycle bean (safely ignored) [evt=" + evt +
+                ", gridName=" + gridName + ']', e);
+        }
+    }
+
+    /**
+     * @param cfg Grid configuration to use.
+     * @param utilityCachePool Utility cache pool.
+     * @param errHnd Error handler to use for notification about startup problems.
+     * @throws IgniteCheckedException Thrown in case of any errors.
+     */
+    @SuppressWarnings({"CatchGenericClass", "unchecked"})
+    public void start(final IgniteConfiguration cfg, ExecutorService utilityCachePool, GridAbsClosure errHnd)
+        throws IgniteCheckedException {
+        gw.compareAndSet(null, new GridKernalGatewayImpl(cfg.getGridName()));
+
+        GridKernalGateway gw = this.gw.get();
+
+        gw.writeLock();
+
+        try {
+            switch (gw.getState()) {
+                case STARTED: {
+                    U.warn(log, "Grid has already been started (ignored).");
+
+                    return;
+                }
+
+                case STARTING: {
+                    U.warn(log, "Grid is already in process of being started (ignored).");
+
+                    return;
+                }
+
+                case STOPPING: {
+                    throw new IgniteCheckedException("Grid is in process of being stopped");
+                }
+
+                case STOPPED: {
+                    break;
+                }
+            }
+
+            gw.setState(STARTING);
+        }
+        finally {
+            gw.writeUnlock();
+        }
+
+        assert cfg != null;
+
+        // Make sure we got proper configuration.
+        validateCommon(cfg);
+
+        gridName = cfg.getGridName();
+
+        this.cfg = cfg;
+
+        log = (GridLoggerProxy)cfg.getGridLogger().getLogger(getClass().getName() +
+            (gridName != null ? '%' + gridName : ""));
+
+        RuntimeMXBean rtBean = ManagementFactory.getRuntimeMXBean();
+
+        // Ack various information.
+        ackAsciiLogo();
+        ackConfigUrl();
+        ackDaemon();
+        ackOsInfo();
+        ackLanguageRuntime();
+        ackRemoteManagement();
+        ackVmArguments(rtBean);
+        ackClassPaths(rtBean);
+        ackSystemProperties();
+        ackEnvironmentVariables();
+        ackSmtpConfiguration();
+        ackCacheConfiguration();
+        ackP2pConfiguration();
+
+        // Run background network diagnostics.
+        GridDiagnostic.runBackgroundCheck(gridName, cfg.getExecutorService(), log);
+
+        boolean notifyEnabled = IgniteSystemProperties.getBoolean(GG_UPDATE_NOTIFIER, true);
+
+        GridUpdateNotifier verChecker0 = null;
+
+        if (notifyEnabled) {
+            try {
+                verChecker0 = new GridUpdateNotifier(gridName, VER, SITE, gw, false);
+
+                verChecker0.checkForNewVersion(cfg.getExecutorService(), log);
+            }
+            catch (IgniteCheckedException e) {
+                if (log.isDebugEnabled())
+                    log.debug("Failed to create GridUpdateNotifier: " + e);
+            }
+        }
+
+        final GridUpdateNotifier verChecker = verChecker0;
+
+        // Ack 3-rd party licenses location.
+        if (log.isInfoEnabled() && cfg.getGridGainHome() != null)
+            log.info("3-rd party licenses can be found at: " + cfg.getGridGainHome() + File.separatorChar + "libs" +
+                File.separatorChar + "licenses");
+
+        // Check that user attributes are not conflicting
+        // with internally reserved names.
+        for (String name : cfg.getUserAttributes().keySet())
+            if (name.startsWith(ATTR_PREFIX))
+                throw new IgniteCheckedException("User attribute has illegal name: '" + name + "'. Note that all names " +
+                    "starting with '" + ATTR_PREFIX + "' are reserved for internal use.");
+
+        // Ack local node user attributes.
+        logNodeUserAttributes();
+
+        // Ack configuration.
+        ackSpis();
+
+        Map<String, Object> attrs = createNodeAttributes(cfg, BUILD_TSTAMP_STR);
+
+        // Spin out SPIs & managers.
+        try {
+            GridKernalContextImpl ctx =
+                new GridKernalContextImpl(log, this, cfg, gw, utilityCachePool, ENT);
+
+            nodeLoc = new ClusterNodeLocalMapImpl(ctx);
+
+            U.onGridStart();
+
+            // Set context into rich adapter.
+            setKernalContext(ctx);
+
+            // Start and configure resource processor first as it contains resources used
+            // by all other managers and processors.
+            GridResourceProcessor rsrcProc = new GridResourceProcessor(ctx);
+
+            rsrcProc.setSpringContext(rsrcCtx);
+
+            ctx.product(new GridProductImpl(ctx, verChecker));
+
+            scheduler = new IgniteSchedulerImpl(ctx);
+
+            startProcessor(ctx, rsrcProc, attrs);
+
+            // Inject resources into lifecycle beans.
+            if (!cfg.isDaemon() && cfg.getLifecycleBeans() != null)
+                for (LifecycleBean bean : cfg.getLifecycleBeans())
+                    if (bean != null)
+                        rsrcProc.inject(bean);
+
+            // Lifecycle notification.
+            notifyLifecycleBeans(BEFORE_GRID_START);
+
+            // Starts lifecycle aware components.
+            U.startLifecycleAware(lifecycleAwares(cfg));
+
+            addHelper(ctx, GGFS_HELPER.create(F.isEmpty(cfg.getGgfsConfiguration())));
+
+            startProcessor(ctx, new IgnitePluginProcessor(ctx, cfg), attrs);
+
+            // Off-heap processor has no dependencies.
+            startProcessor(ctx, new GridOffHeapProcessor(ctx), attrs);
+
+            // Closure processor should be started before all others
+            // (except for resource processor), as many components can depend on it.
+            startProcessor(ctx, new GridClosureProcessor(ctx), attrs);
+
+            // Start some other processors (order & place is important).
+            startProcessor(ctx, (GridProcessor)EMAIL.create(ctx, cfg.getSmtpHost() == null), attrs);
+            startProcessor(ctx, new GridPortProcessor(ctx), attrs);
+            startProcessor(ctx, new GridJobMetricsProcessor(ctx), attrs);
+
+            // Timeout processor needs to be started before managers,
+            // as managers may depend on it.
+            startProcessor(ctx, new GridTimeoutProcessor(ctx), attrs);
+
+            // Start SPI managers.
+            // NOTE: that order matters as there are dependencies between managers.
+            startManager(ctx, createComponent(GridSecurityManager.class, ctx), attrs);
+            startManager(ctx, createComponent(GridSecureSessionManager.class, ctx), attrs);
+            startManager(ctx, new GridIoManager(ctx), attrs);
+            startManager(ctx, new GridCheckpointManager(ctx), attrs);
+
+            startManager(ctx, new GridEventStorageManager(ctx), attrs);
+            startManager(ctx, new GridDeploymentManager(ctx), attrs);
+            startManager(ctx, new GridLoadBalancerManager(ctx), attrs);
+            startManager(ctx, new GridFailoverManager(ctx), attrs);
+            startManager(ctx, new GridCollisionManager(ctx), attrs);
+            startManager(ctx, new GridSwapSpaceManager(ctx), attrs);
+            startManager(ctx, new GridIndexingManager(ctx), attrs);
+
+            ackSecurity(ctx);
+
+            // Start processors before discovery manager, so they will
+            // be able to start receiving messages once discovery completes.
+            startProcessor(ctx, new GridClockSyncProcessor(ctx), attrs);
+            startProcessor(ctx, createComponent(GridLicenseProcessor.class, ctx), attrs);
+            startProcessor(ctx, new GridAffinityProcessor(ctx), attrs);
+            startProcessor(ctx, createComponent(GridSegmentationProcessor.class, ctx), attrs);
+            startProcessor(ctx, new GridQueryProcessor(ctx), attrs);
+            startProcessor(ctx, new GridCacheProcessor(ctx), attrs);
+            startProcessor(ctx, new GridTaskSessionProcessor(ctx), attrs);
+            startProcessor(ctx, new GridJobProcessor(ctx), attrs);
+            startProcessor(ctx, new GridTaskProcessor(ctx), attrs);
+            startProcessor(ctx, (GridProcessor)SCHEDULE.createOptional(ctx), attrs);
+            startProcessor(ctx, createComponent(GridPortableProcessor.class, ctx), attrs);
+            startProcessor(ctx, createComponent(GridInteropProcessor.class, ctx), attrs);
+            startProcessor(ctx, new GridRestProcessor(ctx), attrs);
+            startProcessor(ctx, new GridDataLoaderProcessor(ctx), attrs);
+            startProcessor(ctx, new GridStreamProcessor(ctx), attrs);
+            startProcessor(ctx, (GridProcessor)GGFS.create(ctx, F.isEmpty(cfg.getGgfsConfiguration())), attrs);
+            startProcessor(ctx, new GridContinuousProcessor(ctx), attrs);
+            startProcessor(ctx, (GridProcessor)(cfg.isPeerClassLoadingEnabled() ?
+                IgniteComponentType.HADOOP.create(ctx, true): // No-op when peer class loading is enabled.
+                IgniteComponentType.HADOOP.createIfInClassPath(ctx, cfg.getHadoopConfiguration() != null)), attrs);
+            startProcessor(ctx, new GridServiceProcessor(ctx), attrs);
+
+            // Start plugins.
+            for (PluginProvider provider : ctx.plugins().allProviders()) {
+                ctx.add(new GridPluginComponent(provider));
+
+                provider.start(ctx.plugins().pluginContextForProvider(provider), attrs);
+            }
+
+            ctx.createMessageFactory();
+
+            if (ctx.isEnterprise()) {
+                security = new GridSecurityImpl(ctx);
+                portables = new GridPortablesImpl(ctx);
+            }
+
+            gw.writeLock();
+
+            try {
+                gw.setState(STARTED);
+
+                // Start discovery manager last to make sure that grid is fully initialized.
+                startManager(ctx, new GridDiscoveryManager(ctx), attrs);
+            }
+            finally {
+                gw.writeUnlock();
+            }
+
+            // Check whether physical RAM is not exceeded.
+            checkPhysicalRam();
+
+            // Suggest configuration optimizations.
+            suggestOptimizations(ctx, cfg);
+
+            if (!ctx.isEnterprise())
+                warnNotSupportedFeaturesForOs(cfg);
+
+            // Notify discovery manager the first to make sure that topology is discovered.
+            ctx.discovery().onKernalStart();
+
+            // Notify IO manager the second so further components can send and receive messages.
+            ctx.io().onKernalStart();
+
+            // Callbacks.
+            for (GridComponent comp : ctx) {
+                // Skip discovery manager.
+                if (comp instanceof GridDiscoveryManager)
+                    continue;
+
+                // Skip IO manager.
+                if (comp instanceof GridIoManager)
+                    continue;
+
+                comp.onKernalStart();
+            }
+
+            // Ack the license.
+            ctx.license().ackLicense();
+
+            // Register MBeans.
+            registerKernalMBean();
+            registerLocalNodeMBean();
+            registerExecutorMBeans();
+
+            // Lifecycle bean notifications.
+            notifyLifecycleBeans(AFTER_GRID_START);
+        }
+        catch (Throwable e) {
+            IgniteSpiVersionCheckException verCheckErr = X.cause(e, IgniteSpiVersionCheckException.class);
+
+            if (verCheckErr != null)
+                U.error(log, verCheckErr.getMessage());
+            else if (X.hasCause(e, InterruptedException.class, IgniteInterruptedException.class))
+                U.warn(log, "Grid startup routine has been interrupted (will rollback).");
+            else
+                U.error(log, "Got exception while starting (will rollback startup routine).", e);
+
+            errHnd.apply();
+
+            stop(true);
+
+            if (e instanceof IgniteCheckedException)
+                throw (IgniteCheckedException)e;
+            else
+                throw new IgniteCheckedException(e);
+        }
+
+        // Mark start timestamp.
+        startTime = U.currentTimeMillis();
+
+        // Ack latest version information.
+        if (verChecker != null)
+            verChecker.reportStatus(log);
+
+        if (notifyEnabled) {
+            assert verChecker != null;
+
+            verChecker.reportOnlyNew(true);
+            verChecker.licenseProcessor(ctx.license());
+
+            updateNtfTimer = new Timer("gridgain-update-notifier-timer");
+
+            // Setup periodic version check.
+            updateNtfTimer.scheduleAtFixedRate(new GridTimerTask() {
+                @Override public void safeRun() throws InterruptedException {
+                    verChecker.topologySize(nodes().size());
+
+                    verChecker.checkForNewVersion(cfg.getExecutorService(), log);
+
+                    // Just wait for 10 secs.
+                    Thread.sleep(PERIODIC_VER_CHECK_CONN_TIMEOUT);
+
+                    // Report status if one is available.
+                    // No-op if status is NOT available.
+                    verChecker.reportStatus(log);
+                }
+            }, PERIODIC_VER_CHECK_DELAY, PERIODIC_VER_CHECK_DELAY);
+        }
+
+        String intervalStr = IgniteSystemProperties.getString(GG_STARVATION_CHECK_INTERVAL);
+
+        // Start starvation checker if enabled.
+        boolean starveCheck = !isDaemon() && !"0".equals(intervalStr);
+
+        if (starveCheck) {
+            final long interval = F.isEmpty(intervalStr) ? PERIODIC_STARVATION_CHECK_FREQ : Long.parseLong(intervalStr);
+
+            starveTimer = new Timer("gridgain-starvation-checker");
+
+            starveTimer.scheduleAtFixedRate(new GridTimerTask() {
+                /** Last completed task count. */
+                private long lastCompletedCnt;
+
+                @Override protected void safeRun() {
+                    ExecutorService e = cfg.getExecutorService();
+
+                    if (!(e instanceof ThreadPoolExecutor))
+                        return;
+
+                    ThreadPoolExecutor exec = (ThreadPoolExecutor)e;
+
+                    long completedCnt = exec.getCompletedTaskCount();
+
+                    // If all threads are active and no task has completed since last time and there is
+                    // at least one waiting request, then it is possible starvation.
+                    if (exec.getPoolSize() == exec.getActiveCount() && completedCnt == lastCompletedCnt &&
+                        !exec.getQueue().isEmpty())
+                        LT.warn(log, null, "Possible thread pool starvation detected (no task completed in last " +
+                            interval + "ms, is executorService pool size large enough?)");
+
+                    lastCompletedCnt = completedCnt;
+                }
+            }, interval, interval);
+        }
+
+        if (!isDaemon()) {
+            licTimer = new Timer("gridgain-license-checker");
+
+            // Setup periodic license check.
+            licTimer.scheduleAtFixedRate(new GridTimerTask() {
+                @Override public void safeRun() throws InterruptedException {
+                    try {
+                        ctx.license().checkLicense();
+                    }
+                    // This exception only happens when license processor was unable
+                    // to resolve license violation on its own and this grid instance
+                    // now needs to be shutdown.
+                    //
+                    // Note that in most production configurations the license will
+                    // have certain grace period and license processor will attempt
+                    // to reload the license during the grace period.
+                    //
+                    // This exception thrown here means that grace period, if any,
+                    // has expired and license violation is still unresolved.
+                    catch (IgniteProductLicenseException ignored) {
+                        U.error(log, "License violation is unresolved. GridGain node will shutdown in " +
+                            (SHUTDOWN_DELAY / 1000) + " sec.");
+                        U.error(log, "  ^-- Contact your support for immediate assistance (!)");
+
+                        // Allow interruption to break from here since
+                        // node is stopping anyways.
+                        Thread.sleep(SHUTDOWN_DELAY);
+
+                        G.stop(gridName, true);
+                    }
+                    // Safety net.
+                    catch (Throwable e) {
+                        U.error(log, "Unable to check the license due to system error.", e);
+                        U.error(log, "Grid instance will be stopped...");
+
+                        // Stop the grid if we get unknown license-related error.
+                        // Should never happen. Practically an assertion...
+                        G.stop(gridName, true);
+                    }
+                }
+            }, PERIODIC_LIC_CHECK_DELAY, PERIODIC_LIC_CHECK_DELAY);
+        }
+
+        long metricsLogFreq = cfg.getMetricsLogFrequency();
+
+        if (metricsLogFreq > 0) {
+            metricsLogTimer = new Timer("gridgain-metrics-logger");
+
+            metricsLogTimer.scheduleAtFixedRate(new GridTimerTask() {
+                /** */
+                private final DecimalFormat dblFmt = new DecimalFormat("#.##");
+
+                @Override protected void safeRun() {
+                    if (log.isInfoEnabled()) {
+                        ClusterMetrics m = localNode().metrics();
+
+                        double cpuLoadPct = m.getCurrentCpuLoad() * 100;
+                        double avgCpuLoadPct = m.getAverageCpuLoad() * 100;
+                        double gcPct = m.getCurrentGcCpuLoad() * 100;
+
+                        long heapUsed = m.getHeapMemoryUsed();
+                        long heapMax = m.getHeapMemoryMaximum();
+
+                        long heapUsedInMBytes = heapUsed / 1024 / 1024;
+                        long heapCommInMBytes = m.getHeapMemoryCommitted() / 1024 / 1024;
+
+                        double freeHeapPct = heapMax > 0 ? ((double)((heapMax - heapUsed) * 100)) / heapMax : -1;
+
+                        int hosts = 0;
+                        int nodes = 0;
+                        int cpus = 0;
+
+                        try {
+                            ClusterMetrics metrics = metrics();
+
+                            Collection<ClusterNode> nodes0 = nodes();
+
+                            hosts = U.neighborhood(nodes0).size();
+                            nodes = nodes0.size();
+                            cpus = metrics.getTotalCpus();
+                        }
+                        catch (IgniteCheckedException ignore) {
+                            // No-op.
+                        }
+
+                        int pubPoolActiveThreads = 0;
+                        int pubPoolIdleThreads = 0;
+                        int pubPoolQSize = 0;
+
+                        ExecutorService pubExec = cfg.getExecutorService();
+
+                        if (pubExec instanceof ThreadPoolExecutor) {
+                            ThreadPoolExecutor exec = (ThreadPoolExecutor)pubExec;
+
+                            int poolSize = exec.getPoolSize();
+
+                            pubPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
+                            pubPoolIdleThreads = poolSize - pubPoolActiveThreads;
+                            pubPoolQSize = exec.getQueue().size();
+                        }
+
+                        int sysPoolActiveThreads = 0;
+                        int sysPoolIdleThreads = 0;
+                        int sysPoolQSize = 0;
+
+                        ExecutorService sysExec = cfg.getSystemExecutorService();
+
+                        if (sysExec instanceof ThreadPoolExecutor) {
+                            ThreadPoolExecutor exec = (ThreadPoolExecutor)sysExec;
+
+                            int poolSize = exec.getPoolSize();
+
+                            sysPoolActiveThreads = Math.min(poolSize, exec.getActiveCount());
+                            sysPoolIdleThreads = poolSize - sysPoolActiveThreads;
+                            sysPoolQSize = exec.getQueue().size();
+                        }
+
+                        String msg = NL +
+                            "Metrics for local node (to disable set 'metricsLogFrequency' to 0)" + NL +
+                            "    ^-- H/N/C [hosts=" + hosts + ", nodes=" + nodes + ", CPUs=" + cpus + "]" + NL +
+                            "    ^-- CPU [cur=" + dblFmt.format(cpuLoadPct) + "%, avg=" +
+                                dblFmt.format(avgCpuLoadPct) + "%, GC=" + dblFmt.format(gcPct) + "%]" + NL +
+                            "    ^-- Heap [used=" + dblFmt.format(heapUsedInMBytes) + "MB, free=" +
+                                dblFmt.format(freeHeapPct) + "%, comm=" + dblFmt.format(heapCommInMBytes) + "MB]" + NL +
+                            "    ^-- Public thread pool [active=" + pubPoolActiveThreads + ", idle=" +
+                                pubPoolIdleThreads + ", qSize=" + pubPoolQSize + "]" + NL +
+                            "    ^-- System thread pool [active=" + sysPoolActiveThreads + ", idle=" +
+                                sysPoolIdleThreads + ", qSize=" + sysPoolQSize + "]" + NL +
+                            "    ^-- Outbound messages queue [size=" + m.getOutboundMessagesQueueSize() + "]";
+
+                        log.info(msg);
+                    }
+                }
+            }, metricsLogFreq, metricsLogFreq);
+        }
+
+        ctx.performance().logSuggestions(log, gridName);
+
+        ackBenchmarks();
+        ackVisor();
+
+        ackStart(rtBean);
+
+        if (!isDaemon())
+            ctx.discovery().ackTopology();
+
+        // Send node start email notification, if enabled.
+        if (isSmtpEnabled() && isAdminEmailsSet() && cfg.isLifeCycleEmailNotification()) {
+            SB sb = new SB();
+
+            for (GridPortRecord rec : ctx.ports().records())
+                sb.a(rec.protocol()).a(":").a(rec.port()).a(" ");
+
+            String nid = localNode().id().toString().toUpperCase();
+            String nid8 = U.id8(localNode().id()).toUpperCase();
+
+            IgniteProductLicense lic = ctx.license().license();
+
+            String body =
+                "GridGain node started with the following parameters:" + NL +
+                NL +
+                "----" + NL +
+                "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH + NL +
+                "Grid name: " + gridName + NL +
+                "Node ID: " + nid + NL +
+                "Node order: " + localNode().order() + NL +
+                "Node addresses: " + U.addressesAsString(localNode()) + NL +
+                "Local ports: " + sb + NL +
+                "OS name: " + U.osString() + NL +
+                "OS user: " + System.getProperty("user.name") + NL +
+                "CPU(s): " + localNode().metrics().getTotalCpus() + NL +
+                "Heap: " + U.heapSize(localNode(), 2) + "GB" + NL +
+                "JVM name: " + U.jvmName() + NL +
+                "JVM vendor: " + U.jvmVendor() + NL +
+                "JVM version: " + U.jvmVersion() + NL +
+                "VM name: " + rtBean.getName() + NL;
+
+            if (lic != null) {
+                body +=
+                    "License ID: " + lic.id().toString().toUpperCase() + NL +
+                    "Licensed to: " + lic.userOrganization() + NL;
+            }
+            else
+                assert !ENT;
+
+            body +=
+                "----" + NL +
+                NL +
+                "NOTE:" + NL +
+                "This message is sent automatically to all configured admin emails." + NL +
+                "To change this behavior use 'lifeCycleEmailNotify' grid configuration property." +
+                NL + NL +
+                "| " + SITE + NL +
+                "| support@gridgain.com" + NL;
+
+            sendAdminEmailAsync("GridGain node started: " + nid8, body, false);
+        }
+    }
+
+    /**
+     * Validates common configuration parameters.
+     *
+     * @param cfg Configuration.
+     */
+    private void validateCommon(IgniteConfiguration cfg) {
+        A.notNull(cfg.getNodeId(), "cfg.getNodeId()");
+
+        A.notNull(cfg.getMBeanServer(), "cfg.getMBeanServer()");
+        A.notNull(cfg.getGridLogger(), "cfg.getGridLogger()");
+        A.notNull(cfg.getMarshaller(), "cfg.getMarshaller()");
+        A.notNull(cfg.getExecutorService(), "cfg.getExecutorService()");
+        A.notNull(cfg.getUserAttributes(), "cfg.getUserAttributes()");
+
+        // All SPIs should be non-null.
+        A.notNull(cfg.getSwapSpaceSpi(), "cfg.getSwapSpaceSpi()");
+        A.notNull(cfg.getCheckpointSpi(), "cfg.getCheckpointSpi()");
+        A.notNull(cfg.getCommunicationSpi(), "cfg.getCommunicationSpi()");
+        A.notNull(cfg.getDeploymentSpi(), "cfg.getDeploymentSpi()");
+        A.notNull(cfg.getDiscoverySpi(), "cfg.getDiscoverySpi()");
+        A.notNull(cfg.getEventStorageSpi(), "cfg.getEventStorageSpi()");
+        A.notNull(cfg.getAuthenticationSpi(), "cfg.getAuthenticationSpi()");
+        A.notNull(cfg.getSecureSessionSpi(), "cfg.getSecureSessionSpi()");
+        A.notNull(cfg.getCollisionSpi(), "cfg.getCollisionSpi()");
+        A.notNull(cfg.getFailoverSpi(), "cfg.getFailoverSpi()");
+        A.notNull(cfg.getLoadBalancingSpi(), "cfg.getLoadBalancingSpi()");
+        A.notNull(cfg.getIndexingSpi(), "cfg.getIndexingSpi()");
+
+        A.ensure(cfg.getNetworkTimeout() > 0, "cfg.getNetworkTimeout() > 0");
+        A.ensure(cfg.getNetworkSendRetryDelay() > 0, "cfg.getNetworkSendRetryDelay() > 0");
+        A.ensure(cfg.getNetworkSendRetryCount() > 0, "cfg.getNetworkSendRetryCount() > 0");
+
+        if (!F.isEmpty(cfg.getPluginConfigurations())) {
+            for (PluginConfiguration pluginCfg : cfg.getPluginConfigurations())
+                A.notNull(pluginCfg.providerClass(), "PluginConfiguration.providerClass()");
+        }
+    }
+
+    /**
+     * Checks whether physical RAM is not exceeded.
+     */
+    @SuppressWarnings("ConstantConditions")
+    private void checkPhysicalRam() {
+        long ram = ctx.discovery().localNode().attribute(ATTR_PHY_RAM);
+
+        if (ram != -1) {
+            String macs = ctx.discovery().localNode().attribute(ATTR_MACS);
+
+            long totalHeap = 0;
+
+            for (ClusterNode node : ctx.discovery().allNodes()) {
+                if (macs.equals(node.attribute(ATTR_MACS))) {
+                    long heap = node.metrics().getHeapMemoryMaximum();
+
+                    if (heap != -1)
+                        totalHeap += heap;
+                }
+            }
+
+            if (totalHeap > ram) {
+                U.quietAndWarn(log, "Attempting to start more nodes than physical RAM " +
+                    "available on current host (this can cause significant slowdown)");
+            }
+        }
+    }
+
+    /**
+     * @param ctx Context.
+     * @param cfg Configuration to check for possible performance issues.
+     */
+    private void suggestOptimizations(GridKernalContext ctx, IgniteConfiguration cfg) {
+        GridPerformanceSuggestions perf = ctx.performance();
+
+        if (ctx.collision().enabled())
+            perf.add("Disable collision resolution (remove 'collisionSpi' from configuration)");
+
+        if (ctx.checkpoint().enabled())
+            perf.add("Disable checkpoints (remove 'checkpointSpi' from configuration)");
+
+        if (cfg.isPeerClassLoadingEnabled())
+            perf.add("Disable peer class loading (set 'peerClassLoadingEnabled' to false)");
+
+        if (cfg.isMarshalLocalJobs())
+            perf.add("Disable local jobs marshalling (set 'marshalLocalJobs' to false)");
+
+        if (cfg.getIncludeEventTypes() != null && cfg.getIncludeEventTypes().length != 0)
+            perf.add("Disable grid events (remove 'includeEventTypes' from configuration)");
+
+        if (IgniteOptimizedMarshaller.available() && !(cfg.getMarshaller() instanceof IgniteOptimizedMarshaller))
+            perf.add("Enable optimized marshaller (set 'marshaller' to " +
+                IgniteOptimizedMarshaller.class.getSimpleName() + ')');
+    }
+
+    /**
+     * Warns user about unsupported features which was configured in OS edition.
+     *
+     * @param cfg Grid configuration.
+     */
+    private void warnNotSupportedFeaturesForOs(IgniteConfiguration cfg) {
+        Collection<String> msgs = new ArrayList<>();
+
+        if (!F.isEmpty(cfg.getSegmentationResolvers()))
+            msgs.add("Network segmentation detection.");
+
+        if (cfg.getSecureSessionSpi() != null && !(cfg.getSecureSessionSpi() instanceof NoopSecureSessionSpi))
+            msgs.add("Secure session SPI.");
+
+        if (cfg.getAuthenticationSpi() != null && !(cfg.getAuthenticationSpi() instanceof NoopAuthenticationSpi))
+            msgs.add("Authentication SPI.");
+
+        if (!F.isEmpty(msgs)) {
+            U.quietAndInfo(log, "The following features are not supported in open source edition, " +
+                "related configuration settings will be ignored " +
+                "(consider downloading enterprise edition from http://www.gridgain.com):");
+
+            for (String s : msgs)
+                U.quietAndInfo(log, "  ^-- " + s);
+
+            U.quietAndInfo(log, "");
+        }
+    }
+
+    /**
+     * Creates attributes map and fills it in.
+     *
+     * @param cfg Grid configuration.
+     * @param build Build string.
+     * @return Map of all node attributes.
+     * @throws IgniteCheckedException thrown if was unable to set up attribute.
+     */
+    @SuppressWarnings({"SuspiciousMethodCalls", "unchecked", "TypeMayBeWeakened"})
+    private Map<String, Object> createNodeAttributes(IgniteConfiguration cfg, String build) throws IgniteCheckedException {
+        Map<String, Object> attrs = new HashMap<>();
+
+        final String[] incProps = cfg.getIncludeProperties();
+
+        try {
+            // Stick all environment settings into node attributes.
+            attrs.putAll(F.view(System.getenv(), new P1<String>() {
+                @Override public boolean apply(String name) {
+                    return incProps == null || U.containsStringArray(incProps, name, true) ||
+                        U.isVisorNodeStartProperty(name) || U.isVisorRequiredProperty(name);
+                }
+            }));
+
+            if (log.isDebugEnabled())
+                log.debug("Added environment properties to node attributes.");
+        }
+        catch (SecurityException e) {
+            throw new IgniteCheckedException("Failed to add environment properties to node attributes due to " +
+                "security violation: " + e.getMessage());
+        }
+
+        try {
+            // Stick all system properties into node's attributes overwriting any
+            // identical names from environment properties.
+            for (Map.Entry<Object, Object> e : F.view(System.getProperties(), new P1<Object>() {
+                @Override public boolean apply(Object o) {
+                    String name = (String)o;
+
+                    return incProps == null || U.containsStringArray(incProps, name, true) ||
+                        U.isVisorRequiredProperty(name);
+                }
+            }).entrySet()) {
+                Object val = attrs.get(e.getKey());
+
+                if (val != null && !val.equals(e.getValue()))
+                    U.warn(log, "System property will override environment variable with the same name: "
+                        + e.getKey());
+
+                attrs.put((String)e.getKey(), e.getValue());
+            }
+
+            if (log.isDebugEnabled())
+                log.debug("Added system properties to node attributes.");
+        }
+        catch (SecurityException e) {
+            throw new IgniteCheckedException("Failed to add system properties to node attributes due to security " +
+                "violation: " + e.getMessage());
+        }
+
+        // Add local network IPs and MACs.
+        String ips = F.concat(U.allLocalIps(), ", "); // Exclude loopbacks.
+        String macs = F.concat(U.allLocalMACs(), ", "); // Only enabled network interfaces.
+
+        // Ack network context.
+        if (log.isInfoEnabled()) {
+            log.info("Non-loopback local IPs: " + (F.isEmpty(ips) ? "N/A" : ips));
+            log.info("Enabled local MACs: " + (F.isEmpty(macs) ? "N/A" : macs));
+        }
+
+        // Warn about loopback.
+        if (ips.isEmpty() && macs.isEmpty())
+            U.warn(log, "GridGain is starting on loopback address... Only nodes on the same physical " +
+                "computer can participate in topology.",
+                "GridGain is starting on loopback address...");
+
+        // Stick in network context into attributes.
+        add(attrs, ATTR_IPS, (ips.isEmpty() ? "" : ips));
+        add(attrs, ATTR_MACS, (macs.isEmpty() ? "" : macs));
+
+        // Stick in some system level attributes
+        add(attrs, ATTR_JIT_NAME, U.getCompilerMx() == null ? "" : U.getCompilerMx().getName());
+        add(attrs, ATTR_BUILD_VER, COMPOUND_VER);
+        add(attrs, ATTR_BUILD_DATE, build);
+        add(attrs, ATTR_COMPATIBLE_VERS, (Serializable)compatibleVersions());
+        add(attrs, ATTR_MARSHALLER, cfg.getMarshaller().getClass().getName());
+        add(attrs, ATTR_USER_NAME, System.getProperty("user.name"));
+        add(attrs, ATTR_GRID_NAME, gridName);
+
+        add(attrs, ATTR_PEER_CLASSLOADING, cfg.isPeerClassLoadingEnabled());
+        add(attrs, ATTR_DEPLOYMENT_MODE, cfg.getDeploymentMode());
+        add(attrs, ATTR_LANG_RUNTIME, getLanguage());
+
+        add(attrs, ATTR_JVM_PID, U.jvmPid());
+
+        // Build a string from JVM arguments, because parameters with spaces are split.
+        SB jvmArgs = new SB(512);
+
+        for (String arg : U.jvmArgs()) {
+            if (arg.startsWith("-"))
+                jvmArgs.a("@@@");
+            else
+                jvmArgs.a(' ');
+
+            jvmArgs.a(arg);
+        }
+        // Add it to attributes.
+        add(attrs, ATTR_JVM_ARGS, jvmArgs.toString());
+
+        // Check daemon system property and override configuration if it's set.
+        if (isDaemon())
+            add(attrs, ATTR_DAEMON, "true");
+
+        // In case of the parsing error, JMX remote disabled or port not being set
+        // node attribute won't be set.
+        if (isJmxRemoteEnabled()) {
+            String portStr = System.getProperty("com.sun.management.jmxremote.port");
+
+            if (portStr != null)
+                try {
+                    add(attrs, ATTR_JMX_PORT, Integer.parseInt(portStr));
+                }
+                catch (NumberFormatException ignore) {
+                    // No-op.
+                }
+        }
+
+        // Whether restart is enabled and stick the attribute.
+        add(attrs, ATTR_RESTART_ENABLED, Boolean.toString(isRestartEnabled()));
+
+        // Save port range, port numbers will be stored by rest processor at runtime.
+        if (cfg.getClientConnectionConfiguration() != null)
+            add(attrs, ATTR_REST_PORT_RANGE, cfg.getClientConnectionConfiguration().getRestPortRange());
+
+        try {
+            AuthenticationSpi authSpi = cfg.getAuthenticationSpi();
+
+            boolean securityEnabled = authSpi != null && !U.hasAnnotation(authSpi.getClass(), IgniteSpiNoop.class);
+
+            GridSecurityCredentialsProvider provider = cfg.getSecurityCredentialsProvider();
+
+            if (provider != null) {
+                GridSecurityCredentials cred = provider.credentials();
+
+                if (cred != null)
+                    add(attrs, ATTR_SECURITY_CREDENTIALS, cred);
+                else if (securityEnabled)
+                    throw new IgniteCheckedException("Failed to start node (authentication SPI is configured, " +
+                        "by security credentials provider returned null).");
+            }
+            else if (securityEnabled)
+                throw new IgniteCheckedException("Failed to start node (authentication SPI is configured, " +
+                    "but security credentials provider is not set. Fix the configuration and restart the node).");
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteCheckedException("Failed to create node security credentials", e);
+        }
+
+        // Stick in SPI versions and classes attributes.
+        addAttributes(attrs, cfg.getCollisionSpi());
+        addAttributes(attrs, cfg.getSwapSpaceSpi());
+        addAttributes(attrs, cfg.getDiscoverySpi());
+        addAttributes(attrs, cfg.getFailoverSpi());
+        addAttributes(attrs, cfg.getCommunicationSpi());
+        addAttributes(attrs, cfg.getEventStorageSpi());
+        addAttributes(attrs, cfg.getCheckpointSpi());
+        addAttributes(attrs, cfg.getLoadBalancingSpi());
+        addAttributes(attrs, cfg.getAuthenticationSpi());
+        addAttributes(attrs, cfg.getSecureSessionSpi());
+        addAttributes(attrs, cfg.getDeploymentSpi());
+
+        // Set user attributes for this node.
+        if (cfg.getUserAttributes() != null) {
+            for (Map.Entry<String, ?> e : cfg.getUserAttributes().entrySet()) {
+                if (attrs.containsKey(e.getKey()))
+                    U.warn(log, "User or internal attribute has the same name as environment or system " +
+                        "property and will take precedence: " + e.getKey());
+
+                attrs.put(e.getKey(), e.getValue());
+            }
+        }
+
+        return attrs;
+    }
+
+    /**
+     * Add SPI version and class attributes into node attributes.
+     *
+     * @param attrs Node attributes map to add SPI attributes to.
+     * @param spiList Collection of SPIs to get attributes from.
+     * @throws IgniteCheckedException Thrown if was unable to set up attribute.
+     */
+    private void addAttributes(Map<String, Object> attrs, IgniteSpi... spiList) throws IgniteCheckedException {
+        for (IgniteSpi spi : spiList) {
+            Class<? extends IgniteSpi> spiCls = spi.getClass();
+
+            add(attrs, U.spiAttribute(spi, ATTR_SPI_CLASS), spiCls.getName());
+        }
+    }
+
+    /** @throws IgniteCheckedException If registration failed. */
+    private void registerKernalMBean() throws IgniteCheckedException {
+        try {
+            kernalMBean = U.registerMBean(
+                cfg.getMBeanServer(),
+                cfg.getGridName(),
+                "Kernal",
+                getClass().getSimpleName(),
+                this,
+                IgniteMXBean.class);
+
+            if (log.isDebugEnabled())
+                log.debug("Registered kernal MBean: " + kernalMBean);
+        }
+        catch (JMException e) {
+            kernalMBean = null;
+
+            throw new IgniteCheckedException("Failed to register kernal MBean.", e);
+        }
+    }
+
+    /** @throws IgniteCheckedException If registration failed. */
+    private void registerLocalNodeMBean() throws IgniteCheckedException {
+        ClusterLocalNodeMetricsMXBean mbean = new ClusterLocalNodeMetricsMXBeanImpl(ctx.discovery().localNode());
+
+        try {
+            locNodeMBean = U.registerMBean(
+                cfg.getMBeanServer(),
+                cfg.getGridName(),
+                "Kernal",
+                mbean.getClass().getSimpleName(),
+                mbean,
+                ClusterLocalNodeMetricsMXBean.class);
+
+            if (log.isDebugEnabled())
+                log.debug("Registered local node MBean: " + locNodeMBean);
+        }
+        catch (JMException e) {
+            locNodeMBean = null;
+
+            throw new IgniteCheckedException("Failed to register local node MBean.", e);
+        }
+    }
+
+    /** @throws IgniteCheckedException If registration failed. */
+    private void registerExecutorMBeans() throws IgniteCheckedException {
+        pubExecSvcMBean = registerExecutorMBean(cfg.getExecutorService(), "GridExecutionExecutor");
+        sysExecSvcMBean = registerExecutorMBean(cfg.getSystemExecutorService(), "GridSystemExecutor");
+        mgmtExecSvcMBean = registerExecutorMBean(cfg.getManagementExecutorService(), "GridManagementExecutor");
+        p2PExecSvcMBean = registerExecutorMBean(cfg.getPeerClassLoadingExecutorService(), "GridClassLoadingExecutor");
+
+        ClientConnectionConfiguration clientCfg = cfg.getClientConnectionConfiguration();
+
+        if (clientCfg != null) {
+            restExecSvcMBean = clientCfg.getRestExecutorService() != null ?
+                registerExecutorMBean(clientCfg.getRestExecutorService(), "GridRestExecutor") : null;
+        }
+    }
+
+    /**
+     * @param exec Executor service to register.
+     * @param name Property name for executor.
+     * @return Name for created MBean.
+     * @throws IgniteCheckedException If registration failed.
+     */
+    private ObjectName registerExecutorMBean(ExecutorService exec, String name) throws IgniteCheckedException {
+        assert exec != null;
+
+        try {
+            ObjectName res = U.registerMBean(
+                cfg.getMBeanServer(),
+                cfg.getGridName(),
+                "Thread Pools",
+                name,
+                new IgniteThreadPoolMXBeanAdapter(exec),
+                IgniteThreadPoolMXBean.class);
+
+            if (log.isDebugEnabled())
+                log.debug("Registered executor service MBean: " + res);
+
+            return res;
+        }
+        catch (JMException e) {
+            throw new IgniteCheckedException("Failed to register executor service MBean [name=" + name + ", exec=" + exec + ']',
+                e);
+        }
+    }
+
+    /**
+     * Unregisters given mbean.
+     *
+     * @param mbean MBean to unregister.
+     * @return {@code True} if successfully unregistered, {@code false} otherwise.
+     */
+    private boolean unregisterMBean(@Nullable ObjectName mbean) {
+        if (mbean != null)
+            try {
+                cfg.getMBeanServer().unregisterMBean(mbean);
+
+                if (log.isDebugEnabled())
+                    log.debug("Unregistered MBean: " + mbean);
+
+                return true;
+            }
+            catch (JMException e) {
+                U.error(log, "Failed to unregister MBean.", e);
+
+                return false;
+            }
+
+        return true;
+    }
+
+    /**
+     * @param ctx Kernal context.
+     * @param mgr Manager to start.
+     * @param attrs SPI attributes to set.
+     * @throws IgniteCheckedException Throw in case of any errors.
+     */
+    private void startManager(GridKernalContextImpl ctx, GridManager mgr, Map<String, Object> attrs)
+        throws IgniteCheckedException {
+        mgr.addSpiAttributes(attrs);
+
+        // Set all node attributes into discovery manager,
+        // so they can be distributed to all nodes.
+        if (mgr instanceof GridDiscoveryManager)
+            ((GridDiscoveryManager)mgr).setNodeAttributes(attrs, ctx.product().version());
+
+        // Add manager to registry before it starts to avoid
+        // cases when manager is started but registry does not
+        // have it yet.
+        ctx.add(mgr);
+
+        try {
+            mgr.start();
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteCheckedException("Failed to start manager: " + mgr, e);
+        }
+    }
+
+    /**
+     * @param ctx Kernal context.
+     * @param proc Processor to start.
+     * @param attrs Attributes.
+     * @throws IgniteCheckedException Thrown in case of any error.
+     */
+    private void startProcessor(GridKernalContextImpl ctx, GridProcessor proc, Map<String, Object> attrs)
+        throws IgniteCheckedException {
+        ctx.add(proc);
+
+        try {
+            proc.start();
+
+            proc.addAttributes(attrs);
+        }
+        catch (IgniteCheckedException e) {
+            throw new IgniteCheckedException("Failed to start processor: " + proc, e);
+        }
+    }
+
+    /**
+     * Add helper.
+     *
+     * @param ctx Context.
+     * @param helper Helper.
+     */
+    private void addHelper(GridKernalContextImpl ctx, Object helper) {
+        ctx.addHelper(helper);
+    }
+
+    /**
+     * Gets "on" or "off" string for given boolean value.
+     *
+     * @param b Boolean value to convert.
+     * @return Result string.
+     */
+    private String onOff(boolean b) {
+        return b ? "on" : "off";
+    }
+
+    /**
+     *
+     * @return Whether or not REST is enabled.
+     */
+    private boolean isRestEnabled() {
+        assert cfg != null;
+
+        return cfg.getClientConnectionConfiguration() != null;
+    }
+
+    /**
+     * Acks remote management.
+     */
+    private void ackRemoteManagement() {
+        assert log != null;
+
+        if (!log.isInfoEnabled())
+            return;
+
+        SB sb = new SB();
+
+        sb.a("Remote Management [");
+
+        boolean on = isJmxRemoteEnabled();
+
+        sb.a("restart: ").a(onOff(isRestartEnabled())).a(", ");
+        sb.a("REST: ").a(onOff(isRestEnabled())).a(", ");
+        sb.a("JMX (");
+        sb.a("remote: ").a(onOff(on));
+
+        if (on) {
+            sb.a(", ");
+
+            sb.a("port: ").a(System.getProperty("com.sun.management.jmxremote.port", "<n/a>")).a(", ");
+            sb.a("auth: ").a(onOff(Boolean.getBoolean("com.sun.management.jmxremote.authenticate"))).a(", ");
+
+            // By default SSL is enabled, that's why additional check for null is needed.
+            // See http://docs.oracle.com/javase/6/docs/technotes/guides/management/agent.html
+            sb.a("ssl: ").a(onOff(Boolean.getBoolean("com.sun.management.jmxremote.ssl") ||
+                System.getProperty("com.sun.management.jmxremote.ssl") == null));
+        }
+
+        sb.a(")");
+
+        sb.a(']');
+
+        log.info(sb.toString());
+    }
+
+    /**
+     * Acks configuration URL.
+     */
+    private void ackConfigUrl() {
+        assert log != null;
+
+        if (log.isInfoEnabled())
+            log.info("Config URL: " + System.getProperty(GG_CONFIG_URL, "n/a"));
+    }
+
+    /**
+     * Acks Visor instructions.
+     */
+    private void ackVisor() {
+        assert log != null;
+
+        if (isDaemon())
+            return;
+
+        if (ctx.isEnterprise())
+            U.quietAndInfo(log, "To start GUI Management & Monitoring run ggvisorui.{sh|bat}");
+        else
+            U.quietAndInfo(log, "To start Console Management & Monitoring run ggvisorcmd.{sh|bat}");
+    }
+
+    /**
+     * Acks benchmarking instructions.
+     */
+    private void ackBenchmarks() {
+        if (!isDaemon())
+            U.quietAndInfo(log, "If running benchmarks, see http://bit.ly/GridGain-Benchmarking");
+    }
+
+    /**
+     * Acks ASCII-logo. Thanks to http://patorjk.com/software/taag
+     */
+    private void ackAsciiLogo() {
+        assert log != null;
+
+        String fileName = log.fileName();
+
+        if (System.getProperty(GG_NO_ASCII) == null) {
+            String ver = "ver. " + ACK_VER;
+
+            // Big thanks to: http://patorjk.com/software/taag
+            // Font name "Small Slant"
+            if (log.isQuiet()) {
+                U.quiet(false,
+                    "   __________  ________________ ",
+                    "  /  _/ ___/ |/ /  _/_  __/ __/ ",
+                    " _/ // (_ /    // /  / / / _/   ",
+                    "/___/\\___/_/|_/___/ /_/ /___/  ",
+                    " ",
+                    ver,
+                    COPYRIGHT,
+                    "",
+                    "Quiet mode.");
+
+                if (fileName != null)
+                    U.quiet(false, "  ^-- Logging to file '" +  fileName + '\'');
+
+                U.quiet(false,
+                    "  ^-- To see **FULL** console log here add -DGRIDGAIN_QUIET=false or \"-v\" to ggstart.{sh|bat}",
+                    "");
+            }
+
+            if (log.isInfoEnabled()) {
+                log.info(NL + NL +
+                    ">>>    __________  ________________  " + NL +
+                    ">>>   /  _/ ___/ |/ /  _/_  __/ __/  " + NL +
+                    ">>>  _/ // (_ /    // /  / / / _/    " + NL +
+                    ">>> /___/\\___/_/|_/___/ /_/ /___/   " + NL +
+                    ">>> " + NL +
+                    ">>> " + ver + NL +
+                    ">>> " + COPYRIGHT + NL
+                );
+            }
+        }
+    }
+
+    /**
+     * Prints start info.
+     *
+     * @param rtBean Java runtime bean.
+     */
+    private void ackStart(RuntimeMXBean rtBean) {
+        if (log.isQuiet()) {
+            U.quiet(false, "");
+            U.quiet(false, "GridGain node started OK (id=" + U.id8(localNode().id()) +
+                (F.isEmpty(gridName) ? "" : ", grid=" + gridName) + ')');
+        }
+
+        if (log.isInfoEnabled()) {
+            log.info("");
+
+            String ack = "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH;
+
+            String dash = U.dash(ack.length());
+
+            SB sb = new SB();
+
+            for (GridPortRecord rec : ctx.ports().records())
+                sb.a(rec.protocol()).a(":").a(rec.port()).a(" ");
+
+            String str =
+                NL + NL +
+                    ">>> " + dash + NL +
+                    ">>> " + ack + NL +
+                    ">>> " + dash + NL +
+                    ">>> OS name: " + U.osString() + NL +
+                    ">>> CPU(s): " + localNode().metrics().getTotalCpus() + NL +
+                    ">>> Heap: " + U.heapSize(localNode(), 2) + "GB" + NL +
+                    ">>> VM name: " + rtBean.getName() + NL +
+                    ">>> Grid name: " + gridName + NL +
+                    ">>> Local node [" +
+                    "ID=" + localNode().id().toString().toUpperCase() +
+                    ", order=" + localNode().order() +
+                    "]" + NL +
+                    ">>> Local node addresses: " + U.addressesAsString(localNode()) + NL +
+                    ">>> Local ports: " + sb + NL;
+
+            str += ">>> GridGain documentation: http://" + SITE + "/documentation" + NL;
+
+            log.info(str);
+        }
+    }
+
+    /**
+     * Logs out OS information.
+     */
+    private void ackOsInfo() {
+        assert log != null;
+
+        if (log.isInfoEnabled()) {
+            log.info("OS: " + U.osString());
+            log.info("OS user: " + System.getProperty("user.name"));
+        }
+    }
+
+    /**
+     * Logs out language runtime.
+     */
+    private void ackLanguageRuntime() {
+        assert log != null;
+
+        if (log.isInfoEnabled()) {
+            log.info("Language runtime: " + getLanguage());
+            log.info("VM information: " + U.jdkString());
+            log.info("VM total memory: " + U.heapSize(2) + "GB");
+        }
+    }
+
+    /**
+     * @return Language runtime.
+     */
+    @SuppressWarnings("ThrowableInstanceNeverThrown")
+    private String getLanguage() {
+        boolean scala = false;
+        boolean groovy = false;
+        boolean clojure = false;
+
+        for (StackTraceElement elem : Thread.currentThread().getStackTrace()) {
+            String s = elem.getClassName().toLowerCase();
+
+            if (s.contains("scala")) {
+                scala = true;
+
+                break;
+            }
+            else if (s.contains("groovy")) {
+                groovy = true;
+
+                break;
+            }
+            else if (s.contains("clojure")) {
+                clojure = true;
+
+                break;
+            }
+        }
+
+        if (scala) {
+            try (InputStream in = getClass().getResourceAsStream("/library.properties")) {
+                Properties props = new Properties();
+
+                if (in != null)
+                    props.load(in);
+
+                return "Scala ver. " + props.getProperty("version.number", "<unknown>");
+            }
+            catch (Throwable ignore) {
+                return "Scala ver. <unknown>";
+            }
+        }
+
+        // How to get Groovy and Clojure version at runtime?!?
+        return groovy ? "Groovy" : clojure ? "Clojure" : U.jdkName() + " ver. " + U.jdkVersion();
+    }
+
+    /**
+     * Stops grid instance.
+     *
+     * @param cancel Whether or not to cancel running jobs.
+     */
+    public void stop(boolean cancel) {
+        // Make sure that thread stopping grid is not interrupted.
+        boolean interrupted = Thread.interrupted();
+
+        try {
+            stop0(cancel);
+        }
+        finally {
+            if (interrupted)
+                Thread.currentThread().interrupt();
+        }
+    }
+
+    /**
+     * @param cancel Whether or not to cancel running jobs.
+     */
+    private void stop0(boolean cancel) {
+        String nid = getLocalNodeId().toString().toUpperCase();
+        String nid8 = U.id8(getLocalNodeId()).toUpperCase();
+
+        gw.compareAndSet(null, new GridKernalGatewayImpl(gridName));
+
+        GridKernalGateway gw = this.gw.get();
+
+        if (stopGuard.compareAndSet(false, true)) {
+            // Only one thread is allowed to perform stop sequence.
+            boolean firstStop = false;
+
+            GridKernalState state = gw.getState();
+
+            if (state == STARTED)
+                firstStop = true;
+            else if (state == STARTING)
+                U.warn(log, "Attempt to stop starting grid. This operation " +
+                    "cannot be guaranteed to be successful.");
+
+            if (firstStop) {
+                // Notify lifecycle beans.
+                if (log.isDebugEnabled())
+                    log.debug("Notifying lifecycle beans.");
+
+                notifyLifecycleBeansEx(LifecycleEventType.BEFORE_GRID_STOP);
+            }
+
+            IgniteEmailProcessorAdapter email = ctx.email();
+
+            List<GridComponent> comps = ctx.components();
+
+            // Callback component in reverse order while kernal is still functional
+            // if called in the same thread, at least.
+            for (ListIterator<GridComponent> it = comps.listIterator(comps.size()); it.hasPrevious();) {
+                GridComponent comp = it.previous();
+
+                try {
+                    comp.onKernalStop(cancel);
+                }
+                catch (Throwable e) {
+                    errOnStop = true;
+
+                    U.error(log, "Failed to pre-stop processor: " + comp, e);
+                }
+            }
+
+            // Cancel update notification timer.
+            if (updateNtfTimer != null)
+                updateNtfTimer.cancel();
+
+            if (starveTimer != null)
+                starveTimer.cancel();
+
+            // Cancel license timer.
+            if (licTimer != null)
+                licTimer.cancel();
+
+            // Cancel metrics log timer.
+            if (metricsLogTimer != null)
+                metricsLogTimer.cancel();
+
+            gw.writeLock();
+
+            try {
+                assert gw.getState() == STARTED || gw.getState() == STARTING;
+
+                // No more kernal calls from this point on.
+                gw.setState(STOPPING);
+
+                // Clear node local store.
+                nodeLoc.clear();
+
+                if (log.isDebugEnabled())
+                    log.debug("Grid " + (gridName == null ? "" : '\'' + gridName + "' ") + "is stopping.");
+            }
+            finally {
+                gw.writeUnlock();
+            }
+
+            // Unregister MBeans.
+            if (!(
+                unregisterMBean(pubExecSvcMBean) &
+                    unregisterMBean(sysExecSvcMBean) &
+                    unregisterMBean(mgmtExecSvcMBean) &
+                    unregisterMBean(p2PExecSvcMBean) &
+                    unregisterMBean(kernalMBean) &
+                    unregisterMBean(locNodeMBean) &
+                    unregisterMBean(restExecSvcMBean)
+            ))
+                errOnStop = false;
+
+            // Stop components in reverse order.
+            for (ListIterator<GridComponent> it = comps.listIterator(comps.size()); it.hasPrevious();) {
+                GridComponent comp = it.previous();
+
+                try {
+                    comp.stop(cancel);
+
+                    if (log.isDebugEnabled())
+                        log.debug("Component stopped: " + comp);
+                }
+                catch (Throwable e) {
+                    errOnStop = true;
+
+                    U.error(log, "Failed to stop component (ignoring): " + comp, e);
+                }
+            }
+
+            // Stops lifecycle aware components.
+            U.stopLifecycleAware(log, lifecycleAwares(cfg));
+
+            // Lifecycle notification.
+            notifyLifecycleBeansEx(LifecycleEventType.AFTER_GRID_STOP);
+
+            // Clean internal class/classloader caches to avoid stopped contexts held in memory.
+            IgniteOptimizedMarshaller.clearCache();
+            IgniteMarshallerExclusions.clearCache();
+            GridEnumCache.clear();
+
+            gw.writeLock();
+
+            try {
+                gw.setState(STOPPED);
+            }
+            finally {
+                gw.writeUnlock();
+            }
+
+            // Ack stop.
+            if (log.isQuiet()) {
+                if (!errOnStop)
+                    U.quiet(false, "GridGain node stopped OK [uptime=" +
+                        X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + ']');
+                else
+                    U.quiet(true, "GridGain node stopped wih ERRORS [uptime=" +
+                        X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + ']');
+            }
+
+            if (log.isInfoEnabled())
+                if (!errOnStop) {
+                    String ack = "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH +
+                        " stopped OK";
+
+                    String dash = U.dash(ack.length());
+
+                    log.info(NL + NL +
+                        ">>> " + dash + NL +
+                        ">>> " + ack + NL +
+                        ">>> " + dash + NL +
+                        ">>> Grid name: " + gridName + NL +
+                        ">>> Grid uptime: " + X.timeSpan2HMSM(U.currentTimeMillis() - startTime) +
+                        NL +
+                        NL);
+                }
+                else {
+                    String ack = "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH +
+                        " stopped with ERRORS";
+
+                    String dash = U.dash(ack.length());
+
+                    log.info(NL + NL +
+                        ">>> " + ack + NL +
+                        ">>> " + dash + NL +
+                        ">>> Grid name: " + gridName + NL +
+                        ">>> Grid uptime: " + X.timeSpan2HMSM(U.currentTimeMillis() - startTime) +
+                        NL +
+                        ">>> See log above for detailed error message." + NL +
+                        ">>> Note that some errors during stop can prevent grid from" + NL +
+                        ">>> maintaining correct topology since this node may have" + NL +
+                        ">>> not exited grid properly." + NL +
+                        NL);
+                }
+
+            // Send node start email notification, if enabled.
+            if (isSmtpEnabled() && isAdminEmailsSet() && cfg.isLifeCycleEmailNotification()) {
+                String errOk = errOnStop ? "with ERRORS" : "OK";
+
+                String headline = "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR +
+                    " stopped " + errOk + ":";
+                String subj = "GridGain node stopped " + errOk + ": " + nid8;
+
+                IgniteProductLicense lic = ctx.license() != null ? ctx.license().license() : null;
+
+                String body =
+                    headline + NL + NL +
+                    "----" + NL +
+                    "GridGain ver. " + COMPOUND_VER + '#' + BUILD_TSTAMP_STR + "-sha1:" + REV_HASH + NL +
+                    "Grid name: " + gridName + NL +
+                    "Node ID: " + nid + NL +
+                    "Node uptime: " + X.timeSpan2HMSM(U.currentTimeMillis() - startTime) + NL;
+
+                if (lic != null) {
+                    body +=
+                        "License ID: " + lic.id().toString().toUpperCase() + NL +
+                        "Licensed to: " + lic.userOrganization() + NL;
+                }
+                else
+                    assert !ENT;
+
+                body +=
+                    "----" + NL +
+                    NL +
+                    "NOTE:" + NL +
+                    "This message is sent automatically to all configured admin emails." + NL +
+                    "To change this behavior use 'lifeCycleEmailNotify' grid configuration property.";
+
+                if (errOnStop)
+                    body +=
+                        NL + NL +
+                            "NOTE:" + NL +
+                            "See node's log for detailed error message." + NL +
+                            "Some errors during stop can prevent grid from" + NL +
+                            "maintaining correct topology since this node may " + NL +
+                            "have not exited grid properly.";
+
+                body +=
+                    NL + NL +
+                        "| " + SITE + NL +
+                        "| support@gridgain.com" + NL;
+
+                if (email != null) {
+                    try {
+                        email.sendNow(subj,
+                            body,
+                            false,
+                            Arrays.asList(cfg.getAdminEmails()));
+                    }
+                    catch (IgniteCheckedException e) {
+                        U.error(log, "Failed to send lifecycle email notification.", e);
+                    }
+                }
+            }
+
+            U.onGridStop();
+        }
+        else {
+            // Proper notification.
+            if (log.isDebugEnabled()) {
+                if (gw.getState() == STOPPED)
+                    log.debug("Grid is already stopped. Nothing to do.");
+                else
+                    log.debug("Grid is being stopped by another thread. Aborting this stop sequence " +
+                        "allowing other thread to finish.");
+            }
+        }
+    }
+
+    /**
+     * USED ONLY FOR TESTING.
+     *
+     * @param <K> Key type.
+     * @param <V> Value type.
+     * @return Internal cache instance.
+     */
+    /*@java.test.only*/
+    public <K, V> GridCacheAdapter<K, V> internalCache() {
+        return internalCache(null);
+    }
+
+    /**
+     * USED ONLY FOR TESTING.
+     *
+     * @param name Cache name.
+     * @param <K>  Key type.
+     * @param <V>  Value type.
+     * @return Internal cache instance.
+     */
+    /*@java.test.only*/
+    public <K, V> GridCacheAdapter<K, V> internalCache(@Nullable String name) {
+        return ctx.cache().internalCache(name);
+    }
+
+    /**
+     * It's intended for use by internal marshalling implementation only.
+     *
+     * @return Kernal context.
+     */
+    public GridKernalContext context() {
+        return ctx;
+    }
+
+    /**
+     * Prints all system properties in debug mode.
+     */
+    private void ackSystemProperties() {
+        assert log != null;
+
+        if (log.isDebugEnabled())
+            for (Object key : U.asIterable(System.getProperties().keys()))
+                log.debug("System property [" + key + '=' + System.getProperty((String) key) + ']');
+    }
+
+    /**
+     * Prints all user attributes in info mode.
+     */
+    private void logNodeUserAttributes() {
+        assert log != null;
+
+        if (log.isInfoEnabled())
+            for (Map.Entry<?, ?> attr : cfg.getUserAttributes().entrySet())
+                log.info("Local node user attribute [" + attr.getKey() + '=' + attr.getValue() + ']');
+    }
+
+    /**
+     * Prints all environment variables in debug mode.
+     */
+    private void ackEnvironmentVariables() {
+        assert log != null;
+
+        if (log.isDebugEnabled())
+            for (Map.Entry<?, ?> envVar : System.getenv().entrySet())
+                log.debug("Environment variable [" + envVar.getKey() + '=' + envVar.getValue() + ']');
+    }
+
+    /**
+     * Acks daemon mode status.
+     */
+    private void ackDaemon() {
+        assert log != null;
+
+        if (log.isInfoEnabled())
+            log.info("Daemon mode: " + (isDaemon() ? "on" : "off"));
+    }
+
+    /**
+     *
+     * @return {@code True} is this node is daemon.
+     */
+    private boolean isDaemon() {
+        assert cfg != null;
+
+        return cfg.isDaemon() || "true".equalsIgnoreCase(System.getProperty(GG_DAEMON));
+    }
+
+    /**
+     * Whether or not remote JMX management is enabled for this node. Remote JMX management is
+     * enabled when the following system property is set:
+     * <ul>
+     *     <li>{@code com.sun.management.jmxremote}</li>
+     * </ul>
+     *
+     * @return {@code True} if remote JMX management is enabled - {@code false} otherwise.
+     */
+    @Override public boolean isJmxRemoteEnabled() {
+        return System.getProperty("com.sun.management.jmxremote") != null;
+    }
+
+    /**
+     * Whether or not node restart is enabled. Node restart us supported when this node was started
+     * with {@code bin/ggstart.{sh|bat}} script using {@code -r} argument. Node can be
+     * programmatically restarted using {@link org.apache.ignite.Ignition#restart(boolean)}} method.
+     *
+     * @return {@code True} if restart mode is enabled, {@code false} otherwise.
+     * @see org.apache.ignite.Ignition#restart(boolean)
+     */
+    @Override public boolean isRestartEnabled() {
+        return System.getProperty(GG_SUCCESS_FILE) != null;
+    }
+
+    /**
+     * Whether or not SMTP is configured. Note that SMTP is considered configured if
+     * SMTP host is provided in configuration (see {@link org.apache.ignite.configuration.IgniteConfiguration#getSmtpHost()}.
+     * <p>
+     * If SMTP is not configured all emails notifications will be disabled.
+     *
+     * @return {@code True} if SMTP is configured - {@code false} otherwise.
+     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpFromEmail()
+     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpHost()
+     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpPassword()
+     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpPort()
+     * @see org.apache.ignite.configuration.IgniteConfiguration#getSmtpUsername()
+     * @see org.apache.ignite.configuration.IgniteConfiguration#isSmtpSsl()
+     * @see org.apache.ignite.configuration.IgniteConfiguration#isSmtpStartTls()
+     * @see #sendAdminEmailAsync(String, String, boolean)
+     */
+    @Override public boolean isSmtpEnabled() {
+        assert cfg != null;
+
+        return cfg.getSmtpHost() != null;
+    }
+
+    /**
+     * Prints all configuration properties in info mode and SPIs in debug mode.
+     */
+    private void ackSpis() {
+        assert log != null;
+
+        if (log.isDebugEnabled()) {
+            log.debug("+-------------+");
+            log.debug("START SPI LIST:");
+            log.debug("+-------------+");
+            log.debug("Grid checkpoint SPI     : " + Arrays.toString(cfg.getCheckpointSpi()));
+            log.debug("Grid collision SPI      : " + cfg.getCollisionSpi());
+            log.debug("Grid communication SPI  : " + cfg.getCommunicationSpi());
+            log.debug("Grid deployment SPI     : " + cfg.getDeploymentSpi());
+            log.debug("Grid discovery SPI      : " + cfg.getDiscoverySpi());
+            log.debug("Grid event storage SPI  : " + cfg.getEventStorageSpi());
+            log.debug("Grid failover SPI       : " + Arrays.toString(cfg.getFailoverSpi()));
+            log.debug("Grid load balancing SPI : " + Arrays.toString(cfg.getLoadBalancingSpi()));
+            log.debug("Grid authentication SPI : " + cfg.getAuthenticationSpi());
+            log.debug("Grid secure session SPI : " + cfg.getSecureSessionSpi());
+            log.debug("Grid swap space SPI     : " + cfg.getSwapSpaceSpi());
+        }
+    }
+
+    /**
+     *
+     */
+    private void ackCacheConfiguration() {
+        CacheConfiguration[] cacheCfgs = cfg.getCacheConfiguration();
+
+        if (cacheCfgs == null || cacheCfgs.length == 0)
+            U.warn(log, "Cache is not configured - in-memory data grid is off.");
+        else {
+            SB sb = new SB();
+
+            for (CacheConfiguration c : cacheCfgs) {
+                String name = c.getName();
+
+                if (name == null)
+                    name = "<default>";
+
+                sb.a("'").a(name).a("', ");
+            }
+
+            String names = sb.toString();
+
+            U.log(log, "Configured caches [" + names.substring(0, names.length() - 2) + ']');
+        }
+    }
+
+    /**
+     *
+     */
+    private void ackP2pConfiguration() {
+        assert cfg != null;
+
+        if (cfg.isPeerClassLoadingEnabled())
+            U.warn(
+                log,
+                "Peer class loading is enabled (disable it in production for performance and " +
+                    "deployment consistency reasons)",
+                "Peer class loading is enabled (disable it for better performance)"
+            );
+    }
+
+    /**
+     * Prints security status.
+     *
+     * @param ctx Kernal context.
+     */
+    private void ackSecurity(GridKernalContext ctx) {
+        assert log != null;
+
+        if (log.isInfoEnabled())
+            log.info("Security status [authentication=" + onOff(ctx.security().enabled()) + ", " +
+                "secure-session=" + onOff(ctx.secureSession().enabled()) + ']');
+    }
+
+    /**
+     * Prints out SMTP configuration.
+     */
+    private void ackSmtpConfiguration() {
+        assert log != null;
+
+        String host = cfg.getSmtpHost();
+
+        boolean ssl = cfg.isSmtpSsl();
+        int port = cfg.getSmtpPort();
+
+        if (host == null) {
+            U.warn(log, "SMTP is not configured - email notifications are off.");
+
+            return;
+        }
+
+        String from = cfg.getSmtpFromEmail();
+
+        if (log.isQuiet())
+            U.quiet(false, "SMTP enabled [host=" + host + ":" + port + ", ssl=" + (ssl ? "on" : "off") + ", from=" +
+                from + ']');
+
+        if (log.isInfoEnabled()) {
+            String[] adminEmails = cfg.getAdminEmails();
+
+            log.info("SMTP enabled [host=" + host + ", port=" + port + ", ssl=" + ssl + ", from=" + from + ']');
+            log.info("Admin emails: " + (!isAdminEmailsSet() ? "N/A" : Arrays.toString(adminEmails)));
+        }
+
+        if (!isAd

<TRUNCATED>

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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
index 4d2ece7..8db40b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProcessor.java
@@ -147,13 +147,13 @@ public class GridServiceProcessor extends GridProcessorAdapter {
         ManagedServiceConfiguration[] cfgs = ctx.config().getServiceConfiguration();
 
         if (cfgs != null) {
-            Collection<IgniteFuture<?>> futs = new ArrayList<>();
+            Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
             for (ManagedServiceConfiguration c : ctx.config().getServiceConfiguration())
                 futs.add(deploy(c));
 
             // Await for services to deploy.
-            for (IgniteFuture<?> f : futs)
+            for (IgniteInternalFuture<?> f : futs)
                 f.get();
         }
 
@@ -262,7 +262,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @param svc Service.
      * @return Future.
      */
-    public IgniteFuture<?> deployNodeSingleton(ClusterGroup prj, String name, ManagedService svc) {
+    public IgniteInternalFuture<?> deployNodeSingleton(ClusterGroup prj, String name, ManagedService svc) {
         return deployMultiple(prj, name, svc, 0, 1);
     }
 
@@ -271,7 +271,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @param svc Service.
      * @return Future.
      */
-    public IgniteFuture<?> deployClusterSingleton(ClusterGroup prj, String name, ManagedService svc) {
+    public IgniteInternalFuture<?> deployClusterSingleton(ClusterGroup prj, String name, ManagedService svc) {
         return deployMultiple(prj, name, svc, 1, 1);
     }
 
@@ -282,7 +282,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @param maxPerNodeCnt Max per-node count.
      * @return Future.
      */
-    public IgniteFuture<?> deployMultiple(ClusterGroup prj, String name, ManagedService svc, int totalCnt,
+    public IgniteInternalFuture<?> deployMultiple(ClusterGroup prj, String name, ManagedService svc, int totalCnt,
         int maxPerNodeCnt) {
         ManagedServiceConfiguration cfg = new ManagedServiceConfiguration();
 
@@ -302,7 +302,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @param  affKey Affinity key.
      * @return Future.
      */
-    public IgniteFuture<?> deployKeyAffinitySingleton(String name, ManagedService svc, String cacheName, Object affKey) {
+    public IgniteInternalFuture<?> deployKeyAffinitySingleton(String name, ManagedService svc, String cacheName, Object affKey) {
         A.notNull(affKey, "affKey");
 
         ManagedServiceConfiguration cfg = new ManagedServiceConfiguration();
@@ -321,7 +321,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @param cfg Service configuration.
      * @return Future for deployment.
      */
-    public IgniteFuture<?> deploy(ManagedServiceConfiguration cfg) {
+    public IgniteInternalFuture<?> deploy(ManagedServiceConfiguration cfg) {
         A.notNull(cfg, "cfg");
 
         validate(cfg);
@@ -412,7 +412,7 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @param name Service name.
      * @return Future.
      */
-    public IgniteFuture<?> cancel(String name) {
+    public IgniteInternalFuture<?> cancel(String name) {
         while (true) {
             try {
                 GridFutureAdapter<?> fut = new GridFutureAdapter<>(ctx);
@@ -450,8 +450,8 @@ public class GridServiceProcessor extends GridProcessorAdapter {
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    public IgniteFuture<?> cancelAll() {
-        Collection<IgniteFuture<?>> futs = new ArrayList<>();
+    public IgniteInternalFuture<?> cancelAll() {
+        Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
         for (CacheEntry<Object, Object> e : cache.entrySetx()) {
             if (!(e.getKey() instanceof GridServiceDeploymentKey))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/IgniteStreamerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/IgniteStreamerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/IgniteStreamerImpl.java
index 3542615..760cf12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/IgniteStreamerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/streamer/IgniteStreamerImpl.java
@@ -751,8 +751,8 @@ public class IgniteStreamerImpl implements IgniteStreamerEx, Externalizable {
 
             execSvc.submit(worker);
 
-            batchFut.listenAsync(new CI1<IgniteFuture<Object>>() {
-                @Override public void apply(IgniteFuture<Object> t) {
+            batchFut.listenAsync(new CI1<IgniteInternalFuture<Object>>() {
+                @Override public void apply(IgniteInternalFuture<Object> t) {
                     BatchExecutionFuture fut = (BatchExecutionFuture)t;
 
                     if (log.isDebugEnabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java
index 82ae077..f384437 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/GridUtils.java
@@ -7289,10 +7289,10 @@ public abstract class GridUtils {
      * @param f Future to listen to.
      * @param log Logger.
      */
-    public static void asyncLogError(IgniteFuture<?> f, final IgniteLogger log) {
+    public static void asyncLogError(IgniteInternalFuture<?> f, final IgniteLogger log) {
         if (f != null)
-            f.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> f) {
+            f.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> f) {
                     try {
                         f.get();
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
index 47bc380..db16ae8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundFuture.java
@@ -40,10 +40,10 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
     private static final long serialVersionUID = 0L;
 
     /** Futures. */
-    private final ConcurrentLinkedDeque8<IgniteFuture<T>> futs = new ConcurrentLinkedDeque8<>();
+    private final ConcurrentLinkedDeque8<IgniteInternalFuture<T>> futs = new ConcurrentLinkedDeque8<>();
 
     /** Pending futures. */
-    private final Collection<IgniteFuture<T>> pending = new ConcurrentLinkedDeque8<>();
+    private final Collection<IgniteInternalFuture<T>> pending = new ConcurrentLinkedDeque8<>();
 
     /** Listener call count. */
     private final AtomicInteger lsnrCalls = new AtomicInteger();
@@ -97,7 +97,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      * @param futs Futures to add.
      */
     public GridCompoundFuture(GridKernalContext ctx, @Nullable IgniteReducer<T, R> rdc,
-        @Nullable Iterable<IgniteFuture<T>> futs) {
+        @Nullable Iterable<IgniteInternalFuture<T>> futs) {
         super(ctx);
 
         this.rdc = rdc;
@@ -108,7 +108,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
     /** {@inheritDoc} */
     @Override public boolean cancel() throws IgniteCheckedException {
         if (onCancelled()) {
-            for (IgniteFuture<T> fut : futs)
+            for (IgniteInternalFuture<T> fut : futs)
                 fut.cancel();
 
             return true;
@@ -122,7 +122,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      *
      * @return Collection of futures.
      */
-    public Collection<IgniteFuture<T>> futures() {
+    public Collection<IgniteInternalFuture<T>> futures() {
         return futs;
     }
 
@@ -131,7 +131,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      *
      * @return Pending futures.
      */
-    public Collection<IgniteFuture<T>> pending() {
+    public Collection<IgniteInternalFuture<T>> pending() {
         return pending;
     }
 
@@ -166,7 +166,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      *
      * @param fut Future to add.
      */
-    public void add(IgniteFuture<T> fut) {
+    public void add(IgniteInternalFuture<T> fut) {
         assert fut != null;
 
         pending.add(fut);
@@ -188,7 +188,7 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      *
      * @param futs Futures to add.
      */
-    public void addAll(@Nullable IgniteFuture<T>... futs) {
+    public void addAll(@Nullable IgniteInternalFuture<T>... futs) {
         addAll(F.asList(futs));
     }
 
@@ -197,9 +197,9 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
      *
      * @param futs Futures to add.
      */
-    public void addAll(@Nullable Iterable<IgniteFuture<T>> futs) {
+    public void addAll(@Nullable Iterable<IgniteInternalFuture<T>> futs) {
         if (futs != null)
-            for (IgniteFuture<T> fut : futs)
+            for (IgniteInternalFuture<T> fut : futs)
                 add(fut);
     }
 
@@ -291,8 +291,8 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
             "cancelled", isCancelled(),
             "err", error(),
             "futs",
-                F.viewReadOnly(futs, new C1<IgniteFuture<T>, String>() {
-                    @Override public String apply(IgniteFuture<T> f) {
+                F.viewReadOnly(futs, new C1<IgniteInternalFuture<T>, String>() {
+                    @Override public String apply(IgniteInternalFuture<T> f) {
                         return Boolean.toString(f.isDone());
                     }
                 })
@@ -302,12 +302,12 @@ public class GridCompoundFuture<T, R> extends GridFutureAdapter<R> {
     /**
      * Listener for futures.
      */
-    private class Listener implements IgniteInClosure<IgniteFuture<T>> {
+    private class Listener implements IgniteInClosure<IgniteInternalFuture<T>> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
-        @Override public void apply(IgniteFuture<T> fut) {
+        @Override public void apply(IgniteInternalFuture<T> fut) {
             pending.remove(fut);
 
             try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundIdentityFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundIdentityFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundIdentityFuture.java
index cf5323c..85b898c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundIdentityFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridCompoundIdentityFuture.java
@@ -57,7 +57,7 @@ public class GridCompoundIdentityFuture<T> extends GridCompoundFuture<T, T> {
      * @param futs Futures to add.
      */
     public GridCompoundIdentityFuture(GridKernalContext ctx, @Nullable IgniteReducer<T, T> rdc,
-        @Nullable Iterable<IgniteFuture<T>> futs) {
+        @Nullable Iterable<IgniteInternalFuture<T>> futs) {
         super(ctx, rdc, futs);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java
index 516b0fc..e3df9f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridEmbeddedFuture.java
@@ -35,7 +35,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
     private static final long serialVersionUID = 0L;
 
     /** Embedded future to wait for. */
-    private IgniteFuture<B> embedded;
+    private IgniteInternalFuture<B> embedded;
 
     /**
      * Empty constructor required by {@link Externalizable}.
@@ -49,7 +49,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
      * @param embedded Embedded future.
      * @param c Closure to execute upon completion of embedded future.
      */
-    public GridEmbeddedFuture(GridKernalContext ctx, IgniteFuture<B> embedded, final IgniteBiClosure<B, Exception, A> c) {
+    public GridEmbeddedFuture(GridKernalContext ctx, IgniteInternalFuture<B> embedded, final IgniteBiClosure<B, Exception, A> c) {
         super(ctx);
 
         assert embedded != null;
@@ -59,7 +59,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
 
         embedded.listenAsync(new AL1() {
             @SuppressWarnings({"ErrorNotRethrown", "CatchGenericClass"})
-            @Override public void applyx(IgniteFuture<B> embedded) {
+            @Override public void applyx(IgniteInternalFuture<B> embedded) {
                 try {
                     onDone(c.apply(embedded.get(), null));
                 }
@@ -83,7 +83,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
      * @param c Closure which runs upon completion of embedded closure and which returns another future.
      * @param ctx Context.
      */
-    public GridEmbeddedFuture(boolean syncNotify, IgniteFuture<B> embedded, IgniteBiClosure<B, Exception, IgniteFuture<A>> c,
+    public GridEmbeddedFuture(boolean syncNotify, IgniteInternalFuture<B> embedded, IgniteBiClosure<B, Exception, IgniteInternalFuture<A>> c,
         GridKernalContext ctx) {
         this(embedded, c, ctx);
 
@@ -97,7 +97,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
      * @param embedded Closure.
      * @param c Closure which runs upon completion of embedded closure and which returns another future.
      */
-    public GridEmbeddedFuture(IgniteFuture<B> embedded, final IgniteBiClosure<B, Exception, IgniteFuture<A>> c,
+    public GridEmbeddedFuture(IgniteInternalFuture<B> embedded, final IgniteBiClosure<B, Exception, IgniteInternalFuture<A>> c,
         GridKernalContext ctx) {
         super(ctx);
 
@@ -107,9 +107,9 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
         this.embedded = embedded;
 
         embedded.listenAsync(new AL1() {
-            @Override public void applyx(IgniteFuture<B> embedded) {
+            @Override public void applyx(IgniteInternalFuture<B> embedded) {
                 try {
-                    IgniteFuture<A> next = c.apply(embedded.get(), null);
+                    IgniteInternalFuture<A> next = c.apply(embedded.get(), null);
 
                     if (next == null) {
                         onDone();
@@ -118,7 +118,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
                     }
 
                     next.listenAsync(new AL2() {
-                        @Override public void applyx(IgniteFuture<A> next) {
+                        @Override public void applyx(IgniteInternalFuture<A> next) {
                             try {
                                 onDone(next.get());
                             }
@@ -163,8 +163,8 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
      * @param c1 Closure which runs upon completion of embedded future and which returns another future.
      * @param c2 Closure will runs upon completion of future returned by {@code c1} closure.
      */
-    public GridEmbeddedFuture(GridKernalContext ctx, IgniteFuture<B> embedded, final IgniteBiClosure<B, Exception,
-        IgniteFuture<A>> c1, final IgniteBiClosure<A, Exception, A> c2) {
+    public GridEmbeddedFuture(GridKernalContext ctx, IgniteInternalFuture<B> embedded, final IgniteBiClosure<B, Exception,
+        IgniteInternalFuture<A>> c1, final IgniteBiClosure<A, Exception, A> c2) {
         super(ctx);
 
         assert embedded != null;
@@ -174,9 +174,9 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
         this.embedded = embedded;
 
         embedded.listenAsync(new AL1() {
-            @Override public void applyx(IgniteFuture<B> embedded) {
+            @Override public void applyx(IgniteInternalFuture<B> embedded) {
                 try {
-                    IgniteFuture<A> next = c1.apply(embedded.get(), null);
+                    IgniteInternalFuture<A> next = c1.apply(embedded.get(), null);
 
                     if (next == null) {
                         onDone();
@@ -185,7 +185,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
                     }
 
                     next.listenAsync(new AL2() {
-                        @Override public void applyx(IgniteFuture<A> next) {
+                        @Override public void applyx(IgniteInternalFuture<A> next) {
                             try {
                                 onDone(c2.apply(next.get(), null));
                             }
@@ -258,12 +258,12 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
     /**
      * Make sure that listener does not throw exceptions.
      */
-    private abstract class AsyncListener1 implements IgniteInClosure<IgniteFuture<B>> {
+    private abstract class AsyncListener1 implements IgniteInClosure<IgniteInternalFuture<B>> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
-        @Override public final void apply(IgniteFuture<B> f) {
+        @Override public final void apply(IgniteInternalFuture<B> f) {
             try {
                 applyx(f);
             }
@@ -284,18 +284,18 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
          * @param f Future.
          * @throws Exception In case of error.
          */
-        protected abstract void applyx(IgniteFuture<B> f) throws Exception;
+        protected abstract void applyx(IgniteInternalFuture<B> f) throws Exception;
     }
 
     /**
      * Make sure that listener does not throw exceptions.
      */
-    private abstract class AsyncListener2 implements IgniteInClosure<IgniteFuture<A>> {
+    private abstract class AsyncListener2 implements IgniteInClosure<IgniteInternalFuture<A>> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
-        @Override public final void apply(IgniteFuture<A> f) {
+        @Override public final void apply(IgniteInternalFuture<A> f) {
             try {
                 applyx(f);
             }
@@ -316,6 +316,6 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
          * @param f Future.
          * @throws Exception In case of error.
          */
-        protected abstract void applyx(IgniteFuture<A> f) throws Exception;
+        protected abstract void applyx(IgniteInternalFuture<A> f) throws Exception;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
index 8ff11af..d4039c0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFuture.java
@@ -31,7 +31,7 @@ import static org.apache.ignite.IgniteSystemProperties.*;
 /**
  * Future that is completed at creation time.
  */
-public class GridFinishedFuture<T> implements IgniteFuture<T>, Externalizable {
+public class GridFinishedFuture<T> implements IgniteInternalFuture<T>, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -166,7 +166,7 @@ public class GridFinishedFuture<T> implements IgniteFuture<T>, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public void listenAsync(final IgniteInClosure<? super IgniteFuture<T>> lsnr) {
+    @Override public void listenAsync(final IgniteInClosure<? super IgniteInternalFuture<T>> lsnr) {
         if (ctx == null)
             throw new IllegalStateException("Cannot attach listener to deserialized future (context is null): " + this);
 
@@ -183,12 +183,12 @@ public class GridFinishedFuture<T> implements IgniteFuture<T>, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteFuture<T>>... lsnr) {
+    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<T>>... lsnr) {
         // No-op.
     }
 
     /** {@inheritDoc} */
-    @Override public <R> IgniteFuture<R> chain(final IgniteClosure<? super IgniteFuture<T>, R> doneCb) {
+    @Override public <R> IgniteInternalFuture<R> chain(final IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb) {
         GridFutureAdapter<R> fut = new GridFutureAdapter<R>(ctx, syncNotify) {
             @Override public String toString() {
                 return "ChainFuture[orig=" + GridFinishedFuture.this + ", doneCb=" + doneCb + ']';

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFutureEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFutureEx.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFutureEx.java
index 0db9f20..6daf3b2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFutureEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFinishedFutureEx.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.future;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -31,7 +32,7 @@ import java.util.concurrent.*;
  * {@link GridFinishedFuture} as it does not take context as a parameter and
  * performs notifications in the same thread.
  */
-public class GridFinishedFutureEx<T> implements IgniteFuture<T>, Externalizable {
+public class GridFinishedFutureEx<T> implements IgniteInternalFuture<T>, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -143,7 +144,7 @@ public class GridFinishedFutureEx<T> implements IgniteFuture<T>, Externalizable
     }
 
     /** {@inheritDoc} */
-    @Override public <R> IgniteFuture<R> chain(IgniteClosure<? super IgniteFuture<T>, R> doneCb) {
+    @Override public <R> IgniteInternalFuture<R> chain(IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb) {
         try {
             return new GridFinishedFutureEx<>(doneCb.apply(this));
         }
@@ -159,13 +160,13 @@ public class GridFinishedFutureEx<T> implements IgniteFuture<T>, Externalizable
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked"})
-    @Override public void listenAsync(IgniteInClosure<? super IgniteFuture<T>> lsnr) {
+    @Override public void listenAsync(IgniteInClosure<? super IgniteInternalFuture<T>> lsnr) {
         if (lsnr != null)
             lsnr.apply(this);
     }
 
     /** {@inheritDoc} */
-    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteFuture<T>>... lsnr) {
+    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<T>>... lsnr) {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
index eb7a0bf..9a8a439 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapter.java
@@ -35,7 +35,7 @@ import static org.apache.ignite.IgniteSystemProperties.*;
 /**
  * Future adapter.
  */
-public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements IgniteFuture<R>, Externalizable {
+public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements IgniteInternalFuture<R>, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -77,7 +77,7 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
     private boolean valid = true;
 
     /** Asynchronous listeners. */
-    private Collection<IgniteInClosure<? super IgniteFuture<R>>> lsnrs;
+    private Collection<IgniteInClosure<? super IgniteInternalFuture<R>>> lsnrs;
 
     /** Context. */
     protected GridKernalContext ctx;
@@ -259,7 +259,7 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
     }
 
     /** {@inheritDoc} */
-    @Override public void listenAsync(@Nullable final IgniteInClosure<? super IgniteFuture<R>> lsnr) {
+    @Override public void listenAsync(@Nullable final IgniteInClosure<? super IgniteInternalFuture<R>> lsnr) {
         if (lsnr != null) {
             checkValid();
 
@@ -298,7 +298,7 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
     }
 
     /** {@inheritDoc} */
-    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteFuture<R>>... lsnr) {
+    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<R>>... lsnr) {
         synchronized (mux) {
             if (lsnrs == null)
                 return;
@@ -307,10 +307,10 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
                 lsnrs.clear();
             else {
                 // Iterate through the whole list, removing all occurrences, if any.
-                for (Iterator<IgniteInClosure<? super IgniteFuture<R>>> it = lsnrs.iterator(); it.hasNext();) {
-                    IgniteInClosure<? super IgniteFuture<R>> l1 = it.next();
+                for (Iterator<IgniteInClosure<? super IgniteInternalFuture<R>>> it = lsnrs.iterator(); it.hasNext();) {
+                    IgniteInClosure<? super IgniteInternalFuture<R>> l1 = it.next();
 
-                    for (IgniteInClosure<? super IgniteFuture<R>> l2 : lsnr)
+                    for (IgniteInClosure<? super IgniteInternalFuture<R>> l2 : lsnr)
                         // Must be l1.equals(l2), not l2.equals(l1), because of the way listeners are added.
                         if (l1.equals(l2))
                             it.remove();
@@ -320,7 +320,7 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<T> chain(final IgniteClosure<? super IgniteFuture<R>, T> doneCb) {
+    @Override public <T> IgniteInternalFuture<T> chain(final IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb) {
         return new ChainFuture<>(ctx, syncNotify, this, doneCb);
     }
 
@@ -328,7 +328,7 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
      * Notifies all registered listeners.
      */
     private void notifyListeners() {
-        final Collection<IgniteInClosure<? super IgniteFuture<R>>> lsnrs0;
+        final Collection<IgniteInClosure<? super IgniteInternalFuture<R>>> lsnrs0;
 
         synchronized (mux) {
             lsnrs0 = lsnrs;
@@ -342,7 +342,7 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
         assert !lsnrs0.isEmpty();
 
         if (concurNotify) {
-            for (final IgniteInClosure<? super IgniteFuture<R>> lsnr : lsnrs0)
+            for (final IgniteInClosure<? super IgniteInternalFuture<R>> lsnr : lsnrs0)
                 ctx.closure().runLocalSafe(new GPR() {
                     @Override public void run() {
                         notifyListener(lsnr);
@@ -356,13 +356,13 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
                     @Override public void run() {
                         // Since concurrent notifications are off, we notify
                         // all listeners in one thread.
-                        for (IgniteInClosure<? super IgniteFuture<R>> lsnr : lsnrs0)
+                        for (IgniteInClosure<? super IgniteInternalFuture<R>> lsnr : lsnrs0)
                             notifyListener(lsnr);
                     }
                 }, true);
             }
             else
-                for (IgniteInClosure<? super IgniteFuture<R>> lsnr : lsnrs0)
+                for (IgniteInClosure<? super IgniteInternalFuture<R>> lsnr : lsnrs0)
                     notifyListener(lsnr);
         }
     }
@@ -372,7 +372,7 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
      *
      * @param lsnr Listener.
      */
-    private void notifyListener(IgniteInClosure<? super IgniteFuture<R>> lsnr) {
+    private void notifyListener(IgniteInClosure<? super IgniteInternalFuture<R>> lsnr) {
         assert lsnr != null;
 
         try {
@@ -593,7 +593,7 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
         private GridFutureAdapter<R> fut;
 
         /** */
-        private IgniteClosure<? super IgniteFuture<R>, T> doneCb;
+        private IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb;
 
         /**
          *
@@ -609,7 +609,7 @@ public class GridFutureAdapter<R> extends AbstractQueuedSynchronizer implements
          * @param doneCb Closure.
          */
         ChainFuture(GridKernalContext ctx, boolean syncNotify,
-            GridFutureAdapter<R> fut, IgniteClosure<? super IgniteFuture<R>, T> doneCb) {
+            GridFutureAdapter<R> fut, IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb) {
             super(ctx, syncNotify);
 
             this.fut = fut;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapterEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapterEx.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapterEx.java
index 177561a..6d60b92 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapterEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureAdapterEx.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.util.future;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.tostring.*;
@@ -34,7 +35,7 @@ import java.util.concurrent.locks.*;
 /**
  * Future adapter without kernal context.
  */
-public class GridFutureAdapterEx<R> extends AbstractQueuedSynchronizer implements IgniteFuture<R>, Externalizable {
+public class GridFutureAdapterEx<R> extends AbstractQueuedSynchronizer implements IgniteInternalFuture<R>, Externalizable {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -64,7 +65,7 @@ public class GridFutureAdapterEx<R> extends AbstractQueuedSynchronizer implement
     private boolean valid = true;
 
     /** Asynchronous listener. */
-    private final ConcurrentLinkedDeque8<IgniteInClosure<? super IgniteFuture<R>>>
+    private final ConcurrentLinkedDeque8<IgniteInClosure<? super IgniteInternalFuture<R>>>
         lsnrs = new ConcurrentLinkedDeque8<>();
 
     /**
@@ -209,21 +210,21 @@ public class GridFutureAdapterEx<R> extends AbstractQueuedSynchronizer implement
 
     /** {@inheritDoc} */
     @SuppressWarnings({"unchecked", "TooBroadScope"})
-    @Override public void listenAsync(@Nullable final IgniteInClosure<? super IgniteFuture<R>> lsnr) {
+    @Override public void listenAsync(@Nullable final IgniteInClosure<? super IgniteInternalFuture<R>> lsnr) {
         if (lsnr != null) {
             checkValid();
 
             boolean done;
 
-            IgniteInClosure<? super IgniteFuture<R>> lsnr0 = lsnr;
+            IgniteInClosure<? super IgniteInternalFuture<R>> lsnr0 = lsnr;
 
             done = isDone();
 
             if (!done) {
-                lsnr0 = new IgniteInClosure<IgniteFuture<R>>() {
+                lsnr0 = new IgniteInClosure<IgniteInternalFuture<R>>() {
                     private final AtomicBoolean called = new AtomicBoolean();
 
-                    @Override public void apply(IgniteFuture<R> t) {
+                    @Override public void apply(IgniteInternalFuture<R> t) {
                         if (called.compareAndSet(false, true))
                             lsnr.apply(t);
                     }
@@ -248,15 +249,15 @@ public class GridFutureAdapterEx<R> extends AbstractQueuedSynchronizer implement
     }
 
     /** {@inheritDoc} */
-    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteFuture<R>>... lsnr) {
+    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<R>>... lsnr) {
         if (lsnr == null || lsnr.length == 0)
             lsnrs.clear();
         else {
             // Iterate through the whole list, removing all occurrences, if any.
-            for (Iterator<IgniteInClosure<? super IgniteFuture<R>>> it = lsnrs.iterator(); it.hasNext(); ) {
-                IgniteInClosure<? super IgniteFuture<R>> l1 = it.next();
+            for (Iterator<IgniteInClosure<? super IgniteInternalFuture<R>>> it = lsnrs.iterator(); it.hasNext(); ) {
+                IgniteInClosure<? super IgniteInternalFuture<R>> l1 = it.next();
 
-                for (IgniteInClosure<? super IgniteFuture<R>> l2 : lsnr)
+                for (IgniteInClosure<? super IgniteInternalFuture<R>> l2 : lsnr)
                     // Must be l1.equals(l2), not l2.equals(l1), because of the way listeners are added.
                     if (l1.equals(l2))
                         it.remove();
@@ -265,15 +266,15 @@ public class GridFutureAdapterEx<R> extends AbstractQueuedSynchronizer implement
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<T> chain(final IgniteClosure<? super IgniteFuture<R>, T> doneCb) {
+    @Override public <T> IgniteInternalFuture<T> chain(final IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb) {
         final GridFutureAdapterEx<T> fut = new GridFutureAdapterEx<T>() {
             @Override public String toString() {
                 return "ChainFuture[orig=" + GridFutureAdapterEx.this + ", doneCb=" + doneCb + ']';
             }
         };
 
-        listenAsync(new IgniteInClosure<IgniteFuture<R>>() {
-            @Override public void apply(IgniteFuture<R> t) {
+        listenAsync(new IgniteInClosure<IgniteInternalFuture<R>>() {
+            @Override public void apply(IgniteInternalFuture<R> t) {
                 try {
                     fut.onDone(doneCb.apply(t));
                 }
@@ -309,7 +310,7 @@ public class GridFutureAdapterEx<R> extends AbstractQueuedSynchronizer implement
         if (lsnrs.isEmptyx())
             return;
 
-        for (IgniteInClosure<? super IgniteFuture<R>> lsnr : lsnrs)
+        for (IgniteInClosure<? super IgniteInternalFuture<R>> lsnr : lsnrs)
             notifyListener(lsnr);
     }
 
@@ -318,7 +319,7 @@ public class GridFutureAdapterEx<R> extends AbstractQueuedSynchronizer implement
      *
      * @param lsnr Listener.
      */
-    private void notifyListener(IgniteInClosure<? super IgniteFuture<R>> lsnr) {
+    private void notifyListener(IgniteInClosure<? super IgniteInternalFuture<R>> lsnr) {
         assert lsnr != null;
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
index 542f4a8..7caba83 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/future/GridFutureChainListener.java
@@ -25,7 +25,7 @@ import org.apache.ignite.internal.util.lang.*;
 /**
  * Future listener to fill chained future with converted result of the source future.
  */
-public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteFuture<T>> {
+public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteInternalFuture<T>> {
     /** */
     private static final long serialVersionUID = 0L;
 
@@ -36,7 +36,7 @@ public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteFutu
     private final GridFutureAdapter<R> fut;
 
     /** Done callback. */
-    private final IgniteClosure<? super IgniteFuture<T>, R> doneCb;
+    private final IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb;
 
     /**
      * Constructs chain listener.
@@ -46,14 +46,14 @@ public class GridFutureChainListener<T, R> implements IgniteInClosure<IgniteFutu
      * @param doneCb Done callback.
      */
     public GridFutureChainListener(GridKernalContext ctx, GridFutureAdapter<R> fut,
-        IgniteClosure<? super IgniteFuture<T>, R> doneCb) {
+        IgniteClosure<? super IgniteInternalFuture<T>, R> doneCb) {
         this.ctx = ctx;
         this.fut = fut;
         this.doneCb = doneCb;
     }
 
     /** {@inheritDoc} */
-    @Override public void apply(IgniteFuture<T> t) {
+    @Override public void apply(IgniteInternalFuture<T> t) {
         try {
             fut.onDone(doneCb.apply(t));
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
index ced12e0..c21bf5f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/lang/GridFunc.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.future.*;
@@ -499,15 +500,15 @@ public class GridFunc {
     };
 
     /** */
-    private static final IgnitePredicate<IgniteFuture<?>> FINISHED_FUTURE = new IgnitePredicate<IgniteFuture<?>>() {
-        @Override public boolean apply(IgniteFuture<?> f) {
+    private static final IgnitePredicate<IgniteInternalFuture<?>> FINISHED_FUTURE = new IgnitePredicate<IgniteInternalFuture<?>>() {
+        @Override public boolean apply(IgniteInternalFuture<?> f) {
             return f.isDone();
         }
     };
 
     /** */
-    private static final IgnitePredicate<IgniteFuture<?>> UNFINISHED_FUTURE = new IgnitePredicate<IgniteFuture<?>>() {
-        @Override public boolean apply(IgniteFuture<?> f) {
+    private static final IgnitePredicate<IgniteInternalFuture<?>> UNFINISHED_FUTURE = new IgnitePredicate<IgniteInternalFuture<?>>() {
+        @Override public boolean apply(IgniteInternalFuture<?> f) {
             return !f.isDone();
         }
     };
@@ -2741,13 +2742,13 @@ public class GridFunc {
     }
 
     /**
-     * Converts given object with interface {@link org.apache.ignite.lang.IgniteFuture} into an object implementing {@link Future}.
+     * Converts given object with interface {@link org.apache.ignite.internal.IgniteInternalFuture} into an object implementing {@link Future}.
      *
      * @param fut Future to convert.
      * @param <T> Type of computation result.
      * @return Instance implementing {@link Future}.
      */
-    public static <T> Future<T> as(final IgniteFuture<T> fut) {
+    public static <T> Future<T> as(final IgniteInternalFuture<T> fut) {
         A.notNull(fut, "fut");
 
         return new GridSerializableFuture<T>() {
@@ -2809,14 +2810,14 @@ public class GridFunc {
     }
 
     /**
-     * Gets closure that converts {@link org.apache.ignite.lang.IgniteFuture} to {@link Future}.
+     * Gets closure that converts {@link org.apache.ignite.internal.IgniteInternalFuture} to {@link Future}.
      *
      * @param <T> Type of future.
-     * @return Closure that converts {@link org.apache.ignite.lang.IgniteFuture} to {@link Future}.
+     * @return Closure that converts {@link org.apache.ignite.internal.IgniteInternalFuture} to {@link Future}.
      */
-    public static <T> IgniteClosure<IgniteFuture<T>, Future<T>> future() {
-        return new C1<IgniteFuture<T>, Future<T>>() {
-            @Override public Future<T> apply(IgniteFuture<T> fut) {
+    public static <T> IgniteClosure<IgniteInternalFuture<T>, Future<T>> future() {
+        return new C1<IgniteInternalFuture<T>, Future<T>>() {
+            @Override public Future<T> apply(IgniteInternalFuture<T> fut) {
                 return as(fut);
             }
         };
@@ -8440,7 +8441,7 @@ public class GridFunc {
      * @param futs Futures. If none provided - this method is no-op.
      * @throws IgniteCheckedException If any of the futures failed.
      */
-    public static <T> void awaitAll(@Nullable IgniteFuture<T>... futs) throws IgniteCheckedException {
+    public static <T> void awaitAll(@Nullable IgniteInternalFuture<T>... futs) throws IgniteCheckedException {
         if (!isEmpty(futs))
             awaitAll(asList(futs));
     }
@@ -8451,7 +8452,7 @@ public class GridFunc {
      * @param futs Futures. If none provided - this method is no-op.
      * @throws IgniteCheckedException If any of the futures failed.
      */
-    public static <T> void awaitAll(@Nullable Collection<IgniteFuture<T>> futs) throws IgniteCheckedException {
+    public static <T> void awaitAll(@Nullable Collection<IgniteInternalFuture<T>> futs) throws IgniteCheckedException {
         awaitAll(0, null, futs);
     }
 
@@ -8462,7 +8463,7 @@ public class GridFunc {
      * @param futs Futures. If none provided - this method is no-op.
      * @throws IgniteCheckedException If any of the futures failed.
      */
-    public static <T> void awaitAll(long timeout, @Nullable Collection<IgniteFuture<T>> futs) throws IgniteCheckedException {
+    public static <T> void awaitAll(long timeout, @Nullable Collection<IgniteInternalFuture<T>> futs) throws IgniteCheckedException {
         awaitAll(timeout, null, futs);
     }
 
@@ -8478,7 +8479,7 @@ public class GridFunc {
      * @throws IgniteCheckedException If any of the futures failed.
      */
     @Nullable public static <T, R> R awaitAll(long timeout, @Nullable IgniteReducer<T, R> rdc,
-        @Nullable Collection<IgniteFuture<T>> futs) throws IgniteCheckedException {
+        @Nullable Collection<IgniteInternalFuture<T>> futs) throws IgniteCheckedException {
         if (futs == null || futs.isEmpty())
             return null;
 
@@ -8491,7 +8492,7 @@ public class GridFunc {
         // Note that it is important to wait in the natural order of collection and
         // not via listen method, because caller may actually add to this collection
         // concurrently while this method is in progress.
-        for (IgniteFuture<T> fut : futs) {
+        for (IgniteInternalFuture<T> fut : futs) {
             T t;
 
             if (timeout > 0) {
@@ -8523,7 +8524,7 @@ public class GridFunc {
      * @param <T> Type of computation result.
      * @return Completed future.
      */
-    public static <T> IgniteFuture<T> awaitOne(IgniteFuture<T>... futs) {
+    public static <T> IgniteInternalFuture<T> awaitOne(IgniteInternalFuture<T>... futs) {
         return isEmpty(futs) ? new GridFinishedFutureEx<T>() : awaitOne(asList(futs));
     }
 
@@ -8534,22 +8535,22 @@ public class GridFunc {
      * @param <T> Type of computation result.
      * @return Completed future.
      */
-    public static <T> IgniteFuture<T> awaitOne(Iterable<IgniteFuture<T>> futs) {
+    public static <T> IgniteInternalFuture<T> awaitOne(Iterable<IgniteInternalFuture<T>> futs) {
         if (F.isEmpty(futs))
             return new GridFinishedFutureEx<>();
 
         final CountDownLatch latch = new CountDownLatch(1);
 
-        final AtomicReference<IgniteFuture<T>> t = new AtomicReference<>();
+        final AtomicReference<IgniteInternalFuture<T>> t = new AtomicReference<>();
 
-        IgniteInClosure<IgniteFuture<T>> c = null;
+        IgniteInClosure<IgniteInternalFuture<T>> c = null;
 
-        for (IgniteFuture<T> fut : futs) {
+        for (IgniteInternalFuture<T> fut : futs) {
             if (fut != null) {
                 if (!fut.isDone()) {
                     if (c == null) {
-                        c = new CI1<IgniteFuture<T>>() {
-                            @Override public void apply(IgniteFuture<T> fut) {
+                        c = new CI1<IgniteInternalFuture<T>>() {
+                            @Override public void apply(IgniteInternalFuture<T> fut) {
                                 if (t.compareAndSet(null, fut))
                                     latch.countDown();
                             }
@@ -8581,7 +8582,7 @@ public class GridFunc {
         if (interrupted)
             Thread.currentThread().interrupt();
 
-        IgniteFuture<T> f = t.get();
+        IgniteInternalFuture<T> f = t.get();
 
         assert f != null;
 
@@ -8593,7 +8594,7 @@ public class GridFunc {
      *
      * @return Predicate for filtering finished futures.
      */
-    public static IgnitePredicate<IgniteFuture<?>> finishedFutures() {
+    public static IgnitePredicate<IgniteInternalFuture<?>> finishedFutures() {
         return FINISHED_FUTURE;
     }
 
@@ -8602,7 +8603,7 @@ public class GridFunc {
      *
      * @return Predicate for filtering unfinished futures.
      */
-    public static IgnitePredicate<IgniteFuture<?>> unfinishedFutures() {
+    public static IgnitePredicate<IgniteInternalFuture<?>> unfinishedFutures() {
         return UNFINISHED_FUTURE;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
index 9ba7a9b..bdd4383 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/nio/GridNioFuture.java
@@ -25,7 +25,7 @@ import java.io.*;
 import java.util.concurrent.*;
 
 /**
- * Reduced variant of {@link org.apache.ignite.lang.IgniteFuture} interface. Removed asynchronous
+ * Reduced variant of {@link org.apache.ignite.internal.IgniteInternalFuture} interface. Removed asynchronous
  * listen methods which require a valid grid kernal context.
  * @param <R> Type of the result for the future.
  */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
index 6cf45e9..9695249 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/util/typedef/X.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.util.typedef;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
@@ -821,11 +821,11 @@ public final class X {
      * @param futs Futures to wait for.
      * @throws IgniteCheckedException If any of the futures threw exception.
      */
-    public static void waitAll(@Nullable Iterable<IgniteFuture<?>> futs) throws IgniteCheckedException {
+    public static void waitAll(@Nullable Iterable<IgniteInternalFuture<?>> futs) throws IgniteCheckedException {
         if (F.isEmpty(futs))
             return;
 
-        for (IgniteFuture fut : futs)
+        for (IgniteInternalFuture fut : futs)
             fut.get();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadTask.java
index 4d4b8ce..8b7c446 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCachePreloadTask.java
@@ -19,7 +19,7 @@ package org.apache.ignite.internal.visor.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.processors.task.*;
 import org.apache.ignite.internal.visor.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -56,14 +56,14 @@ public class VisorCachePreloadTask extends VisorOneNodeTask<Set<String>, Void> {
 
         /** {@inheritDoc} */
         @Override protected Void run(Set<String> cacheNames) throws IgniteCheckedException {
-            Collection<IgniteFuture<?>> futs = new ArrayList<>();
+            Collection<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
             for(GridCache c : g.cachesx()) {
                 if (cacheNames.contains(c.name()))
                     futs.add(c.forceRepartition());
             }
 
-            for (IgniteFuture f: futs)
+            for (IgniteInternalFuture f: futs)
                 f.get();
 
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
index 4540e8c..23226a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupport.java
@@ -17,6 +17,8 @@
 
 package org.apache.ignite.lang;
 
+import org.apache.ignite.internal.*;
+
 /**
  * TODO: Add interface description.
  */
@@ -38,5 +40,5 @@ public interface IgniteAsyncSupport {
      *
      * @return Future for previous asynchronous operation.
      */
-    public <R> IgniteFuture<R> future();
+    public <R> IgniteInternalFuture<R> future();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupportAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupportAdapter.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupportAdapter.java
index 63cf9f5..e9ab2a1 100644
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupportAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/lang/IgniteAsyncSupportAdapter.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.lang;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 
 /**
  * Adapter for {@link IgniteAsyncSupport}.
@@ -27,7 +28,7 @@ public class IgniteAsyncSupportAdapter<T extends IgniteAsyncSupport> implements
     private static final Object mux = new Object();
 
     /** Future for previous asynchronous operation. */
-    protected ThreadLocal<IgniteFuture<?>> curFut;
+    protected ThreadLocal<IgniteInternalFuture<?>> curFut;
 
     /** */
     private volatile T asyncInstance;
@@ -81,18 +82,18 @@ public class IgniteAsyncSupportAdapter<T extends IgniteAsyncSupport> implements
     }
 
     /** {@inheritDoc} */
-    @Override public <R> IgniteFuture<R> future() {
+    @Override public <R> IgniteInternalFuture<R> future() {
         if (curFut == null)
             throw new IllegalStateException("Asynchronous mode is disabled.");
 
-        IgniteFuture<?> fut = curFut.get();
+        IgniteInternalFuture<?> fut = curFut.get();
 
         if (fut == null)
             throw new IllegalStateException("Asynchronous operation not started.");
 
         curFut.set(null);
 
-        return (IgniteFuture<R>)fut;
+        return (IgniteInternalFuture<R>)fut;
     }
 
     /**
@@ -101,7 +102,7 @@ public class IgniteAsyncSupportAdapter<T extends IgniteAsyncSupport> implements
      *         otherwise waits for future and returns result.
      * @throws IgniteCheckedException If asynchronous mode is disabled and future failed.
      */
-    public <R> R saveOrGet(IgniteFuture<R> fut) throws IgniteCheckedException {
+    public <R> R saveOrGet(IgniteInternalFuture<R> fut) throws IgniteCheckedException {
         if (curFut != null) {
             curFut.set(fut);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/lang/IgniteFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/lang/IgniteFuture.java b/modules/core/src/main/java/org/apache/ignite/lang/IgniteFuture.java
deleted file mode 100644
index 5c67f55..0000000
--- a/modules/core/src/main/java/org/apache/ignite/lang/IgniteFuture.java
+++ /dev/null
@@ -1,189 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.lang;
-
-import org.apache.ignite.*;
-import org.jetbrains.annotations.*;
-
-import java.util.concurrent.*;
-
-/**
- * Extension for standard {@link Future} interface. It adds simplified exception handling,
- * functional programming support and ability to listen for future completion via functional
- * callback.
- * @param <R> Type of the result for the future.
- */
-public interface IgniteFuture<R> {
-    /**
-     * Synchronously waits for completion of the computation and
-     * returns computation result.
-     *
-     * @return Computation result.
-     * @throws org.apache.ignite.IgniteInterruptedException Subclass of {@link IgniteCheckedException} thrown if the wait was interrupted.
-     * @throws IgniteFutureCancelledException Subclass of {@link IgniteCheckedException} throws if computation was cancelled.
-     * @throws IgniteCheckedException If computation failed.
-     */
-    public R get() throws IgniteCheckedException;
-
-    /**
-     * Synchronously waits for completion of the computation for
-     * up to the timeout specified and returns computation result.
-     * This method is equivalent to calling {@link #get(long, TimeUnit) get(long, TimeUnit.MILLISECONDS)}.
-     *
-     * @param timeout The maximum time to wait in milliseconds.
-     * @return Computation result.
-     * @throws org.apache.ignite.IgniteInterruptedException Subclass of {@link IgniteCheckedException} thrown if the wait was interrupted.
-     * @throws IgniteFutureTimeoutException Subclass of {@link IgniteCheckedException} thrown if the wait was timed out.
-     * @throws IgniteFutureCancelledException Subclass of {@link IgniteCheckedException} throws if computation was cancelled.
-     * @throws IgniteCheckedException If computation failed.
-     */
-    public R get(long timeout) throws IgniteCheckedException;
-
-    /**
-     * Synchronously waits for completion of the computation for
-     * up to the timeout specified and returns computation result.
-     *
-     * @param timeout The maximum time to wait.
-     * @param unit The time unit of the {@code timeout} argument.
-     * @return Computation result.
-     * @throws org.apache.ignite.IgniteInterruptedException Subclass of {@link IgniteCheckedException} thrown if the wait was interrupted.
-     * @throws IgniteFutureTimeoutException Subclass of {@link IgniteCheckedException} thrown if the wait was timed out.
-     * @throws IgniteFutureCancelledException Subclass of {@link IgniteCheckedException} throws if computation was cancelled.
-     * @throws IgniteCheckedException If computation failed.
-     */
-    public R get(long timeout, TimeUnit unit) throws IgniteCheckedException;
-
-    /**
-     * Cancels this future.
-     *
-     * @return {@code True} if future was canceled (i.e. was not finished prior to this call).
-     * @throws IgniteCheckedException If cancellation failed.
-     */
-    public boolean cancel() throws IgniteCheckedException;
-
-    /**
-     * Checks if computation is done.
-     *
-     * @return {@code True} if computation is done, {@code false} otherwise.
-     */
-    public boolean isDone();
-
-    /**
-     * Returns {@code true} if this computation was cancelled before it completed normally.
-     *
-     * @return {@code True} if this computation was cancelled before it completed normally.
-     */
-    public boolean isCancelled();
-
-    /**
-     * Gets start time for this future.
-     *
-     * @return Start time for this future.
-     */
-    public long startTime();
-
-    /**
-     * Gets duration in milliseconds between start of the future and current time if future
-     * is not finished, or between start and finish of this future.
-     *
-     * @return Time in milliseconds this future has taken to execute.
-     */
-    public long duration();
-
-    /**
-     * Flag to turn on or off synchronous listener notification. If this flag is {@code true}, then
-     * upon future completion the notification may happen in the same thread that created
-     * the future. This becomes especially important when adding listener to a future that
-     * is already {@code done} - if this flag is {@code true}, then listener will be
-     * immediately notified within the same thread.
-     * <p>
-     * Default value is {@code false}. To change the default, set
-     * {@link IgniteSystemProperties#GG_FUT_SYNC_NOTIFICATION} system property to {@code true}.
-     *
-     * @param syncNotify Flag to turn on or off synchronous listener notification.
-     */
-    public void syncNotify(boolean syncNotify);
-
-    /**
-     * Gets value of synchronous listener notification flag. If this flag is {@code true}, then
-     * upon future completion the notification may happen in the same thread that created
-     * the future. This becomes especially important when adding listener to a future that
-     * is already {@code done} - if this flag is {@code true}, then listener will be
-     * immediately notified within the same thread.
-     * <p>
-     * Default value is {@code false}. To change the default, set
-     * {@link IgniteSystemProperties#GG_FUT_SYNC_NOTIFICATION} system property to {@code true}.
-     *
-     * @return Synchronous listener notification flag.
-     */
-    public boolean syncNotify();
-
-    /**
-     * Flag to turn on or off concurrent listener notification. This flag comes into play only
-     * when a future has more than one listener subscribed to it. If this flag is {@code true},
-     * then all listeners will be notified concurrently by different threads; otherwise,
-     * listeners will be notified one after another within one thread (depending on
-     * {@link #syncNotify()} flag, these notifications may happen either in the same thread which
-     * started the future, or in a different thread).
-     * <p>
-     * Default value is {@code false}. To change the default, set
-     * {@link IgniteSystemProperties#GG_FUT_CONCURRENT_NOTIFICATION} system property to {@code true}.
-     *
-     * @param concurNotify Flag to turn on or off concurrent listener notification.
-     */
-    public void concurrentNotify(boolean concurNotify);
-
-    /**
-     * Gets value concurrent listener notification flag. This flag comes into play only
-     * when a future has more than one listener subscribed to it. If this flag is {@code true},
-     * then all listeners will be notified concurrently by different threads; otherwise,
-     * listeners will be notified one after another within one thread (depending on
-     * {@link #syncNotify()} flag, these notifications may happen either in the same thread which
-     * started the future, or in a different thread).
-     * <p>
-     * Default value is {@code false}. To change the default, set
-     * {@link IgniteSystemProperties#GG_FUT_CONCURRENT_NOTIFICATION} system property to {@code true}.
-     *
-     * @return Concurrent listener notification flag
-     */
-    public boolean concurrentNotify();
-
-    /**
-     * Registers listener closure to be asynchronously notified whenever future completes.
-     *
-     * @param lsnr Listener closure to register. If not provided - this method is no-op.
-     */
-    public void listenAsync(@Nullable IgniteInClosure<? super IgniteFuture<R>> lsnr);
-
-    /**
-     * Removes given listeners from the future. If no listener is passed in, then all listeners
-     * will be removed.
-     *
-     * @param lsnr Listeners to remove.
-     */
-    public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteFuture<R>>... lsnr);
-
-    /**
-     * Make a chained future to convert result of this future (when complete) into a new format.
-     * It is guaranteed that done callback will be called only ONCE.
-     *
-     * @param doneCb Done callback that is applied to this future when it finishes to produce chained future result.
-     * @return Chained future that finishes after this future completes and done callback is called.
-     */
-    public <T> IgniteFuture<T> chain(IgniteClosure<? super IgniteFuture<R>, T> doneCb);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/scheduler/SchedulerFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/scheduler/SchedulerFuture.java b/modules/core/src/main/java/org/apache/ignite/scheduler/SchedulerFuture.java
index 006bec2..c064197 100644
--- a/modules/core/src/main/java/org/apache/ignite/scheduler/SchedulerFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/scheduler/SchedulerFuture.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.scheduler;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 
 import java.util.concurrent.*;
 
@@ -27,7 +27,7 @@ import java.util.concurrent.*;
  * when calling {@link org.apache.ignite.IgniteScheduler#scheduleLocal(Callable, String)} or
  * {@link org.apache.ignite.IgniteScheduler#scheduleLocal(Runnable, String)} methods.
  */
-public interface SchedulerFuture<R> extends IgniteFuture<R> {
+public interface SchedulerFuture<R> extends IgniteInternalFuture<R> {
     /**
      * Gets scheduled task ID.
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
index 9568472..547a332 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpDiscoverySpi.java
@@ -268,7 +268,7 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
     private final Object mux = new Object();
 
     /** Map with proceeding ping requests. */
-    private final ConcurrentMap<InetSocketAddress, IgniteFuture<IgniteBiTuple<UUID, Boolean>>> pingMap =
+    private final ConcurrentMap<InetSocketAddress, IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>>> pingMap =
         new ConcurrentHashMap8<>();
 
     /** Debug mode. */
@@ -1169,7 +1169,7 @@ public class TcpDiscoverySpi extends TcpDiscoverySpiAdapter implements TcpDiscov
 
         GridFutureAdapterEx<IgniteBiTuple<UUID, Boolean>> fut = new GridFutureAdapterEx<>();
 
-        IgniteFuture<IgniteBiTuple<UUID, Boolean>> oldFut = pingMap.putIfAbsent(addr, fut);
+        IgniteInternalFuture<IgniteBiTuple<UUID, Boolean>> oldFut = pingMap.putIfAbsent(addr, fut);
 
         if (oldFut != null)
             return oldFut.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/GridTestTaskSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/GridTestTaskSession.java b/modules/core/src/test/java/org/apache/ignite/GridTestTaskSession.java
index b1d495a..b1b0825 100644
--- a/modules/core/src/test/java/org/apache/ignite/GridTestTaskSession.java
+++ b/modules/core/src/test/java/org/apache/ignite/GridTestTaskSession.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite;
 
-import org.apache.ignite.*;
 import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
@@ -213,7 +213,7 @@ public class GridTestTaskSession implements ComputeTaskSession {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> mapFuture() {
+    @Override public IgniteInternalFuture<?> mapFuture() {
         assert false : "Not implemented";
 
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
index 0547573..34d5aba 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/GridCacheBalancingStoreSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.cache.store;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
@@ -47,7 +48,7 @@ public class GridCacheBalancingStoreSelfTest extends GridCommonAbstractTest {
 
         final AtomicBoolean finish = new AtomicBoolean();
 
-        IgniteFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new IgniteCallable<Void>() {
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new IgniteCallable<Void>() {
             @Override public Void call() throws Exception {
                 try {
                     ThreadLocalRandom rnd = ThreadLocalRandom.current();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
index 38a87e4..09c4e4f 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
@@ -23,7 +23,6 @@ import org.apache.ignite.cache.store.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
@@ -113,7 +112,7 @@ public class GridCacheJdbcBlobStoreMultithreadedSelfTest extends GridCommonAbstr
      * @throws Exception If failed.
      */
     public void testMultithreadedPut() throws Exception {
-        IgniteFuture<?> fut1 = runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut1 = runMultiThreadedAsync(new Callable<Object>() {
             private final Random rnd = new Random();
 
             @Override public Object call() throws Exception {
@@ -127,7 +126,7 @@ public class GridCacheJdbcBlobStoreMultithreadedSelfTest extends GridCommonAbstr
             }
         }, 4, "put");
 
-        IgniteFuture<?> fut2 = runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut2 = runMultiThreadedAsync(new Callable<Object>() {
             private final Random rnd = new Random();
 
             @Override public Object call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
index f79a68a..f184821 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridJobMasterLeaveAwareSelfTest.java
@@ -253,8 +253,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testApply1() throws Exception {
-        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup grid) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup grid) throws IgniteCheckedException {
                 IgniteCompute comp = compute(grid).withAsync();
 
                 comp.apply(new TestClosure(), "arg");
@@ -268,8 +268,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testApply2() throws Exception {
-        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup grid) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup grid) throws IgniteCheckedException {
                 IgniteCompute comp = compute(grid).withAsync();
 
                 comp.apply(new TestClosure(), Arrays.asList("arg1", "arg2"));
@@ -283,8 +283,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testApply3() throws Exception {
-        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup grid) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup grid) throws IgniteCheckedException {
                 IgniteCompute comp = compute(grid).withAsync();
 
                 comp.apply(new TestClosure(),
@@ -308,8 +308,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testRun1() throws Exception {
-        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 comp.run(new TestRunnable());
@@ -323,8 +323,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testRun2() throws Exception {
-        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 comp.run(Arrays.asList(new TestRunnable(), new TestRunnable()));
@@ -338,8 +338,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCall1() throws Exception {
-        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 comp.call(new TestCallable());
@@ -353,8 +353,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCall2() throws Exception {
-        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 comp.call(Arrays.asList(new TestCallable(), new TestCallable()));
@@ -368,8 +368,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testCall3() throws Exception {
-        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(2, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 comp.call(
@@ -393,8 +393,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testBroadcast1() throws Exception {
-        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 comp.broadcast(new TestRunnable());
@@ -408,8 +408,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testBroadcast2() throws Exception {
-        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 comp.broadcast(new TestCallable());
@@ -423,8 +423,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testBroadcast3() throws Exception {
-        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 comp.broadcast(new TestClosure(), "arg");
@@ -438,8 +438,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAffinityRun() throws Exception {
-        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 CacheAffinity<Object> aff = prj.ignite().cache(null).affinity();
@@ -457,8 +457,8 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testAffinityCall() throws Exception {
-        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteFuture<?>>() {
-            @Override public IgniteFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
+        testMasterLeaveAwareCallback(1, new CX1<ClusterGroup, IgniteInternalFuture<?>>() {
+            @Override public IgniteInternalFuture<?> applyx(ClusterGroup prj) throws IgniteCheckedException {
                 IgniteCompute comp = compute(prj).withAsync();
 
                 CacheAffinity<Object> aff = prj.ignite().cache(null).affinity();
@@ -500,7 +500,7 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
      * @param taskStarter Task started.
      * @throws Exception If failed.
      */
-    private void testMasterLeaveAwareCallback(int expJobs, IgniteClosure<ClusterGroup, IgniteFuture<?>> taskStarter)
+    private void testMasterLeaveAwareCallback(int expJobs, IgniteClosure<ClusterGroup, IgniteInternalFuture<?>> taskStarter)
         throws Exception {
         jobLatch = new CountDownLatch(expJobs);
         invokeLatch  = new CountDownLatch(expJobs);
@@ -510,7 +510,7 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
 
         int lastGridIdx = GRID_CNT - 1;
 
-        IgniteFuture<?> fut = taskStarter.apply(grid(lastGridIdx).forPredicate(excludeLastPredicate()));
+        IgniteInternalFuture<?> fut = taskStarter.apply(grid(lastGridIdx).forPredicate(excludeLastPredicate()));
 
         jobLatch.await();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java
index d6ec8ad..8b174c5 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridMultipleJobsSelfTest.java
@@ -152,13 +152,13 @@ public class GridMultipleJobsSelfTest extends GridCommonAbstractTest {
 
                     comp.call(job);
 
-                    IgniteFuture<Boolean> fut = comp.future();
+                    IgniteInternalFuture<Boolean> fut = comp.future();
 
                     if (cnt % LOG_MOD == 0)
                         X.println("Submitted jobs: " + cnt);
 
-                    fut.listenAsync(new CIX1<IgniteFuture<Boolean>>() {
-                        @Override public void applyx(IgniteFuture<Boolean> f) throws IgniteCheckedException {
+                    fut.listenAsync(new CIX1<IgniteInternalFuture<Boolean>>() {
+                        @Override public void applyx(IgniteInternalFuture<Boolean> f) throws IgniteCheckedException {
                             try {
                                 assert f.get();
                             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
index 9da8a0b..8350f81 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridProjectionAbstractTest.java
@@ -327,7 +327,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
 
         comp.broadcast(runJob);
 
-        IgniteFuture fut = comp.future();
+        IgniteInternalFuture fut = comp.future();
 
         waitForExecution(fut);
 
@@ -349,7 +349,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
 
         comp.run(jobs);
 
-        IgniteFuture fut = comp.future();
+        IgniteInternalFuture fut = comp.future();
 
         waitForExecution(fut);
 
@@ -369,7 +369,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
 
         comp.broadcast(calJob);
 
-        IgniteFuture fut = comp.future();
+        IgniteInternalFuture fut = comp.future();
 
         waitForExecution(fut);
 
@@ -391,7 +391,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
 
         comp.call(jobs);
 
-        IgniteFuture fut = comp.future();
+        IgniteInternalFuture fut = comp.future();
 
         waitForExecution(fut);
 
@@ -411,7 +411,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
 
         comp.apply(clrJob, (String) null);
 
-        IgniteFuture fut = comp.future();
+        IgniteInternalFuture fut = comp.future();
 
         waitForExecution(fut);
 
@@ -433,7 +433,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
 
         comp.apply(clrJob, args);
 
-        IgniteFuture fut = comp.future();
+        IgniteInternalFuture fut = comp.future();
 
         waitForExecution(fut);
 
@@ -453,7 +453,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
 
         comp.broadcast(new TestClosure(), "arg");
 
-        IgniteFuture<Collection<String>> fut = comp.future();
+        IgniteInternalFuture<Collection<String>> fut = comp.future();
 
         waitForExecution(fut);
 
@@ -480,7 +480,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
 
         comp.apply(clrJob, args, rdc);
 
-        IgniteFuture fut = comp.future();
+        IgniteInternalFuture fut = comp.future();
 
         waitForExecution(fut);
 
@@ -502,7 +502,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
 
         comp.call(jobs, rdc);
 
-        IgniteFuture fut = comp.future();
+        IgniteInternalFuture fut = comp.future();
 
         waitForExecution(fut);
 
@@ -602,7 +602,7 @@ public abstract class GridProjectionAbstractTest extends GridCommonAbstractTest
      * @throws InterruptedException Thrown if wait was interrupted.
      */
     @SuppressWarnings({"UnconditionalWait"})
-    private void waitForExecution(IgniteFuture fut) throws InterruptedException {
+    private void waitForExecution(IgniteInternalFuture fut) throws InterruptedException {
         long sleep = 250;
 
         long threshold = System.currentTimeMillis() + WAIT_TIMEOUT;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/internal/GridReduceSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridReduceSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridReduceSelfTest.java
index db2e960..1a5c7fc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridReduceSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridReduceSelfTest.java
@@ -108,7 +108,7 @@ public class GridReduceSelfTest extends GridCommonAbstractTest {
                 }
             });
 
-            IgniteFuture<Long> fut = comp.future();
+            IgniteInternalFuture<Long> fut = comp.future();
 
             assertEquals((Long)1L, fut.get());
 


[50/50] [abbrv] incubator-ignite git commit: #ignite-16: WIP.

Posted by vo...@apache.org.
#ignite-16: WIP.


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

Branch: refs/heads/ignite-16
Commit: d0444b79c5028074128f56f555b0414041dc878d
Parents: 9ea9607
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Fri Jan 30 17:05:00 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Fri Jan 30 17:05:00 2015 +0300

----------------------------------------------------------------------
 .../configuration/IgniteConfiguration.java      |  22 ----
 .../ignite/internal/GridKernalContext.java      |   8 --
 .../ignite/internal/GridKernalContextImpl.java  |  12 --
 .../org/apache/ignite/internal/IgniteEx.java    |   8 --
 .../apache/ignite/internal/IgniteKernal.java    |   7 --
 .../org/apache/ignite/internal/IgnitionEx.java  |   1 -
 .../processors/cache/GridCacheStoreManager.java |   1 -
 .../cache/GridCacheWriteBehindStore.java        |   1 -
 .../processors/interop/GridInteropAware.java    |  49 ---------
 .../interop/GridInteropProcessor.java           |  82 --------------
 .../interop/GridInteropProcessorAdapter.java    |  31 ------
 .../processors/interop/GridInteropTarget.java   | 109 -------------------
 .../interop/os/GridOsInteropProcessor.java      |  80 --------------
 .../internal/processors/interop/os/package.html |  23 ----
 .../internal/processors/interop/package.html    |  23 ----
 .../plugin/IgnitePluginProcessor.java           |  10 +-
 16 files changed, 6 insertions(+), 461 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
index 778ca62..e6d513b 100644
--- a/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/configuration/IgniteConfiguration.java
@@ -389,9 +389,6 @@ public class IgniteConfiguration {
     /** Transactions configuration. */
     private TransactionsConfiguration txCfg = new TransactionsConfiguration();
 
-    /** Interop configuration. */
-    private InteropConfiguration interopCfg;
-
     /** */
     private Collection<? extends PluginConfiguration> pluginCfgs;
 
@@ -585,7 +582,6 @@ public class IgniteConfiguration {
         hadoopCfg = cfg.getHadoopConfiguration();
         inclEvtTypes = cfg.getIncludeEventTypes();
         includeProps = cfg.getIncludeProperties();
-        interopCfg = cfg.getInteropConfiguration() != null ? cfg.getInteropConfiguration().copy() : null;
         jettyPath = cfg.getRestJettyPath();
         licUrl = cfg.getLicenseUrl();
         lifecycleBeans = cfg.getLifecycleBeans();
@@ -3117,24 +3113,6 @@ public class IgniteConfiguration {
     }
 
     /**
-     * Gets interop configuration.
-     *
-     * @return Interop configuration.
-     */
-    @Nullable public InteropConfiguration getInteropConfiguration() {
-        return interopCfg;
-    }
-
-    /**
-     * Sets interop configuration.
-     *
-     * @param interopCfg Interop configuration.
-     */
-    public void setInteropConfiguration(@Nullable InteropConfiguration interopCfg) {
-        this.interopCfg = interopCfg;
-    }
-
-    /**
      * Gets transactions configuration.
      *
      * @return Transactions configuration.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
index 17f3ec8..58923b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContext.java
@@ -43,7 +43,6 @@ import org.apache.ignite.internal.processors.continuous.*;
 import org.apache.ignite.internal.processors.dataload.*;
 import org.apache.ignite.internal.processors.email.*;
 import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.interop.*;
 import org.apache.ignite.internal.processors.job.*;
 import org.apache.ignite.internal.processors.jobmetrics.*;
 import org.apache.ignite.internal.processors.license.*;
@@ -330,13 +329,6 @@ public interface GridKernalContext extends Iterable<GridComponent> {
     public GridPortableProcessor portable();
 
     /**
-     * Gets interop processor.
-     *
-     * @return Interop processor.
-     */
-    public GridInteropProcessor interop();
-
-    /**
      * Gets query processor.
      *
      * @return Query processor.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
index 88dbef6..13eb341 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernalContextImpl.java
@@ -46,7 +46,6 @@ import org.apache.ignite.internal.processors.continuous.*;
 import org.apache.ignite.internal.processors.dataload.*;
 import org.apache.ignite.internal.processors.email.*;
 import org.apache.ignite.internal.processors.hadoop.*;
-import org.apache.ignite.internal.processors.interop.*;
 import org.apache.ignite.internal.processors.job.*;
 import org.apache.ignite.internal.processors.jobmetrics.*;
 import org.apache.ignite.internal.processors.license.*;
@@ -255,10 +254,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
 
     /** */
     @GridToStringExclude
-    private GridInteropProcessor interopProc;
-
-    /** */
-    @GridToStringExclude
     private IgniteSpringProcessor spring;
 
     /** */
@@ -444,8 +439,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
             hadoopProc = (IgniteHadoopProcessorAdapter)comp;
         else if (comp instanceof GridPortableProcessor)
             portableProc = (GridPortableProcessor)comp;
-        else if (comp instanceof GridInteropProcessor)
-            interopProc = (GridInteropProcessor)comp;
         else if (comp instanceof IgnitePluginProcessor)
             pluginProc = (IgnitePluginProcessor)comp;
         else if (comp instanceof GridQueryProcessor)
@@ -697,11 +690,6 @@ public class GridKernalContextImpl implements GridKernalContext, Externalizable
     }
 
     /** {@inheritDoc} */
-    @Override public GridInteropProcessor interop() {
-        return interopProc;
-    }
-
-    /** {@inheritDoc} */
     @Override public GridQueryProcessor query() {
         return qryProc;
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
index 6aa8bf5..aa94560 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteEx.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
-import org.apache.ignite.internal.processors.interop.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -133,11 +132,4 @@ public interface IgniteEx extends Ignite, ClusterGroupEx, IgniteCluster {
      * @return GGFS.
      */
     @Nullable public IgniteFs ggfsx(@Nullable String name);
-
-    /**
-     * Gets interop processor.
-     *
-     * @return Interop processor.
-     */
-    public GridInteropProcessor interop();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
index 7854bd9..0df86f9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteKernal.java
@@ -56,7 +56,6 @@ import org.apache.ignite.internal.processors.closure.*;
 import org.apache.ignite.internal.processors.continuous.*;
 import org.apache.ignite.internal.processors.dataload.*;
 import org.apache.ignite.internal.processors.email.*;
-import org.apache.ignite.internal.processors.interop.*;
 import org.apache.ignite.internal.processors.job.*;
 import org.apache.ignite.internal.processors.jobmetrics.*;
 import org.apache.ignite.internal.processors.license.*;
@@ -742,7 +741,6 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
             startProcessor(ctx, new GridTaskProcessor(ctx), attrs);
             startProcessor(ctx, (GridProcessor)SCHEDULE.createOptional(ctx), attrs);
             startProcessor(ctx, createComponent(GridPortableProcessor.class, ctx), attrs);
-            startProcessor(ctx, createComponent(GridInteropProcessor.class, ctx), attrs);
             startProcessor(ctx, new GridRestProcessor(ctx), attrs);
             startProcessor(ctx, new GridDataLoaderProcessor(ctx), attrs);
             startProcessor(ctx, new GridStreamProcessor(ctx), attrs);
@@ -3104,11 +3102,6 @@ public class IgniteKernal extends ClusterGroupAdapter implements IgniteEx, Ignit
     }
 
     /** {@inheritDoc} */
-    @Override public GridInteropProcessor interop() {
-        return ctx.interop();
-    }
-
-    /** {@inheritDoc} */
     @Override public <K> Map<ClusterNode, Collection<K>> mapKeysToNodes(String cacheName,
         @Nullable Collection<? extends K> keys) throws IgniteCheckedException {
         if (F.isEmpty(keys))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
index d962de0..3857851 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgnitionEx.java
@@ -1418,7 +1418,6 @@ public class IgnitionEx {
             myCfg.setSecurityCredentialsProvider(cfg.getSecurityCredentialsProvider());
             myCfg.setServiceConfiguration(cfg.getServiceConfiguration());
             myCfg.setWarmupClosure(cfg.getWarmupClosure());
-            myCfg.setInteropConfiguration(cfg.getInteropConfiguration());
             myCfg.setPluginConfigurations(cfg.getPluginConfigurations());
             myCfg.setTransactionsConfiguration(new TransactionsConfiguration(cfg.getTransactionsConfiguration()));
             myCfg.setQueryConfiguration(cfg.getQueryConfiguration());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
index 61408cb..f7fe02e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheStoreManager.java
@@ -26,7 +26,6 @@ import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.transactions.*;
-import org.apache.ignite.internal.processors.interop.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java
index a31d3b5..4999ce0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheWriteBehindStore.java
@@ -24,7 +24,6 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.thread.*;
-import org.apache.ignite.internal.processors.interop.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.internal.util.tostring.*;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropAware.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropAware.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropAware.java
deleted file mode 100644
index 81035d8..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropAware.java
+++ /dev/null
@@ -1,49 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.interop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-
-/**
- * Interface for interop-aware components.
- */
-public interface GridInteropAware {
-    /**
-     * Sets configuration parameters.
-     *
-     * @param params Configuration parameters.
-     */
-    public void configure(Object... params);
-
-    /**
-     * Initializes interop-aware component.
-     *
-     * @param ctx Context.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public void initialize(GridKernalContext ctx) throws IgniteCheckedException;
-
-    /**
-     * Destroys interop-aware component.
-     *
-     * @param ctx Context.
-     * @throws IgniteCheckedException In case of error.
-     */
-    public void destroy(GridKernalContext ctx) throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropProcessor.java
deleted file mode 100644
index 33c841a..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropProcessor.java
+++ /dev/null
@@ -1,82 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.interop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.*;
-import org.jetbrains.annotations.*;
-
-/**
- * Interop processor.
- */
-public interface GridInteropProcessor extends GridProcessor {
-    /**
-     * Release start latch.
-     */
-    public void releaseStart();
-
-    /**
-     * Await start on native side.
-     *
-     * @throws IgniteCheckedException If failed.
-     */
-    public void awaitStart() throws IgniteCheckedException;
-
-    /**
-     * @return Environment pointer.
-     */
-    public long environmentPointer() throws IgniteCheckedException;
-
-    /**
-     * @return Grid name.
-     */
-    public String gridName();
-
-    /**
-     * Gets native wrapper for default Grid projection.
-     *
-     * @return Native compute wrapper.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridInteropTarget projection() throws IgniteCheckedException;
-
-    /**
-     * Gets native wrapper for cache with the given name.
-     *
-     * @param name Cache name ({@code null} for default cache).
-     * @return Native cache wrapper.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridInteropTarget cache(@Nullable String name) throws IgniteCheckedException;
-
-    /**
-     * Gets native wrapper for data loader for cache with the given name.
-     *
-     * @param cacheName Cache name ({@code null} for default cache).
-     * @return Native data loader wrapper.
-     * @throws IgniteCheckedException If failed.
-     */
-    public GridInteropTarget dataLoader(@Nullable String cacheName) throws IgniteCheckedException;
-
-    /**
-     * Stops grid.
-     *
-     * @param cancel Cancel flag.
-     */
-    public void close(boolean cancel);
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropProcessorAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropProcessorAdapter.java
deleted file mode 100644
index 91ea27e..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropProcessorAdapter.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.interop;
-
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.*;
-
-/**
- * Interop processor adapter.
- */
-public abstract class GridInteropProcessorAdapter extends GridProcessorAdapter implements GridInteropProcessor {
-    /** {@inheritDoc} */
-    protected GridInteropProcessorAdapter(GridKernalContext ctx) {
-        super(ctx);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropTarget.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropTarget.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropTarget.java
deleted file mode 100644
index af8b5d2..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/GridInteropTarget.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.interop;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.processors.portable.*;
-import org.jetbrains.annotations.*;
-
-/**
- * Interop target abstraction.
- */
-public interface GridInteropTarget {
-    /**
-     * Synchronous IN operation.
-     *
-     * @param type Operation type.
-     * @param stream Input stream.
-     * @return Value specific for the given operation otherwise.
-     * @throws IgniteCheckedException In case of failure.
-     */
-    public int inOp(int type, GridPortableInputStream stream) throws IgniteCheckedException;
-
-    /**
-     * Synchronous IN operation which returns managed object as result.
-     *
-     * @param type Operation type.
-     * @param stream Input stream.
-     * @return Managed result.
-     * @throws IgniteCheckedException If case of failure.
-     */
-    public Object inOpObject(int type, GridPortableInputStream stream) throws IgniteCheckedException;
-
-    /**
-     * Synchronous OUT operation.
-     *
-     * @param type Operation type.
-     * @param stream Native stream address.
-     * @param arr Native array address.
-     * @param cap Capacity.
-     * @throws IgniteCheckedException In case of failure.
-     */
-    public void outOp(int type, long stream, long arr, int cap) throws IgniteCheckedException;
-
-    /**
-     * Synchronous IN-OUT operation.
-     *
-     * @param type Operation type.
-     * @param inStream Input stream.
-     * @param outStream Native stream address.
-     * @param outArr Native array address.
-     * @param outCap Capacity.
-     * @throws IgniteCheckedException In case of failure.
-     */
-    public void inOutOp(int type, GridPortableInputStream inStream, long outStream, long outArr, int outCap)
-        throws IgniteCheckedException;
-
-    /**
-     * Synchronous IN-OUT operation with optional argument.
-     *
-     * @param type Operation type.
-     * @param inStream Input stream.
-     * @param outStream Native stream address.
-     * @param outArr Native array address.
-     * @param outCap Capacity.
-     * @param arg Argument (optional).
-     * @throws IgniteCheckedException In case of failure.
-     */
-    public void inOutOp(int type, GridPortableInputStream inStream, long outStream, long outArr, int outCap,
-        @Nullable Object arg) throws IgniteCheckedException;
-
-    /**
-     * Asynchronous IN operation.
-     *
-     * @param type Operation type.
-     * @param futId Future ID.
-     * @param in Input stream.
-     * @throws IgniteCheckedException In case of failure.
-     */
-    public void inOpAsync(int type, long futId, GridPortableInputStream in) throws IgniteCheckedException;
-
-    /**
-     * Asynchronous IN-OUT operation.
-     *
-     * @param type Operation type.
-     * @param futId Future ID.
-     * @param in Input stream.
-     * @param outStream Native stream address.
-     * @param outArr Native array address.
-     * @param outCap Capacity.
-     * @throws IgniteCheckedException In case of failure.
-     */
-    public void inOutOpAsync(int type, long futId, GridPortableInputStream in, long outStream, long outArr, int outCap)
-        throws IgniteCheckedException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/os/GridOsInteropProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/os/GridOsInteropProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/os/GridOsInteropProcessor.java
deleted file mode 100644
index 6a592d1..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/os/GridOsInteropProcessor.java
+++ /dev/null
@@ -1,80 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal.processors.interop.os;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.interop.*;
-import org.jetbrains.annotations.*;
-
-/**
- * OS interop processor.
- */
-public class GridOsInteropProcessor extends GridInteropProcessorAdapter {
-    /** Common error message. */
-    private static final String ERR_MSG = "Interop feature is not supported in OS edition.";
-
-    /**
-     * Constructor.
-     *
-     * @param ctx Context.
-     */
-    public GridOsInteropProcessor(GridKernalContext ctx) {
-        super(ctx);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void releaseStart() {
-        throw new UnsupportedOperationException(ERR_MSG);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void awaitStart() throws IgniteCheckedException {
-        throw new UnsupportedOperationException(ERR_MSG);
-    }
-
-    /** {@inheritDoc} */
-    @Override public long environmentPointer() throws IgniteCheckedException {
-        throw new UnsupportedOperationException(ERR_MSG);
-    }
-
-    /** {@inheritDoc} */
-    @Override public String gridName() {
-        throw new UnsupportedOperationException(ERR_MSG);
-    }
-
-    /** {@inheritDoc} */
-    @Override public void close(boolean cancel) {
-        throw new UnsupportedOperationException(ERR_MSG);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridInteropTarget projection() throws IgniteCheckedException {
-        throw new UnsupportedOperationException(ERR_MSG);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridInteropTarget cache(@Nullable String name) throws IgniteCheckedException {
-        throw new UnsupportedOperationException(ERR_MSG);
-    }
-
-    /** {@inheritDoc} */
-    @Override public GridInteropTarget dataLoader(@Nullable String cacheName) throws IgniteCheckedException {
-        throw new UnsupportedOperationException(ERR_MSG);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/os/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/os/package.html b/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/os/package.html
deleted file mode 100644
index 20815a0..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/os/package.html
+++ /dev/null
@@ -1,23 +0,0 @@
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-  -->
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<html>
-<body>
-    <!-- Package description. -->
-    No-op implementation of interop processor.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/package.html
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/package.html b/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/package.html
deleted file mode 100644
index 57a4e47..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/interop/package.html
+++ /dev/null
@@ -1,23 +0,0 @@
-<!--
-  Licensed to the Apache Software Foundation (ASF) under one or more
-  contributor license agreements.  See the NOTICE file distributed with
-  this work for additional information regarding copyright ownership.
-  The ASF licenses this file to You under the Apache License, Version 2.0
-  (the "License"); you may not use this file except in compliance with
-  the License.  You may obtain a copy of the License at
-
-       http://www.apache.org/licenses/LICENSE-2.0
-
-  Unless required by applicable law or agreed to in writing, software
-  distributed under the License is distributed on an "AS IS" BASIS,
-  WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-  See the License for the specific language governing permissions and
-  limitations under the License.
-  -->
-<!DOCTYPE html PUBLIC "-//W3C//DTD HTML 4.01 Transitional//EN" "http://www.w3.org/TR/html4/loose.dtd">
-<html>
-<body>
-    <!-- Package description. -->
-    Interop processor.
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d0444b79/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java
index 5d3ad42..9cd7943 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/plugin/IgnitePluginProcessor.java
@@ -132,8 +132,9 @@ public class IgnitePluginProcessor extends GridProcessorAdapter {
      * @param name Plugin name.
      * @return Plugin provider.
      */
-    @Nullable public PluginProvider pluginProvider(String name) {
-        return plugins.get(name);
+    @SuppressWarnings("unchecked")
+    @Nullable public <T extends PluginProvider> T pluginProvider(String name) {
+        return (T)plugins.get(name);
     }
 
     /**
@@ -147,8 +148,9 @@ public class IgnitePluginProcessor extends GridProcessorAdapter {
      * @param provider Plugin context.
      * @return Plugin context.
      */
-    public PluginContext pluginContextForProvider(PluginProvider provider) {
-        return pluginCtxMap.get(provider);
+    @SuppressWarnings("unchecked")
+    public <T extends PluginContext> T pluginContextForProvider(PluginProvider provider) {
+        return (T)pluginCtxMap.get(provider);
     }
 
     /**


[45/50] [abbrv] incubator-ignite git commit: GridNamedInstance -> IgniteNamedInstance GridgainEx -> IgnitionEx

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c5247ab3/modules/core/src/main/java/org/apache/ignite/internal/GridGainEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridGainEx.java b/modules/core/src/main/java/org/apache/ignite/internal/GridGainEx.java
deleted file mode 100644
index 65c2776..0000000
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridGainEx.java
+++ /dev/null
@@ -1,2399 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cache.*;
-import org.apache.ignite.cache.affinity.rendezvous.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.fs.*;
-import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
-import org.apache.ignite.lifecycle.*;
-import org.apache.ignite.logger.*;
-import org.apache.ignite.logger.java.*;
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.marshaller.jdk.*;
-import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.mxbean.*;
-import org.apache.ignite.spi.*;
-import org.apache.ignite.spi.authentication.*;
-import org.apache.ignite.spi.authentication.noop.*;
-import org.apache.ignite.spi.indexing.*;
-import org.apache.ignite.streamer.*;
-import org.apache.ignite.thread.*;
-import org.apache.ignite.internal.processors.resource.*;
-import org.apache.ignite.internal.processors.spring.*;
-import org.apache.ignite.plugin.segmentation.*;
-import org.apache.ignite.spi.checkpoint.*;
-import org.apache.ignite.spi.checkpoint.noop.*;
-import org.apache.ignite.spi.collision.*;
-import org.apache.ignite.spi.collision.noop.*;
-import org.apache.ignite.spi.communication.*;
-import org.apache.ignite.spi.communication.tcp.*;
-import org.apache.ignite.spi.deployment.*;
-import org.apache.ignite.spi.deployment.local.*;
-import org.apache.ignite.spi.discovery.*;
-import org.apache.ignite.spi.discovery.tcp.*;
-import org.apache.ignite.spi.discovery.tcp.ipfinder.multicast.*;
-import org.apache.ignite.spi.eventstorage.*;
-import org.apache.ignite.spi.eventstorage.memory.*;
-import org.apache.ignite.spi.failover.*;
-import org.apache.ignite.spi.failover.always.*;
-import org.apache.ignite.spi.loadbalancing.*;
-import org.apache.ignite.spi.loadbalancing.roundrobin.*;
-import org.apache.ignite.spi.securesession.*;
-import org.apache.ignite.spi.securesession.noop.*;
-import org.apache.ignite.spi.swapspace.*;
-import org.apache.ignite.spi.swapspace.file.*;
-import org.apache.ignite.spi.swapspace.noop.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.jdk8.backport.*;
-import org.jetbrains.annotations.*;
-
-import javax.management.*;
-import java.io.*;
-import java.lang.management.*;
-import java.lang.reflect.*;
-import java.net.*;
-import java.util.*;
-import java.util.Map.*;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.*;
-import java.util.logging.*;
-
-import static org.apache.ignite.configuration.IgniteConfiguration.*;
-import static org.apache.ignite.IgniteState.*;
-import static org.apache.ignite.IgniteSystemProperties.*;
-import static org.apache.ignite.cache.CacheAtomicityMode.*;
-import static org.apache.ignite.cache.CacheDistributionMode.*;
-import static org.apache.ignite.cache.CacheMode.*;
-import static org.apache.ignite.cache.CachePreloadMode.*;
-import static org.apache.ignite.cache.CacheWriteSynchronizationMode.*;
-import static org.apache.ignite.internal.IgniteComponentType.*;
-import static org.apache.ignite.plugin.segmentation.GridSegmentationPolicy.*;
-
-/**
- * This class defines a factory for the main GridGain API. It controls Grid life cycle
- * and allows listening for grid events.
- * <h1 class="header">Grid Loaders</h1>
- * Although user can apply grid factory directly to start and stop grid, grid is
- * often started and stopped by grid loaders. Grid loaders can be found in
- * {@link org.apache.ignite.startup} package, for example:
- * <ul>
- * <li>{@code GridCommandLineStartup}</li>
- * <li>{@code GridServletStartup}</li>
- * </ul>
- * <h1 class="header">Examples</h1>
- * Use {@link #start()} method to start grid with default configuration. You can also use
- * {@link org.apache.ignite.configuration.IgniteConfiguration} to override some default configuration. Below is an
- * example on how to start grid with <strong>URI deployment</strong>.
- * <pre name="code" class="java">
- * GridConfiguration cfg = new GridConfiguration();
- */
-public class GridGainEx {
-    /** Default configuration path relative to GridGain home. */
-    public static final String DFLT_CFG = "config/default-config.xml";
-
-    /** Map of named grids. */
-    private static final ConcurrentMap<Object, GridNamedInstance> grids = new ConcurrentHashMap8<>();
-
-    /** Map of grid states ever started in this JVM. */
-    private static final Map<Object, IgniteState> gridStates = new ConcurrentHashMap8<>();
-
-    /** Mutex to synchronize updates of default grid reference. */
-    private static final Object dfltGridMux = new Object();
-
-    /** Default grid. */
-    private static volatile GridNamedInstance dfltGrid;
-
-    /** Default grid state. */
-    private static volatile IgniteState dfltGridState;
-
-    /** List of state listeners. */
-    private static final Collection<IgniteListener> lsnrs = new GridConcurrentHashSet<>(4);
-
-    /** */
-    private static volatile boolean daemon;
-
-    /**
-     * Checks runtime version to be 1.7.x or 1.8.x.
-     * This will load pretty much first so we must do these checks here.
-     */
-    static {
-        // Check 1.8 just in case for forward compatibility.
-        if (!U.jdkVersion().contains("1.7") &&
-            !U.jdkVersion().contains("1.8"))
-            throw new IllegalStateException("GridGain requires Java 7 or above. Current Java version " +
-                "is not supported: " + U.jdkVersion());
-
-        // To avoid nasty race condition in UUID.randomUUID() in JDK prior to 6u34.
-        // For details please see:
-        // http://bugs.sun.com/bugdatabase/view_bug.do?bug_id=7071826
-        // http://www.oracle.com/technetwork/java/javase/2col/6u34-bugfixes-1733379.html
-        // http://hg.openjdk.java.net/jdk6/jdk6/jdk/rev/563d392b3e5c
-        UUID.randomUUID();
-    }
-
-    /**
-     * Enforces singleton.
-     */
-    private GridGainEx() {
-        // No-op.
-    }
-
-    /**
-     * Sets daemon flag.
-     * <p>
-     * If daemon flag is set then all grid instances created by the factory will be
-     * daemon, i.e. the local node for these instances will be a daemon node. Note that
-     * if daemon flag is set - it will override the same settings in {@link org.apache.ignite.configuration.IgniteConfiguration#isDaemon()}.
-     * Note that you can set on and off daemon flag at will.
-     *
-     * @param daemon Daemon flag to set.
-     */
-    public static void setDaemon(boolean daemon) {
-        GridGainEx.daemon = daemon;
-    }
-
-    /**
-     * Gets daemon flag.
-     * <p>
-     * If daemon flag it set then all grid instances created by the factory will be
-     * daemon, i.e. the local node for these instances will be a daemon node. Note that
-     * if daemon flag is set - it will override the same settings in {@link org.apache.ignite.configuration.IgniteConfiguration#isDaemon()}.
-     * Note that you can set on and off daemon flag at will.
-     *
-     * @return Daemon flag.
-     */
-    public static boolean isDaemon() {
-        return daemon;
-    }
-
-    /**
-     * Gets state of grid default grid.
-     *
-     * @return Default grid state.
-     */
-    public static IgniteState state() {
-        return state(null);
-    }
-
-    /**
-     * Gets states of named grid. If name is {@code null}, then state of
-     * default no-name grid is returned.
-     *
-     * @param name Grid name. If name is {@code null}, then state of
-     *      default no-name grid is returned.
-     * @return Grid state.
-     */
-    public static IgniteState state(@Nullable String name) {
-        GridNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
-
-        if (grid == null) {
-            IgniteState state = name != null ? gridStates.get(name) : dfltGridState;
-
-            return state != null ? state : STOPPED;
-        }
-
-        return grid.state();
-    }
-
-    /**
-     * Stops default grid. This method is identical to {@code G.stop(null, cancel)} apply.
-     * Note that method does not wait for all tasks to be completed.
-     *
-     * @param cancel If {@code true} then all jobs currently executing on
-     *      default grid will be cancelled by calling {@link org.apache.ignite.compute.ComputeJob#cancel()}
-     *      method. Note that just like with {@link Thread#interrupt()}, it is
-     *      up to the actual job to exit from execution
-     * @return {@code true} if default grid instance was indeed stopped,
-     *      {@code false} otherwise (if it was not started).
-     */
-    public static boolean stop(boolean cancel) {
-        return stop(null, cancel);
-    }
-
-    /**
-     * Stops named grid. If {@code cancel} flag is set to {@code true} then
-     * all jobs currently executing on local node will be interrupted. If
-     * grid name is {@code null}, then default no-name grid will be stopped.
-     * If wait parameter is set to {@code true} then grid will wait for all
-     * tasks to be finished.
-     *
-     * @param name Grid name. If {@code null}, then default no-name grid will
-     *      be stopped.
-     * @param cancel If {@code true} then all jobs currently will be cancelled
-     *      by calling {@link org.apache.ignite.compute.ComputeJob#cancel()} method. Note that just like with
-     *      {@link Thread#interrupt()}, it is up to the actual job to exit from
-     *      execution. If {@code false}, then jobs currently running will not be
-     *      canceled. In either case, grid node will wait for completion of all
-     *      jobs running on it before stopping.
-     * @return {@code true} if named grid instance was indeed found and stopped,
-     *      {@code false} otherwise (the instance with given {@code name} was
-     *      not found).
-     */
-    public static boolean stop(@Nullable String name, boolean cancel) {
-        GridNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
-
-        if (grid != null && grid.state() == STARTED) {
-            grid.stop(cancel);
-
-            boolean fireEvt;
-
-            if (name != null)
-                fireEvt = grids.remove(name, grid);
-            else {
-                synchronized (dfltGridMux) {
-                    fireEvt = dfltGrid == grid;
-
-                    if (fireEvt)
-                        dfltGrid = null;
-                }
-            }
-
-            if (fireEvt)
-                notifyStateChange(grid.getName(), grid.state());
-
-            return true;
-        }
-
-        // We don't have log at this point...
-        U.warn(null, "Ignoring stopping grid instance that was already stopped or never started: " + name);
-
-        return false;
-    }
-
-    /**
-     * Stops <b>all</b> started grids. If {@code cancel} flag is set to {@code true} then
-     * all jobs currently executing on local node will be interrupted.
-     * If wait parameter is set to {@code true} then grid will wait for all
-     * tasks to be finished.
-     * <p>
-     * <b>Note:</b> it is usually safer and more appropriate to stop grid instances individually
-     * instead of blanket operation. In most cases, the party that started the grid instance
-     * should be responsible for stopping it.
-     *
-     * @param cancel If {@code true} then all jobs currently executing on
-     *      all grids will be cancelled by calling {@link org.apache.ignite.compute.ComputeJob#cancel()}
-     *      method. Note that just like with {@link Thread#interrupt()}, it is
-     *      up to the actual job to exit from execution
-     */
-    public static void stopAll(boolean cancel) {
-        GridNamedInstance dfltGrid0 = dfltGrid;
-
-        if (dfltGrid0 != null) {
-            dfltGrid0.stop(cancel);
-
-            boolean fireEvt;
-
-            synchronized (dfltGridMux) {
-                fireEvt = dfltGrid == dfltGrid0;
-
-                if (fireEvt)
-                    dfltGrid = null;
-            }
-
-            if (fireEvt)
-                notifyStateChange(dfltGrid0.getName(), dfltGrid0.state());
-        }
-
-        // Stop the rest and clear grids map.
-        for (GridNamedInstance grid : grids.values()) {
-            grid.stop(cancel);
-
-            boolean fireEvt = grids.remove(grid.getName(), grid);
-
-            if (fireEvt)
-                notifyStateChange(grid.getName(), grid.state());
-        }
-    }
-
-    /**
-     * Restarts <b>all</b> started grids. If {@code cancel} flag is set to {@code true} then
-     * all jobs currently executing on the local node will be interrupted.
-     * If {@code wait} parameter is set to {@code true} then grid will wait for all
-     * tasks to be finished.
-     * <p>
-     * <b>Note:</b> it is usually safer and more appropriate to stop grid instances individually
-     * instead of blanket operation. In most cases, the party that started the grid instance
-     * should be responsible for stopping it.
-     * <p>
-     * Note also that restarting functionality only works with the tools that specifically
-     * support GridGain's protocol for restarting. Currently only standard <tt>ggstart.{sh|bat}</tt>
-     * scripts support restarting of JVM GridGain's process.
-     *
-     * @param cancel If {@code true} then all jobs currently executing on
-     *      all grids will be cancelled by calling {@link org.apache.ignite.compute.ComputeJob#cancel()}
-     *      method. Note that just like with {@link Thread#interrupt()}, it is
-     *      up to the actual job to exit from execution.
-     * @see org.apache.ignite.Ignition#RESTART_EXIT_CODE
-     */
-    public static void restart(boolean cancel) {
-        String file = System.getProperty(GG_SUCCESS_FILE);
-
-        if (file == null)
-            U.warn(null, "Cannot restart node when restart not enabled.");
-        else {
-            try {
-                new File(file).createNewFile();
-            }
-            catch (IOException e) {
-                U.error(null, "Failed to create restart marker file (restart aborted): " + e.getMessage());
-
-                return;
-            }
-
-            U.log(null, "Restarting node. Will exit (" + Ignition.RESTART_EXIT_CODE + ").");
-
-            // Set the exit code so that shell process can recognize it and loop
-            // the start up sequence again.
-            System.setProperty(GG_RESTART_CODE, Integer.toString(Ignition.RESTART_EXIT_CODE));
-
-            stopAll(cancel);
-
-            // This basically leaves loaders hang - we accept it.
-            System.exit(Ignition.RESTART_EXIT_CODE);
-        }
-    }
-
-    /**
-     * Stops <b>all</b> started grids. If {@code cancel} flag is set to {@code true} then
-     * all jobs currently executing on the local node will be interrupted.
-     * If {@code wait} parameter is set to {@code true} then grid will wait for all
-     * tasks to be finished.
-     * <p>
-     * <b>Note:</b> it is usually safer and more appropriate to stop grid instances individually
-     * instead of blanket operation. In most cases, the party that started the grid instance
-     * should be responsible for stopping it.
-     * <p>
-     * Note that upon completion of this method, the JVM with forcefully exist with
-     * exit code {@link org.apache.ignite.Ignition#KILL_EXIT_CODE}.
-     *
-     * @param cancel If {@code true} then all jobs currently executing on
-     *      all grids will be cancelled by calling {@link org.apache.ignite.compute.ComputeJob#cancel()}
-     *      method. Note that just like with {@link Thread#interrupt()}, it is
-     *      up to the actual job to exit from execution.
-     * @see org.apache.ignite.Ignition#KILL_EXIT_CODE
-     */
-    public static void kill(boolean cancel) {
-        stopAll(cancel);
-
-        // This basically leaves loaders hang - we accept it.
-        System.exit(Ignition.KILL_EXIT_CODE);
-    }
-
-    /**
-     * Starts grid with default configuration. By default this method will
-     * use grid configuration defined in {@code GRIDGAIN_HOME/config/default-config.xml}
-     * configuration file. If such file is not found, then all system defaults will be used.
-     *
-     * @return Started grid.
-     * @throws IgniteCheckedException If default grid could not be started. This exception will be thrown
-     *      also if default grid has already been started.
-     */
-    public static Ignite start() throws IgniteCheckedException {
-        return start((GridSpringResourceContext)null);
-    }
-
-    /**
-     * Starts grid with default configuration. By default this method will
-     * use grid configuration defined in {@code GRIDGAIN_HOME/config/default-config.xml}
-     * configuration file. If such file is not found, then all system defaults will be used.
-     *
-     * @param springCtx Optional Spring application context, possibly {@code null}.
-     *      Spring bean definitions for bean injection are taken from this context.
-     *      If provided, this context can be injected into grid tasks and grid jobs using
-     *      {@link org.apache.ignite.resources.IgniteSpringApplicationContextResource @IgniteSpringApplicationContextResource} annotation.
-     * @return Started grid.
-     * @throws IgniteCheckedException If default grid could not be started. This exception will be thrown
-     *      also if default grid has already been started.
-     */
-    public static Ignite start(@Nullable GridSpringResourceContext springCtx) throws IgniteCheckedException {
-        URL url = U.resolveGridGainUrl(DFLT_CFG);
-
-        if (url != null)
-            return start(DFLT_CFG, null, springCtx);
-
-        U.warn(null, "Default Spring XML file not found (is GRIDGAIN_HOME set?): " + DFLT_CFG);
-
-        return start0(new GridStartContext(new IgniteConfiguration(), null, springCtx)).grid();
-    }
-
-    /**
-     * Starts grid with given configuration. Note that this method is no-op if grid with the name
-     * provided in given configuration is already started.
-     *
-     * @param cfg Grid configuration. This cannot be {@code null}.
-     * @return Started grid.
-     * @throws IgniteCheckedException If grid could not be started. This exception will be thrown
-     *      also if named grid has already been started.
-     */
-    public static Ignite start(IgniteConfiguration cfg) throws IgniteCheckedException {
-        return start(cfg, null);
-    }
-
-    /**
-     * Starts grid with given configuration. Note that this method is no-op if grid with the name
-     * provided in given configuration is already started.
-     *
-     * @param cfg Grid configuration. This cannot be {@code null}.
-     * @param springCtx Optional Spring application context, possibly {@code null}.
-     *      Spring bean definitions for bean injection are taken from this context.
-     *      If provided, this context can be injected into grid tasks and grid jobs using
-     *      {@link org.apache.ignite.resources.IgniteSpringApplicationContextResource @IgniteSpringApplicationContextResource} annotation.
-     * @return Started grid.
-     * @throws IgniteCheckedException If grid could not be started. This exception will be thrown
-     *      also if named grid has already been started.
-     */
-    public static Ignite start(IgniteConfiguration cfg, @Nullable GridSpringResourceContext springCtx) throws IgniteCheckedException {
-        A.notNull(cfg, "cfg");
-
-        return start0(new GridStartContext(cfg, null, springCtx)).grid();
-    }
-
-    /**
-     * Starts all grids specified within given Spring XML configuration file. If grid with given name
-     * is already started, then exception is thrown. In this case all instances that may
-     * have been started so far will be stopped too.
-     * <p>
-     * Usually Spring XML configuration file will contain only one Grid definition. Note that
-     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
-     * the Grid configuration bean is ignored.
-     *
-     * @param springCfgPath Spring XML configuration file path or URL.
-     * @return Started grid. If Spring configuration contains multiple grid instances,
-     *      then the 1st found instance is returned.
-     * @throws IgniteCheckedException If grid could not be started or configuration
-     *      read. This exception will be thrown also if grid with given name has already
-     *      been started or Spring XML configuration file is invalid.
-     */
-    public static Ignite start(@Nullable String springCfgPath) throws IgniteCheckedException {
-        return springCfgPath == null ? start() : start(springCfgPath, null);
-    }
-
-    /**
-     * Starts all grids specified within given Spring XML configuration file. If grid with given name
-     * is already started, then exception is thrown. In this case all instances that may
-     * have been started so far will be stopped too.
-     * <p>
-     * Usually Spring XML configuration file will contain only one Grid definition. Note that
-     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
-     * the Grid configuration bean is ignored.
-     *
-     * @param springCfgPath Spring XML configuration file path or URL.
-     * @param gridName Grid name that will override default.
-     * @return Started grid. If Spring configuration contains multiple grid instances,
-     *      then the 1st found instance is returned.
-     * @throws IgniteCheckedException If grid could not be started or configuration
-     *      read. This exception will be thrown also if grid with given name has already
-     *      been started or Spring XML configuration file is invalid.
-     */
-    public static Ignite start(@Nullable String springCfgPath, @Nullable String gridName) throws IgniteCheckedException {
-        if (springCfgPath == null) {
-            IgniteConfiguration cfg = new IgniteConfiguration();
-
-            if (cfg.getGridName() == null && !F.isEmpty(gridName))
-                cfg.setGridName(gridName);
-
-            return start(cfg);
-        }
-        else
-            return start(springCfgPath, gridName, null);
-    }
-
-    /**
-     * Start Grid for interop scenario.
-     *
-     * @param springCfgPath Spring config path.
-     * @param gridName Grid name.
-     * @param cfgClo Configuration closure.
-     * @return Started Grid.
-     * @throws IgniteCheckedException If failed.
-     */
-    public static Ignite startInterop(@Nullable String springCfgPath, @Nullable String gridName,
-        IgniteClosure<IgniteConfiguration, IgniteConfiguration> cfgClo) throws IgniteCheckedException {
-        URL url = resolveSpringUrl(springCfgPath);
-
-        return start(url, gridName, null, cfgClo);
-    }
-
-    /**
-     * Loads all grid configurations specified within given Spring XML configuration file.
-     * <p>
-     * Usually Spring XML configuration file will contain only one Grid definition. Note that
-     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
-     * the Grid configuration bean is ignored.
-     *
-     * @param springCfgUrl Spring XML configuration file path or URL. This cannot be {@code null}.
-     * @return Tuple containing all loaded configurations and Spring context used to load them.
-     * @throws IgniteCheckedException If grid could not be started or configuration
-     *      read. This exception will be thrown also if grid with given name has already
-     *      been started or Spring XML configuration file is invalid.
-     */
-    public static IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> loadConfigurations(
-        URL springCfgUrl) throws IgniteCheckedException {
-        IgniteSpringProcessor spring = SPRING.create(false);
-
-        return spring.loadConfigurations(springCfgUrl);
-    }
-
-    /**
-     * Loads all grid configurations specified within given Spring XML configuration file.
-     * <p>
-     * Usually Spring XML configuration file will contain only one Grid definition. Note that
-     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
-     * the Grid configuration bean is ignored.
-     *
-     * @param springCfgPath Spring XML configuration file path. This cannot be {@code null}.
-     * @return Tuple containing all loaded configurations and Spring context used to load them.
-     * @throws IgniteCheckedException If grid could not be started or configuration
-     *      read. This exception will be thrown also if grid with given name has already
-     *      been started or Spring XML configuration file is invalid.
-     */
-    public static IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> loadConfigurations(
-        String springCfgPath) throws IgniteCheckedException {
-        A.notNull(springCfgPath, "springCfgPath");
-
-        URL url;
-
-        try {
-            url = new URL(springCfgPath);
-        }
-        catch (MalformedURLException e) {
-            url = U.resolveGridGainUrl(springCfgPath);
-
-            if (url == null)
-                throw new IgniteCheckedException("Spring XML configuration path is invalid: " + springCfgPath +
-                    ". Note that this path should be either absolute or a relative local file system path, " +
-                    "relative to META-INF in classpath or valid URL to GRIDGAIN_HOME.", e);
-        }
-
-        return loadConfigurations(url);
-    }
-
-    /**
-     * Loads first found grid configuration specified within given Spring XML configuration file.
-     * <p>
-     * Usually Spring XML configuration file will contain only one Grid definition. Note that
-     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
-     * the Grid configuration bean is ignored.
-     *
-     * @param springCfgUrl Spring XML configuration file path or URL. This cannot be {@code null}.
-     * @return First found configuration and Spring context used to load it.
-     * @throws IgniteCheckedException If grid could not be started or configuration
-     *      read. This exception will be thrown also if grid with given name has already
-     *      been started or Spring XML configuration file is invalid.
-     */
-    public static IgniteBiTuple<IgniteConfiguration, GridSpringResourceContext> loadConfiguration(URL springCfgUrl)
-        throws IgniteCheckedException {
-        IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> t = loadConfigurations(springCfgUrl);
-
-        return F.t(F.first(t.get1()), t.get2());
-    }
-
-    /**
-     * Loads first found grid configuration specified within given Spring XML configuration file.
-     * <p>
-     * Usually Spring XML configuration file will contain only one Grid definition. Note that
-     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
-     * the Grid configuration bean is ignored.
-     *
-     * @param springCfgPath Spring XML configuration file path. This cannot be {@code null}.
-     * @return First found configuration and Spring context used to load it.
-     * @throws IgniteCheckedException If grid could not be started or configuration
-     *      read. This exception will be thrown also if grid with given name has already
-     *      been started or Spring XML configuration file is invalid.
-     */
-    public static IgniteBiTuple<IgniteConfiguration, GridSpringResourceContext> loadConfiguration(String springCfgPath)
-        throws IgniteCheckedException {
-        IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> t =
-            loadConfigurations(springCfgPath);
-
-        return F.t(F.first(t.get1()), t.get2());
-    }
-
-    /**
-     * Starts all grids specified within given Spring XML configuration file. If grid with given name
-     * is already started, then exception is thrown. In this case all instances that may
-     * have been started so far will be stopped too.
-     * <p>
-     * Usually Spring XML configuration file will contain only one Grid definition. Note that
-     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
-     * the Grid configuration bean is ignored.
-     *
-     * @param springCfgPath Spring XML configuration file path or URL. This cannot be {@code null}.
-     * @param gridName Grid name that will override default.
-     * @param springCtx Optional Spring application context, possibly {@code null}.
-     *      Spring bean definitions for bean injection are taken from this context.
-     *      If provided, this context can be injected into grid tasks and grid jobs using
-     *      {@link org.apache.ignite.resources.IgniteSpringApplicationContextResource @IgniteSpringApplicationContextResource} annotation.
-     * @return Started grid. If Spring configuration contains multiple grid instances,
-     *      then the 1st found instance is returned.
-     * @throws IgniteCheckedException If grid could not be started or configuration
-     *      read. This exception will be thrown also if grid with given name has already
-     *      been started or Spring XML configuration file is invalid.
-     */
-    public static Ignite start(String springCfgPath, @Nullable String gridName,
-        @Nullable GridSpringResourceContext springCtx) throws IgniteCheckedException {
-        URL url = resolveSpringUrl(springCfgPath);
-
-        return start(url, gridName, springCtx);
-    }
-
-    /**
-     * Starts all grids specified within given Spring XML configuration file URL. If grid with given name
-     * is already started, then exception is thrown. In this case all instances that may
-     * have been started so far will be stopped too.
-     * <p>
-     * Usually Spring XML configuration file will contain only one Grid definition. Note that
-     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
-     * the Grid configuration bean is ignored.
-     *
-     * @param springCfgUrl Spring XML configuration file URL. This cannot be {@code null}.
-     * @return Started grid. If Spring configuration contains multiple grid instances,
-     *      then the 1st found instance is returned.
-     * @throws IgniteCheckedException If grid could not be started or configuration
-     *      read. This exception will be thrown also if grid with given name has already
-     *      been started or Spring XML configuration file is invalid.
-     */
-    public static Ignite start(URL springCfgUrl) throws IgniteCheckedException {
-        return start(springCfgUrl, null, null);
-    }
-
-    /**
-     * Starts all grids specified within given Spring XML configuration file URL. If grid with given name
-     * is already started, then exception is thrown. In this case all instances that may
-     * have been started so far will be stopped too.
-     * <p>
-     * Usually Spring XML configuration file will contain only one Grid definition. Note that
-     * Grid configuration bean(s) is retrieved form configuration file by type, so the name of
-     * the Grid configuration bean is ignored.
-     *
-     * @param springCfgUrl Spring XML configuration file URL. This cannot be {@code null}.
-     * @param gridName Grid name that will override default.
-     * @param springCtx Optional Spring application context, possibly {@code null}.
-     *      Spring bean definitions for bean injection are taken from this context.
-     *      If provided, this context can be injected into grid tasks and grid jobs using
-     *      {@link org.apache.ignite.resources.IgniteSpringApplicationContextResource @IgniteSpringApplicationContextResource} annotation.
-     * @return Started grid. If Spring configuration contains multiple grid instances,
-     *      then the 1st found instance is returned.
-     * @throws IgniteCheckedException If grid could not be started or configuration
-     *      read. This exception will be thrown also if grid with given name has already
-     *      been started or Spring XML configuration file is invalid.
-     */
-    public static Ignite start(URL springCfgUrl, @Nullable String gridName,
-        @Nullable GridSpringResourceContext springCtx) throws IgniteCheckedException {
-        return start(springCfgUrl, gridName, springCtx, null);
-    }
-
-    /**
-     * Internal Spring-based start routine.
-     *
-     * @param springCfgUrl Spring XML configuration file URL. This cannot be {@code null}.
-     * @param gridName Grid name that will override default.
-     * @param springCtx Optional Spring application context.
-     * @param cfgClo Optional closure to change configuration before it is used to start the grid.
-     * @return Started grid.
-     * @throws IgniteCheckedException If failed.
-     */
-    private static Ignite start(URL springCfgUrl, @Nullable String gridName,
-        @Nullable GridSpringResourceContext springCtx,
-        @Nullable IgniteClosure<IgniteConfiguration, IgniteConfiguration> cfgClo)
-        throws IgniteCheckedException {
-        A.notNull(springCfgUrl, "springCfgUrl");
-
-        boolean isLog4jUsed = U.gridClassLoader().getResource("org/apache/log4j/Appender.class") != null;
-
-        IgniteBiTuple<Object, Object> t = null;
-
-        Collection<Handler> savedHnds = null;
-
-        if (isLog4jUsed)
-            t = U.addLog4jNoOpLogger();
-        else
-            savedHnds = U.addJavaNoOpLogger();
-
-        IgniteBiTuple<Collection<IgniteConfiguration>, ? extends GridSpringResourceContext> cfgMap;
-
-        try {
-            cfgMap = loadConfigurations(springCfgUrl);
-        }
-        finally {
-            if (isLog4jUsed && t != null)
-                U.removeLog4jNoOpLogger(t);
-
-            if (!isLog4jUsed)
-                U.removeJavaNoOpLogger(savedHnds);
-        }
-
-        List<GridNamedInstance> grids = new ArrayList<>(cfgMap.size());
-
-        try {
-            for (IgniteConfiguration cfg : cfgMap.get1()) {
-                assert cfg != null;
-
-                if (cfg.getGridName() == null && !F.isEmpty(gridName))
-                    cfg.setGridName(gridName);
-
-                if (cfgClo != null) {
-                    cfg = cfgClo.apply(cfg);
-
-                    assert cfg != null;
-                }
-
-                // Use either user defined context or our one.
-                GridNamedInstance grid = start0(
-                    new GridStartContext(cfg, springCfgUrl, springCtx == null ? cfgMap.get2() : springCtx));
-
-                // Add it if it was not stopped during startup.
-                if (grid != null)
-                    grids.add(grid);
-            }
-        }
-        catch (IgniteCheckedException e) {
-            // Stop all instances started so far.
-            for (GridNamedInstance grid : grids) {
-                try {
-                    grid.stop(true);
-                }
-                catch (Exception e1) {
-                    U.error(grid.log, "Error when stopping grid: " + grid, e1);
-                }
-            }
-
-            throw e;
-        }
-
-        // Return the first grid started.
-        GridNamedInstance res = !grids.isEmpty() ? grids.get(0) : null;
-
-        return res != null ? res.grid() : null;
-    }
-
-    /**
-     * Resolve Spring configuration URL.
-     *
-     * @param springCfgPath Spring XML configuration file path or URL. This cannot be {@code null}.
-     * @return URL.
-     * @throws IgniteCheckedException If failed.
-     */
-    private static URL resolveSpringUrl(String springCfgPath) throws IgniteCheckedException {
-        A.notNull(springCfgPath, "springCfgPath");
-
-        URL url;
-
-        try {
-            url = new URL(springCfgPath);
-        }
-        catch (MalformedURLException e) {
-            url = U.resolveGridGainUrl(springCfgPath);
-
-            if (url == null)
-                throw new IgniteCheckedException("Spring XML configuration path is invalid: " + springCfgPath +
-                    ". Note that this path should be either absolute or a relative local file system path, " +
-                    "relative to META-INF in classpath or valid URL to GRIDGAIN_HOME.", e);
-        }
-
-        return url;
-    }
-
-    /**
-     * Starts grid with given configuration.
-     *
-     * @param startCtx Start context.
-     * @return Started grid.
-     * @throws IgniteCheckedException If grid could not be started.
-     */
-    private static GridNamedInstance start0(GridStartContext startCtx) throws IgniteCheckedException {
-        assert startCtx != null;
-
-        String name = startCtx.config().getGridName();
-
-        if (name != null && name.isEmpty())
-            throw new IgniteCheckedException("Non default grid instances cannot have empty string name.");
-
-        GridNamedInstance grid = new GridNamedInstance(name);
-
-        GridNamedInstance old;
-
-        if (name != null)
-            old = grids.putIfAbsent(name, grid);
-        else {
-            synchronized (dfltGridMux) {
-                old = dfltGrid;
-
-                if (old == null)
-                    dfltGrid = grid;
-            }
-        }
-
-        if (old != null) {
-            if (name == null)
-                throw new IgniteCheckedException("Default grid instance has already been started.");
-            else
-                throw new IgniteCheckedException("Grid instance with this name has already been started: " + name);
-        }
-
-        if (startCtx.config().getWarmupClosure() != null)
-            startCtx.config().getWarmupClosure().apply(startCtx.config());
-
-        startCtx.single(grids.size() == 1);
-
-        boolean success = false;
-
-        try {
-            grid.start(startCtx);
-
-            notifyStateChange(name, STARTED);
-
-            success = true;
-        }
-        finally {
-            if (!success) {
-                if (name != null)
-                    grids.remove(name, grid);
-                else {
-                    synchronized (dfltGridMux) {
-                        if (dfltGrid == grid)
-                            dfltGrid = null;
-                    }
-                }
-
-                grid = null;
-            }
-        }
-
-        if (grid == null)
-            throw new IgniteCheckedException("Failed to start grid with provided configuration.");
-
-        return grid;
-    }
-
-    /**
-     * Gets an instance of default no-name grid. Note that
-     * caller of this method should not assume that it will return the same
-     * instance every time.
-     * <p>
-     * This method is identical to {@code G.grid(null)} apply.
-     *
-     * @return An instance of default no-name grid. This method never returns
-     *      {@code null}.
-     * @throws org.apache.ignite.IgniteIllegalStateException Thrown if default grid was not properly
-     *      initialized or grid instance was stopped or was not started.
-     */
-    public static Ignite grid() throws IgniteIllegalStateException {
-        return grid((String)null);
-    }
-
-    /**
-     * Gets a list of all grids started so far.
-     *
-     * @return List of all grids started so far.
-     */
-    public static List<Ignite> allGrids() {
-        List<Ignite> allIgnites = new ArrayList<>(grids.size() + 1);
-
-        for (GridNamedInstance grid : grids.values()) {
-            Ignite g = grid.grid();
-
-            if (g != null)
-                allIgnites.add(g);
-        }
-
-        GridNamedInstance dfltGrid0 = dfltGrid;
-
-        if (dfltGrid0 != null) {
-            IgniteKernal g = dfltGrid0.grid();
-
-            if (g != null)
-                allIgnites.add(g);
-        }
-
-        return allIgnites;
-    }
-
-    /**
-     * Gets a grid instance for given local node ID. Note that grid instance and local node have
-     * one-to-one relationship where node has ID and instance has name of the grid to which
-     * both grid instance and its node belong. Note also that caller of this method
-     * should not assume that it will return the same instance every time.
-     *
-     * @param locNodeId ID of local node the requested grid instance is managing.
-     * @return An instance of named grid. This method never returns
-     *      {@code null}.
-     * @throws org.apache.ignite.IgniteIllegalStateException Thrown if grid was not properly
-     *      initialized or grid instance was stopped or was not started.
-     */
-    public static Ignite grid(UUID locNodeId) throws IgniteIllegalStateException {
-        A.notNull(locNodeId, "locNodeId");
-
-        GridNamedInstance dfltGrid0 = dfltGrid;
-
-        if (dfltGrid0 != null) {
-            IgniteKernal g = dfltGrid0.grid();
-
-            if (g != null && g.getLocalNodeId().equals(locNodeId))
-                return g;
-        }
-
-        for (GridNamedInstance grid : grids.values()) {
-            IgniteKernal g = grid.grid();
-
-            if (g != null && g.getLocalNodeId().equals(locNodeId))
-                return g;
-        }
-
-        throw new IgniteIllegalStateException("Grid instance with given local node ID was not properly " +
-            "started or was stopped: " + locNodeId);
-    }
-
-    /**
-     * Gets an named grid instance. If grid name is {@code null} or empty string,
-     * then default no-name grid will be returned. Note that caller of this method
-     * should not assume that it will return the same instance every time.
-     * <p>
-     * Note that Java VM can run multiple grid instances and every grid instance (and its
-     * node) can belong to a different grid. Grid name defines what grid a particular grid
-     * instance (and correspondingly its node) belongs to.
-     *
-     * @param name Grid name to which requested grid instance belongs to. If {@code null},
-     *      then grid instance belonging to a default no-name grid will be returned.
-     * @return An instance of named grid. This method never returns
-     *      {@code null}.
-     * @throws org.apache.ignite.IgniteIllegalStateException Thrown if default grid was not properly
-     *      initialized or grid instance was stopped or was not started.
-     */
-    public static Ignite grid(@Nullable String name) throws IgniteIllegalStateException {
-        GridNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
-
-        Ignite res;
-
-        if (grid == null || (res = grid.grid()) == null)
-            throw new IgniteIllegalStateException("Grid instance was not properly started " +
-                "or was already stopped: " + name);
-
-        return res;
-    }
-
-    /**
-     * Gets grid instance without waiting its initialization.
-     *
-     * @param name Grid name.
-     * @return Grid instance.
-     */
-    public static IgniteKernal gridx(@Nullable String name) {
-        GridNamedInstance grid = name != null ? grids.get(name) : dfltGrid;
-
-        IgniteKernal res;
-
-        if (grid == null || (res = grid.gridx()) == null)
-            throw new IllegalStateException("Grid instance was not properly started or was already stopped: " + name);
-
-        return res;
-    }
-
-    /**
-     * Adds a lsnr for grid life cycle events.
-     * <p>
-     * Note that unlike other listeners in GridGain this listener will be
-     * notified from the same thread that triggers the state change. Because of
-     * that it is the responsibility of the user to make sure that listener logic
-     * is light-weight and properly handles (catches) any runtime exceptions, if any
-     * are expected.
-     *
-     * @param lsnr Listener for grid life cycle events. If this listener was already added
-     *      this method is no-op.
-     */
-    public static void addListener(IgniteListener lsnr) {
-        A.notNull(lsnr, "lsnr");
-
-        lsnrs.add(lsnr);
-    }
-
-    /**
-     * Removes lsnr added by {@link #addListener(org.apache.ignite.lifecycle.IgniteListener)} method.
-     *
-     * @param lsnr Listener to remove.
-     * @return {@code true} if lsnr was added before, {@code false} otherwise.
-     */
-    public static boolean removeListener(IgniteListener lsnr) {
-        A.notNull(lsnr, "lsnr");
-
-        return lsnrs.remove(lsnr);
-    }
-
-    /**
-     * @param gridName Grid instance name.
-     * @param state Factory state.
-     */
-    private static void notifyStateChange(@Nullable String gridName, IgniteState state) {
-        if (gridName != null)
-            gridStates.put(gridName, state);
-        else
-            dfltGridState = state;
-
-        for (IgniteListener lsnr : lsnrs)
-            lsnr.onStateChange(gridName, state);
-    }
-
-    /**
-     * Start context encapsulates all starting parameters.
-     */
-    private static final class GridStartContext {
-        /** User-defined configuration. */
-        private IgniteConfiguration cfg;
-
-        /** Optional configuration path. */
-        private URL cfgUrl;
-
-        /** Optional Spring application context. */
-        private GridSpringResourceContext springCtx;
-
-        /** Whether or not this is a single grid instance in current VM. */
-        private boolean single;
-
-        /**
-         *
-         * @param cfg User-defined configuration.
-         * @param cfgUrl Optional configuration path.
-         * @param springCtx Optional Spring application context.
-         */
-        GridStartContext(IgniteConfiguration cfg, @Nullable URL cfgUrl, @Nullable GridSpringResourceContext springCtx) {
-            assert(cfg != null);
-
-            this.cfg = cfg;
-            this.cfgUrl = cfgUrl;
-            this.springCtx = springCtx;
-        }
-
-        /**
-         * @return Whether or not this is a single grid instance in current VM.
-         */
-        public boolean single() {
-            return single;
-        }
-
-        /**
-         * @param single Whether or not this is a single grid instance in current VM.
-         */
-        public void single(boolean single) {
-            this.single = single;
-        }
-
-        /**
-         * @return User-defined configuration.
-         */
-        IgniteConfiguration config() {
-            return cfg;
-        }
-
-        /**
-         * @param cfg User-defined configuration.
-         */
-        void config(IgniteConfiguration cfg) {
-            this.cfg = cfg;
-        }
-
-        /**
-         * @return Optional configuration path.
-         */
-        URL configUrl() {
-            return cfgUrl;
-        }
-
-        /**
-         * @param cfgUrl Optional configuration path.
-         */
-        void configUrl(URL cfgUrl) {
-            this.cfgUrl = cfgUrl;
-        }
-
-        /**
-         * @return Optional Spring application context.
-         */
-        public GridSpringResourceContext springContext() {
-            return springCtx;
-        }
-    }
-
-    /**
-     * Grid data container.
-     */
-    private static final class GridNamedInstance {
-        /** Map of registered MBeans. */
-        private static final Map<MBeanServer, GridMBeanServerData> mbeans =
-            new HashMap<>();
-
-        /** */
-        private static final String[] EMPTY_STR_ARR = new String[0];
-
-        /** Empty array of caches. */
-        private static final CacheConfiguration[] EMPTY_CACHE_CONFIGS = new CacheConfiguration[0];
-
-        /** Grid name. */
-        private final String name;
-
-        /** Grid instance. */
-        private volatile IgniteKernal grid;
-
-        /** Executor service. */
-        private ExecutorService execSvc;
-
-        /** Auto executor service flag. */
-        private boolean isAutoExecSvc;
-
-        /** Executor service shutdown flag. */
-        private boolean execSvcShutdown;
-
-        /** System executor service. */
-        private ExecutorService sysExecSvc;
-
-        /** Auto system service flag. */
-        private boolean isAutoSysSvc;
-
-        /** System executor service shutdown flag. */
-        private boolean sysSvcShutdown;
-
-        /** Management executor service. */
-        private ExecutorService mgmtExecSvc;
-
-        /** Auto management service flag. */
-        private boolean isAutoMgmtSvc;
-
-        /** Management executor service shutdown flag. */
-        private boolean mgmtSvcShutdown;
-
-        /** P2P executor service. */
-        private ExecutorService p2pExecSvc;
-
-        /** Auto P2P service flag. */
-        private boolean isAutoP2PSvc;
-
-        /** P2P executor service shutdown flag. */
-        private boolean p2pSvcShutdown;
-
-        /** GGFS executor service. */
-        private ExecutorService ggfsExecSvc;
-
-        /** Auto GGFS service flag. */
-        private boolean isAutoGgfsSvc;
-
-        /** GGFS executor service shutdown flag. */
-        private boolean ggfsSvcShutdown;
-
-        /** REST requests executor service. */
-        private ExecutorService restExecSvc;
-
-        /** Auto REST service flag. */
-        private boolean isAutoRestSvc;
-
-        /** REST executor service shutdown flag. */
-        private boolean restSvcShutdown;
-
-        /** Utility cache executor service. */
-        private ExecutorService utilityCacheExecSvc;
-
-        /** Grid state. */
-        private volatile IgniteState state = STOPPED;
-
-        /** Shutdown hook. */
-        private Thread shutdownHook;
-
-        /** Grid log. */
-        private IgniteLogger log;
-
-        /** Start guard. */
-        private final AtomicBoolean startGuard = new AtomicBoolean();
-
-        /** Start latch. */
-        private final CountDownLatch startLatch = new CountDownLatch(1);
-
-        /**
-         * Thread that starts this named instance. This field can be non-volatile since
-         * it makes sense only for thread where it was originally initialized.
-         */
-        @SuppressWarnings("FieldAccessedSynchronizedAndUnsynchronized")
-        private Thread starterThread;
-
-        /**
-         * Creates un-started named instance.
-         *
-         * @param name Grid name (possibly {@code null} for default grid).
-         */
-        GridNamedInstance(@Nullable String name) {
-            this.name = name;
-        }
-
-        /**
-         * Gets grid name.
-         *
-         * @return Grid name.
-         */
-        String getName() {
-            return name;
-        }
-
-        /**
-         * Gets grid instance.
-         *
-         * @return Grid instance.
-         */
-        IgniteKernal grid() {
-            if (starterThread != Thread.currentThread())
-                U.awaitQuiet(startLatch);
-
-            return grid;
-        }
-
-        /**
-         * Gets grid instance without waiting for its initialization.
-         *
-         * @return Grid instance.
-         */
-        public IgniteKernal gridx() {
-            return grid;
-        }
-
-        /**
-         * Gets grid state.
-         *
-         * @return Grid state.
-         */
-        IgniteState state() {
-            if (starterThread != Thread.currentThread())
-                U.awaitQuiet(startLatch);
-
-            return state;
-        }
-
-        /**
-         * @param spi SPI implementation.
-         * @throws IgniteCheckedException Thrown in case if multi-instance is not supported.
-         */
-        private void ensureMultiInstanceSupport(IgniteSpi spi) throws IgniteCheckedException {
-            IgniteSpiMultipleInstancesSupport ann = U.getAnnotation(spi.getClass(),
-                IgniteSpiMultipleInstancesSupport.class);
-
-            if (ann == null || !ann.value())
-                throw new IgniteCheckedException("SPI implementation doesn't support multiple grid instances in " +
-                    "the same VM: " + spi);
-        }
-
-        /**
-         * @param spis SPI implementations.
-         * @throws IgniteCheckedException Thrown in case if multi-instance is not supported.
-         */
-        private void ensureMultiInstanceSupport(IgniteSpi[] spis) throws IgniteCheckedException {
-            for (IgniteSpi spi : spis)
-                ensureMultiInstanceSupport(spi);
-        }
-
-        /**
-         * Starts grid with given configuration.
-         *
-         * @param startCtx Starting context.
-         * @throws IgniteCheckedException If start failed.
-         */
-        synchronized void start(GridStartContext startCtx) throws IgniteCheckedException {
-            if (startGuard.compareAndSet(false, true)) {
-                try {
-                    starterThread = Thread.currentThread();
-
-                    start0(startCtx);
-                }
-                catch (Exception e) {
-                    if (log != null)
-                        stopExecutors(log);
-
-                    throw e;
-                }
-                finally {
-                    startLatch.countDown();
-                }
-            }
-            else
-                U.awaitQuiet(startLatch);
-        }
-
-        /**
-         * @param startCtx Starting context.
-         * @throws IgniteCheckedException If start failed.
-         */
-        @SuppressWarnings({"unchecked", "TooBroadScope"})
-        private void start0(GridStartContext startCtx) throws IgniteCheckedException {
-            assert grid == null : "Grid is already started: " + name;
-
-            IgniteConfiguration cfg = startCtx.config();
-
-            if (cfg == null)
-                cfg = new IgniteConfiguration();
-
-            IgniteConfiguration myCfg = new IgniteConfiguration();
-
-            String ggHome = cfg.getGridGainHome();
-
-            // Set GridGain home.
-            if (ggHome == null)
-                ggHome = U.getGridGainHome();
-            else
-                // If user provided GRIDGAIN_HOME - set it as a system property.
-                U.setGridGainHome(ggHome);
-
-            U.setWorkDirectory(cfg.getWorkDirectory(), ggHome);
-
-            /*
-             * Set up all defaults and perform all checks.
-             */
-
-            // Ensure invariant.
-            // It's a bit dirty - but this is a result of late refactoring
-            // and I don't want to reshuffle a lot of code.
-            assert F.eq(name, cfg.getGridName());
-
-            // Set configuration URL, if any, into system property.
-            if (startCtx.configUrl() != null)
-                System.setProperty(GG_CONFIG_URL, startCtx.configUrl().toString());
-
-            myCfg.setGridName(cfg.getGridName());
-
-            UUID nodeId = cfg.getNodeId();
-
-            if (nodeId == null)
-                nodeId = UUID.randomUUID();
-
-            IgniteLogger cfgLog = initLogger(cfg.getGridLogger(), nodeId);
-
-            assert cfgLog != null;
-
-            cfgLog = new GridLoggerProxy(cfgLog, null, name, U.id8(nodeId));
-
-            // Initialize factory's log.
-            log = cfgLog.getLogger(G.class);
-
-            // Check GridGain home folder (after log is available).
-            if (ggHome != null) {
-                File ggHomeFile = new File(ggHome);
-
-                if (!ggHomeFile.exists() || !ggHomeFile.isDirectory())
-                    throw new IgniteCheckedException("Invalid GridGain installation home folder: " + ggHome);
-            }
-
-            myCfg.setGridGainHome(ggHome);
-
-            // Copy values that don't need extra processing.
-            myCfg.setLicenseUrl(cfg.getLicenseUrl());
-            myCfg.setPeerClassLoadingEnabled(cfg.isPeerClassLoadingEnabled());
-            myCfg.setDeploymentMode(cfg.getDeploymentMode());
-            myCfg.setNetworkTimeout(cfg.getNetworkTimeout());
-            myCfg.setClockSyncSamples(cfg.getClockSyncSamples());
-            myCfg.setClockSyncFrequency(cfg.getClockSyncFrequency());
-            myCfg.setDiscoveryStartupDelay(cfg.getDiscoveryStartupDelay());
-            myCfg.setMetricsHistorySize(cfg.getMetricsHistorySize());
-            myCfg.setMetricsExpireTime(cfg.getMetricsExpireTime());
-            myCfg.setMetricsUpdateFrequency(cfg.getMetricsUpdateFrequency());
-            myCfg.setLifecycleBeans(cfg.getLifecycleBeans());
-            myCfg.setLocalEventListeners(cfg.getLocalEventListeners());
-            myCfg.setPeerClassLoadingMissedResourcesCacheSize(cfg.getPeerClassLoadingMissedResourcesCacheSize());
-            myCfg.setIncludeEventTypes(cfg.getIncludeEventTypes());
-            myCfg.setDaemon(cfg.isDaemon());
-            myCfg.setIncludeProperties(cfg.getIncludeProperties());
-            myCfg.setLifeCycleEmailNotification(cfg.isLifeCycleEmailNotification());
-            myCfg.setMetricsLogFrequency(cfg.getMetricsLogFrequency());
-            myCfg.setNetworkSendRetryDelay(cfg.getNetworkSendRetryDelay());
-            myCfg.setNetworkSendRetryCount(cfg.getNetworkSendRetryCount());
-            myCfg.setSecurityCredentialsProvider(cfg.getSecurityCredentialsProvider());
-            myCfg.setServiceConfiguration(cfg.getServiceConfiguration());
-            myCfg.setWarmupClosure(cfg.getWarmupClosure());
-            myCfg.setInteropConfiguration(cfg.getInteropConfiguration());
-            myCfg.setPluginConfigurations(cfg.getPluginConfigurations());
-            myCfg.setTransactionsConfiguration(new TransactionsConfiguration(cfg.getTransactionsConfiguration()));
-            myCfg.setQueryConfiguration(cfg.getQueryConfiguration());
-
-            ClientConnectionConfiguration clientCfg = cfg.getClientConnectionConfiguration();
-
-            if (clientCfg == null) {
-                // If client config is not provided then create config copying values from GridConfiguration.
-                if (cfg.isRestEnabled()) {
-                    clientCfg = new ClientConnectionConfiguration();
-
-                    clientCfg.setClientMessageInterceptor(cfg.getClientMessageInterceptor());
-                    clientCfg.setRestAccessibleFolders(cfg.getRestAccessibleFolders());
-                    clientCfg.setRestExecutorService(cfg.getRestExecutorService());
-                    clientCfg.setRestExecutorServiceShutdown(cfg.getRestExecutorServiceShutdown());
-                    clientCfg.setRestIdleTimeout(cfg.getRestIdleTimeout());
-                    clientCfg.setRestJettyPath(cfg.getRestJettyPath());
-                    clientCfg.setRestPortRange(cfg.getRestPortRange());
-                    clientCfg.setRestSecretKey(cfg.getRestSecretKey());
-                    clientCfg.setRestTcpDirectBuffer(cfg.isRestTcpDirectBuffer());
-                    clientCfg.setRestTcpHost(cfg.getRestTcpHost());
-                    clientCfg.setRestTcpNoDelay(cfg.isRestTcpNoDelay());
-                    clientCfg.setRestTcpPort(cfg.getRestTcpPort());
-                    clientCfg.setRestTcpReceiveBufferSize(cfg.getRestTcpReceiveBufferSize());
-                    clientCfg.setRestTcpSelectorCount(cfg.getRestTcpSelectorCount());
-                    clientCfg.setRestTcpSendBufferSize(cfg.getRestTcpSendBufferSize());
-                    clientCfg.setRestTcpSendQueueLimit(cfg.getRestTcpSendQueueLimit());
-                    clientCfg.setRestTcpSslClientAuth(cfg.isRestTcpSslClientAuth());
-                    clientCfg.setRestTcpSslContextFactory(cfg.getRestTcpSslContextFactory());
-                    clientCfg.setRestTcpSslEnabled(cfg.isRestTcpSslEnabled());
-                }
-            }
-            else
-                clientCfg = new ClientConnectionConfiguration(clientCfg);
-
-
-            String ntfStr = IgniteSystemProperties.getString(GG_LIFECYCLE_EMAIL_NOTIFY);
-
-            if (ntfStr != null)
-                myCfg.setLifeCycleEmailNotification(Boolean.parseBoolean(ntfStr));
-
-            // Local host.
-            String locHost = IgniteSystemProperties.getString(GG_LOCAL_HOST);
-
-            myCfg.setLocalHost(F.isEmpty(locHost) ? cfg.getLocalHost() : locHost);
-
-            // Override daemon flag if it was set on the factory.
-            if (daemon)
-                myCfg.setDaemon(true);
-
-            // Check for deployment mode override.
-            String depModeName = IgniteSystemProperties.getString(GG_DEP_MODE_OVERRIDE);
-
-            if (!F.isEmpty(depModeName)) {
-                if (!F.isEmpty(cfg.getCacheConfiguration())) {
-                    U.quietAndInfo(log, "Skipping deployment mode override for caches (custom closure " +
-                        "execution may not work for console Visor)");
-                }
-                else {
-                    try {
-                        IgniteDeploymentMode depMode = IgniteDeploymentMode.valueOf(depModeName);
-
-                        if (myCfg.getDeploymentMode() != depMode)
-                            myCfg.setDeploymentMode(depMode);
-                    }
-                    catch (IllegalArgumentException e) {
-                        throw new IgniteCheckedException("Failed to override deployment mode using system property " +
-                            "(are there any misspellings?)" +
-                            "[name=" + GG_DEP_MODE_OVERRIDE + ", value=" + depModeName + ']', e);
-                    }
-                }
-            }
-
-            Map<String, ?> attrs = cfg.getUserAttributes();
-
-            if (attrs == null)
-                attrs = Collections.emptyMap();
-
-            MBeanServer mbSrv = cfg.getMBeanServer();
-
-            IgniteMarshaller marsh = cfg.getMarshaller();
-
-            String[] p2pExclude = cfg.getPeerClassLoadingLocalClassPathExclude();
-
-            CommunicationSpi commSpi = cfg.getCommunicationSpi();
-            DiscoverySpi discoSpi = cfg.getDiscoverySpi();
-            EventStorageSpi evtSpi = cfg.getEventStorageSpi();
-            CollisionSpi colSpi = cfg.getCollisionSpi();
-            AuthenticationSpi authSpi = cfg.getAuthenticationSpi();
-            SecureSessionSpi sesSpi = cfg.getSecureSessionSpi();
-            DeploymentSpi deploySpi = cfg.getDeploymentSpi();
-            CheckpointSpi[] cpSpi = cfg.getCheckpointSpi();
-            FailoverSpi[] failSpi = cfg.getFailoverSpi();
-            LoadBalancingSpi[] loadBalancingSpi = cfg.getLoadBalancingSpi();
-            SwapSpaceSpi swapspaceSpi = cfg.getSwapSpaceSpi();
-            GridIndexingSpi indexingSpi = cfg.getIndexingSpi();
-
-            execSvc = cfg.getExecutorService();
-            sysExecSvc = cfg.getSystemExecutorService();
-            p2pExecSvc = cfg.getPeerClassLoadingExecutorService();
-            mgmtExecSvc = cfg.getManagementExecutorService();
-            ggfsExecSvc = cfg.getGgfsExecutorService();
-
-            if (execSvc == null) {
-                isAutoExecSvc = true;
-
-                execSvc = new IgniteThreadPoolExecutor(
-                    "pub-" + cfg.getGridName(),
-                    DFLT_PUBLIC_CORE_THREAD_CNT,
-                    DFLT_PUBLIC_MAX_THREAD_CNT,
-                    DFLT_PUBLIC_KEEP_ALIVE_TIME,
-                    new LinkedBlockingQueue<Runnable>(DFLT_PUBLIC_THREADPOOL_QUEUE_CAP));
-
-                // Pre-start all threads as they are guaranteed to be needed.
-                ((ThreadPoolExecutor)execSvc).prestartAllCoreThreads();
-            }
-
-            if (sysExecSvc == null) {
-                isAutoSysSvc = true;
-
-                // Note that since we use 'LinkedBlockingQueue', number of
-                // maximum threads has no effect.
-                sysExecSvc = new IgniteThreadPoolExecutor(
-                    "sys-" + cfg.getGridName(),
-                    DFLT_SYSTEM_CORE_THREAD_CNT,
-                    DFLT_SYSTEM_MAX_THREAD_CNT,
-                    DFLT_SYSTEM_KEEP_ALIVE_TIME,
-                    new LinkedBlockingQueue<Runnable>(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP));
-
-                // Pre-start all threads as they are guaranteed to be needed.
-                ((ThreadPoolExecutor)sysExecSvc).prestartAllCoreThreads();
-            }
-
-            if (mgmtExecSvc == null) {
-                isAutoMgmtSvc = true;
-
-                // Note that since we use 'LinkedBlockingQueue', number of
-                // maximum threads has no effect.
-                // Note, that we do not pre-start threads here as management pool may
-                // not be needed.
-                mgmtExecSvc = new IgniteThreadPoolExecutor(
-                    "mgmt-" + cfg.getGridName(),
-                    DFLT_MGMT_THREAD_CNT,
-                    DFLT_MGMT_THREAD_CNT,
-                    0,
-                    new LinkedBlockingQueue<Runnable>());
-            }
-
-            if (p2pExecSvc == null) {
-                isAutoP2PSvc = true;
-
-                // Note that since we use 'LinkedBlockingQueue', number of
-                // maximum threads has no effect.
-                // Note, that we do not pre-start threads here as class loading pool may
-                // not be needed.
-                p2pExecSvc = new IgniteThreadPoolExecutor(
-                    "p2p-" + cfg.getGridName(),
-                    DFLT_P2P_THREAD_CNT,
-                    DFLT_P2P_THREAD_CNT,
-                    0,
-                    new LinkedBlockingQueue<Runnable>());
-            }
-
-            if (ggfsExecSvc == null) {
-                isAutoGgfsSvc = true;
-
-                int procCnt = Runtime.getRuntime().availableProcessors();
-
-                // Note that we do not pre-start threads here as ggfs pool may not be needed.
-                ggfsExecSvc = new IgniteThreadPoolExecutor(
-                    "ggfs-" + cfg.getGridName(),
-                    procCnt,
-                    procCnt,
-                    0,
-                    new LinkedBlockingQueue<Runnable>());
-            }
-
-            restExecSvc = clientCfg != null ? clientCfg.getRestExecutorService() : null;
-
-            if (restExecSvc != null && !cfg.isRestEnabled()) {
-                U.warn(log, "REST executor service is configured, but REST is disabled in configuration " +
-                    "(safely ignoring).");
-            }
-            else if (restExecSvc == null && clientCfg != null) {
-                isAutoRestSvc = true;
-
-                restExecSvc = new IgniteThreadPoolExecutor(
-                    "rest-" + cfg.getGridName(),
-                    DFLT_REST_CORE_THREAD_CNT,
-                    DFLT_REST_MAX_THREAD_CNT,
-                    DFLT_REST_KEEP_ALIVE_TIME,
-                    new LinkedBlockingQueue<Runnable>(DFLT_REST_THREADPOOL_QUEUE_CAP)
-                );
-
-                clientCfg.setRestExecutorService(restExecSvc);
-            }
-
-            utilityCacheExecSvc = new IgniteThreadPoolExecutor(
-                "utility-" + cfg.getGridName(),
-                DFLT_SYSTEM_CORE_THREAD_CNT,
-                DFLT_SYSTEM_MAX_THREAD_CNT,
-                DFLT_SYSTEM_KEEP_ALIVE_TIME,
-                new LinkedBlockingQueue<Runnable>(DFLT_SYSTEM_THREADPOOL_QUEUE_CAP));
-
-            execSvcShutdown = cfg.getExecutorServiceShutdown();
-            sysSvcShutdown = cfg.getSystemExecutorServiceShutdown();
-            mgmtSvcShutdown = cfg.getManagementExecutorServiceShutdown();
-            p2pSvcShutdown = cfg.getPeerClassLoadingExecutorServiceShutdown();
-            ggfsSvcShutdown = cfg.getGgfsExecutorServiceShutdown();
-            restSvcShutdown = clientCfg != null && clientCfg.isRestExecutorServiceShutdown();
-
-            if (marsh == null) {
-                if (!U.isHotSpot()) {
-                    U.warn(log, "GridOptimizedMarshaller is not supported on this JVM " +
-                        "(only Java HotSpot VMs are supported). Switching to standard JDK marshalling - " +
-                        "object serialization performance will be significantly slower.",
-                        "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release.");
-
-                    marsh = new IgniteJdkMarshaller();
-                }
-                else if (!IgniteOptimizedMarshaller.available()) {
-                    U.warn(log, "GridOptimizedMarshaller is not supported on this JVM " +
-                        "(only recent 1.6 and 1.7 versions HotSpot VMs are supported). " +
-                        "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release. " +
-                        "Switching to standard JDK marshalling - " +
-                        "object serialization performance will be significantly slower.",
-                        "To enable fast marshalling upgrade to recent 1.6 or 1.7 HotSpot VM release.");
-
-                    marsh = new IgniteJdkMarshaller();
-                }
-                else
-                    marsh = new IgniteOptimizedMarshaller();
-            }
-            else if (marsh instanceof IgniteOptimizedMarshaller && !U.isHotSpot()) {
-                U.warn(log, "Using GridOptimizedMarshaller on untested JVM (only Java HotSpot VMs were tested) - " +
-                    "object serialization behavior could yield unexpected results.",
-                    "Using GridOptimizedMarshaller on untested JVM.");
-            }
-
-            myCfg.setUserAttributes(attrs);
-            myCfg.setMBeanServer(mbSrv == null ? ManagementFactory.getPlatformMBeanServer() : mbSrv);
-            myCfg.setGridLogger(cfgLog);
-            myCfg.setMarshaller(marsh);
-            myCfg.setMarshalLocalJobs(cfg.isMarshalLocalJobs());
-            myCfg.setExecutorService(execSvc);
-            myCfg.setSystemExecutorService(sysExecSvc);
-            myCfg.setManagementExecutorService(mgmtExecSvc);
-            myCfg.setPeerClassLoadingExecutorService(p2pExecSvc);
-            myCfg.setGgfsExecutorService(ggfsExecSvc);
-            myCfg.setExecutorServiceShutdown(execSvcShutdown);
-            myCfg.setSystemExecutorServiceShutdown(sysSvcShutdown);
-            myCfg.setManagementExecutorServiceShutdown(mgmtSvcShutdown);
-            myCfg.setPeerClassLoadingExecutorServiceShutdown(p2pSvcShutdown);
-            myCfg.setGgfsExecutorServiceShutdown(ggfsSvcShutdown);
-            myCfg.setNodeId(nodeId);
-
-            IgniteFsConfiguration[] ggfsCfgs = cfg.getGgfsConfiguration();
-
-            if (ggfsCfgs != null) {
-                IgniteFsConfiguration[] clone = ggfsCfgs.clone();
-
-                for (int i = 0; i < ggfsCfgs.length; i++)
-                    clone[i] = new IgniteFsConfiguration(ggfsCfgs[i]);
-
-                myCfg.setGgfsConfiguration(clone);
-            }
-
-            StreamerConfiguration[] streamerCfgs = cfg.getStreamerConfiguration();
-
-            if (streamerCfgs != null) {
-                StreamerConfiguration[] clone = streamerCfgs.clone();
-
-                for (int i = 0; i < streamerCfgs.length; i++)
-                    clone[i] = new StreamerConfiguration(streamerCfgs[i]);
-
-                myCfg.setStreamerConfiguration(clone);
-            }
-
-            if (p2pExclude == null)
-                p2pExclude = EMPTY_STR_ARR;
-
-            myCfg.setPeerClassLoadingLocalClassPathExclude(p2pExclude);
-
-            /*
-             * Initialize default SPI implementations.
-             */
-
-            if (commSpi == null)
-                commSpi = new TcpCommunicationSpi();
-
-            if (discoSpi == null)
-                discoSpi = new TcpDiscoverySpi();
-
-            if (discoSpi instanceof TcpDiscoverySpi) {
-                TcpDiscoverySpi tcpDisco = (TcpDiscoverySpi)discoSpi;
-
-                if (tcpDisco.getIpFinder() == null)
-                    tcpDisco.setIpFinder(new TcpDiscoveryMulticastIpFinder());
-            }
-
-            if (evtSpi == null)
-                evtSpi = new MemoryEventStorageSpi();
-
-            if (colSpi == null)
-                colSpi = new NoopCollisionSpi();
-
-            if (authSpi == null)
-                authSpi = new NoopAuthenticationSpi();
-
-            if (sesSpi == null)
-                sesSpi = new NoopSecureSessionSpi();
-
-            if (deploySpi == null)
-                deploySpi = new LocalDeploymentSpi();
-
-            if (cpSpi == null)
-                cpSpi = new CheckpointSpi[] {new NoopCheckpointSpi()};
-
-            if (failSpi == null)
-                failSpi = new FailoverSpi[] {new AlwaysFailoverSpi()};
-
-            if (loadBalancingSpi == null)
-                loadBalancingSpi = new LoadBalancingSpi[] {new RoundRobinLoadBalancingSpi()};
-
-            if (swapspaceSpi == null) {
-                boolean needSwap = false;
-
-                CacheConfiguration[] caches = cfg.getCacheConfiguration();
-
-                if (caches != null) {
-                    for (CacheConfiguration c : caches) {
-                        if (c.isSwapEnabled()) {
-                            needSwap = true;
-
-                            break;
-                        }
-                    }
-                }
-
-                swapspaceSpi = needSwap ? new FileSwapSpaceSpi() : new NoopSwapSpaceSpi();
-            }
-
-            if (indexingSpi == null)
-                indexingSpi = new GridNoopIndexingSpi();
-
-            myCfg.setCommunicationSpi(commSpi);
-            myCfg.setDiscoverySpi(discoSpi);
-            myCfg.setCheckpointSpi(cpSpi);
-            myCfg.setEventStorageSpi(evtSpi);
-            myCfg.setAuthenticationSpi(authSpi);
-            myCfg.setSecureSessionSpi(sesSpi);
-            myCfg.setDeploymentSpi(deploySpi);
-            myCfg.setFailoverSpi(failSpi);
-            myCfg.setCollisionSpi(colSpi);
-            myCfg.setLoadBalancingSpi(loadBalancingSpi);
-            myCfg.setSwapSpaceSpi(swapspaceSpi);
-            myCfg.setIndexingSpi(indexingSpi);
-
-            myCfg.setAddressResolver(cfg.getAddressResolver());
-
-            // Set SMTP configuration.
-            myCfg.setSmtpFromEmail(cfg.getSmtpFromEmail());
-            myCfg.setSmtpHost(cfg.getSmtpHost());
-            myCfg.setSmtpPort(cfg.getSmtpPort());
-            myCfg.setSmtpSsl(cfg.isSmtpSsl());
-            myCfg.setSmtpUsername(cfg.getSmtpUsername());
-            myCfg.setSmtpPassword(cfg.getSmtpPassword());
-            myCfg.setAdminEmails(cfg.getAdminEmails());
-
-            // REST configuration.
-            myCfg.setClientConnectionConfiguration(clientCfg);
-
-            // Portable configuration.
-            myCfg.setPortableConfiguration(cfg.getPortableConfiguration());
-
-            // Hadoop configuration.
-            myCfg.setHadoopConfiguration(cfg.getHadoopConfiguration());
-
-            // Validate segmentation configuration.
-            GridSegmentationPolicy segPlc = cfg.getSegmentationPolicy();
-
-            // 1. Warn on potential configuration problem: grid is not configured to wait
-            // for correct segment after segmentation happens.
-            if (!F.isEmpty(cfg.getSegmentationResolvers()) && segPlc == RESTART_JVM && !cfg.isWaitForSegmentOnStart()) {
-                U.warn(log, "Found potential configuration problem (forgot to enable waiting for segment" +
-                    "on start?) [segPlc=" + segPlc + ", wait=false]");
-            }
-
-            myCfg.setSegmentationResolvers(cfg.getSegmentationResolvers());
-            myCfg.setSegmentationPolicy(segPlc);
-            myCfg.setSegmentCheckFrequency(cfg.getSegmentCheckFrequency());
-            myCfg.setWaitForSegmentOnStart(cfg.isWaitForSegmentOnStart());
-            myCfg.setAllSegmentationResolversPassRequired(cfg.isAllSegmentationResolversPassRequired());
-
-            // Override SMTP configuration from system properties
-            // and environment variables, if specified.
-            String fromEmail = IgniteSystemProperties.getString(GG_SMTP_FROM);
-
-            if (fromEmail != null)
-                myCfg.setSmtpFromEmail(fromEmail);
-
-            String smtpHost = IgniteSystemProperties.getString(GG_SMTP_HOST);
-
-            if (smtpHost != null)
-                myCfg.setSmtpHost(smtpHost);
-
-            String smtpUsername = IgniteSystemProperties.getString(GG_SMTP_USERNAME);
-
-            if (smtpUsername != null)
-                myCfg.setSmtpUsername(smtpUsername);
-
-            String smtpPwd = IgniteSystemProperties.getString(GG_SMTP_PWD);
-
-            if (smtpPwd != null)
-                myCfg.setSmtpPassword(smtpPwd);
-
-            int smtpPort = IgniteSystemProperties.getInteger(GG_SMTP_PORT, -1);
-
-            if(smtpPort != -1)
-                myCfg.setSmtpPort(smtpPort);
-
-            myCfg.setSmtpSsl(IgniteSystemProperties.getBoolean(GG_SMTP_SSL));
-
-            String adminEmails = IgniteSystemProperties.getString(GG_ADMIN_EMAILS);
-
-            if (adminEmails != null)
-                myCfg.setAdminEmails(adminEmails.split(","));
-
-            CacheConfiguration[] cacheCfgs = cfg.getCacheConfiguration();
-
-            boolean hasHadoop = IgniteComponentType.HADOOP.inClassPath();
-
-            CacheConfiguration[] copies;
-
-            if (cacheCfgs != null && cacheCfgs.length > 0) {
-                if (!U.discoOrdered(discoSpi) && !U.relaxDiscoveryOrdered())
-                    throw new IgniteCheckedException("Discovery SPI implementation does not support node ordering and " +
-                        "cannot be used with cache (use SPI with @GridDiscoverySpiOrderSupport annotation, " +
-                        "like GridTcpDiscoverySpi)");
-
-                for (CacheConfiguration ccfg : cacheCfgs) {
-                    if (CU.isHadoopSystemCache(ccfg.getName()))
-                        throw new IgniteCheckedException("Cache name cannot be \"" + CU.SYS_CACHE_HADOOP_MR +
-                            "\" because it is reserved for internal purposes.");
-
-                    if (CU.isUtilityCache(ccfg.getName()))
-                        throw new IgniteCheckedException("Cache name cannot start with \"" + CU.UTILITY_CACHE_NAME +
-                            "\" because this prefix is reserved for internal purposes.");
-                }
-
-                copies = new CacheConfiguration[cacheCfgs.length + (hasHadoop ? 2 : 1)];
-
-                int cloneIdx = 1;
-
-                if (hasHadoop)
-                    copies[cloneIdx++] = CU.hadoopSystemCache();
-
-                for (CacheConfiguration ccfg : cacheCfgs)
-                    copies[cloneIdx++] = new CacheConfiguration(ccfg);
-            }
-            else if (hasHadoop) {
-                // Populate system caches
-                copies = new CacheConfiguration[hasHadoop ? 2 : 1];
-
-                copies[1] = CU.hadoopSystemCache();
-            }
-            else
-                copies = new CacheConfiguration[1];
-
-            // Always add utility cache.
-            copies[0] = utilitySystemCache(discoSpi instanceof TcpClientDiscoverySpi);
-
-            myCfg.setCacheConfiguration(copies);
-
-            myCfg.setCacheSanityCheckEnabled(cfg.isCacheSanityCheckEnabled());
-
-            try {
-                // Use reflection to avoid loading undesired classes.
-                Class helperCls = Class.forName("org.gridgain.grid.util.GridConfigurationHelper");
-
-                helperCls.getMethod("overrideConfiguration", IgniteConfiguration.class, Properties.class,
-                    String.class, IgniteLogger.class).invoke(helperCls, myCfg, System.getProperties(), name, log);
-            }
-            catch (Exception ignored) {
-                // No-op.
-            }
-
-            // Ensure that SPIs support multiple grid instances, if required.
-            if (!startCtx.single()) {
-                ensureMultiInstanceSupport(deploySpi);
-                ensureMultiInstanceSupport(commSpi);
-                ensureMultiInstanceSupport(discoSpi);
-                ensureMultiInstanceSupport(cpSpi);
-                ensureMultiInstanceSupport(evtSpi);
-                ensureMultiInstanceSupport(colSpi);
-                ensureMultiInstanceSupport(failSpi);
-                ensureMultiInstanceSupport(authSpi);
-                ensureMultiInstanceSupport(sesSpi);
-                ensureMultiInstanceSupport(loadBalancingSpi);
-                ensureMultiInstanceSupport(swapspaceSpi);
-            }
-
-            // Register GridGain MBean for current grid instance.
-            registerFactoryMbean(myCfg.getMBeanServer());
-
-            boolean started = false;
-
-            try {
-                IgniteKernal grid0 = new IgniteKernal(startCtx.springContext());
-
-                // Init here to make grid available to lifecycle listeners.
-                grid = grid0;
-
-                grid0.start(myCfg, utilityCacheExecSvc, new CA() {
-                    @Override public void apply() {
-                        startLatch.countDown();
-                    }
-                });
-
-                state = STARTED;
-
-                if (log.isDebugEnabled())
-                    log.debug("Grid factory started ok: " + name);
-
-                started = true;
-            }
-            catch (IgniteCheckedException e) {
-                unregisterFactoryMBean();
-
-                throw e;
-            }
-            // Catch Throwable to protect against any possible failure.
-            catch (Throwable e) {
-                unregisterFactoryMBean();
-
-                throw new IgniteCheckedException("Unexpected exception when starting grid.", e);
-            }
-            finally {
-                if (!started)
-                    // Grid was not started.
-                    grid = null;
-            }
-
-            // Do NOT set it up only if GRIDGAIN_NO_SHUTDOWN_HOOK=TRUE is provided.
-            if (!IgniteSystemProperties.getBoolean(GG_NO_SHUTDOWN_HOOK, false)) {
-                try {
-                    Runtime.getRuntime().addShutdownHook(shutdownHook = new Thread() {
-                        @Override public void run() {
-                            if (log.isInfoEnabled())
-                                log.info("Invoking shutdown hook...");
-
-                            GridNamedInstance.this.stop(true);
-                        }
-                    });
-
-                    if (log.isDebugEnabled())
-                        log.debug("Shutdown hook is installed.");
-                }
-                catch (IllegalStateException e) {
-                    stop(true);
-
-                    throw new IgniteCheckedException("Failed to install shutdown hook.", e);
-                }
-            }
-            else {
-                if (log.isDebugEnabled())
-                    log.debug("Shutdown hook has not been installed because environment " +
-                        "or system property " + GG_NO_SHUTDOWN_HOOK + " is set.");
-            }
-        }
-
-        /**
-         * @param cfgLog Configured logger.
-         * @param nodeId Local node ID.
-         * @return Initialized logger.
-         * @throws IgniteCheckedException If failed.
-         */
-        private IgniteLogger initLogger(@Nullable IgniteLogger cfgLog, UUID nodeId) throws IgniteCheckedException {
-            try {
-                if (cfgLog == null) {
-                    Class<?> log4jCls;
-
-                    try {
-                        log4jCls = Class.forName("org.gridgain.grid.logger.log4j.GridLog4jLogger");
-                    }
-                    catch (ClassNotFoundException | NoClassDefFoundError ignored) {
-                        log4jCls = null;
-                    }
-
-                    if (log4jCls != null) {
-                        URL url = U.resolveGridGainUrl("config/gridgain-log4j.xml");
-
-                        if (url == null) {
-                            File cfgFile = new File("config/gridgain-log4j.xml");
-
-                            if (!cfgFile.exists())
-                                cfgFile = new File("../config/gridgain-log4j.xml");
-
-                            if (cfgFile.exists()) {
-                                try {
-                                    url = cfgFile.toURI().toURL();
-                                }
-                                catch (MalformedURLException ignore) {
-                                    // No-op.
-                                }
-                            }
-                        }
-
-                        if (url != null) {
-                            boolean configured = (Boolean)log4jCls.getMethod("isConfigured").invoke(null);
-
-                            if (configured)
-                                url = null;
-                        }
-
-                        if (url != null) {
-                            Constructor<?> ctor = log4jCls.getConstructor(URL.class);
-
-                            cfgLog = (IgniteLogger)ctor.newInstance(url);
-                        }
-                        else
-                            cfgLog = (IgniteLogger)log4jCls.newInstance();
-                    }
-                    else
-                        cfgLog = new IgniteJavaLogger();
-                }
-
-                // Set node IDs for all file appenders.
-                if (cfgLog instanceof IgniteLoggerNodeIdAware)
-                    ((IgniteLoggerNodeIdAware)cfgLog).setNodeId(nodeId);
-
-                return cfgLog;
-            }
-            catch (Exception e) {
-                throw new IgniteCheckedException("Failed to create logger.", e);
-            }
-        }
-
-        /**
-         * Creates utility system cache configuration.
-         *
-         * @param client If {@code true} creates client-only cache configuration.
-         * @return Utility system cache configuration.
-         */
-        private CacheConfiguration utilitySystemCache(boolean client) {
-            CacheConfiguration cache = new CacheConfiguration();
-
-            cache.setName(CU.UTILITY_CACHE_NAME);
-            cache.setCacheMode(REPLICATED);
-            cache.setAtomicityMode(TRANSACTIONAL);
-            cache.setSwapEnabled(false);
-            cache.setQueryIndexEnabled(false);
-            cache.setPreloadMode(SYNC);
-            cache.setWriteSynchronizationMode(FULL_SYNC);
-            cache.setAffinity(new CacheRendezvousAffinityFunction(false, 100));
-
-            if (client)
-                cache.setDistributionMode(CLIENT_ONLY);
-
-            return cache;
-        }
-
-        /**
-         * Stops grid.
-         *
-         * @param cancel Flag indicating whether all currently running jobs
-         *      should be cancelled.
-         */
-        void stop(boolean cancel) {
-            // Stop cannot be called prior to start from public API,
-            // since it checks for STARTED state. So, we can assert here.
-            assert startGuard.get();
-
-            stop0(cancel);
-        }
-
-        /**
-         * @param cancel Flag indicating whether all currently running jobs
-         *      should be cancelled.
-         */
-        private synchronized void stop0(boolean cancel) {
-            IgniteKernal grid0 = grid;
-
-            // Double check.
-            if (grid0 == null) {
-                if (log != null)
-                    U.warn(log, "Attempting to stop an already stopped grid instance (ignore): " + name);
-
-                return;
-            }
-
-            if (shutdownHook != null)
-                try {
-                    Runtime.getRuntime().removeShutdownHook(shutdownHook);
-
-                    shutdownHook = null;
-
-                    if (log.isDebugEnabled())
-                        log.debug("Shutdown hook is removed.");
-                }
-                catch (IllegalStateException e) {
-                    // Shutdown is in progress...
-                    if (log.isDebugEnabled())
-                        log.debug("Shutdown is in progress (ignoring): " + e.getMessage());
-                }
-
-            // Unregister GridGain MBean.
-            unregisterFactoryMBean();
-
-            try {
-                grid0.stop(cancel);
-
-                if (log.isDebugEnabled())
-                    log.debug("Grid instance stop

<TRUNCATED>

[11/50] [abbrv] incubator-ignite git commit: # ignite-91

Posted by vo...@apache.org.
# ignite-91


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

Branch: refs/heads/ignite-16
Commit: e45b390e167d239b5c3eec88672001639964c31e
Parents: a92f084
Author: sboikov <se...@inria.fr>
Authored: Wed Jan 28 23:31:42 2015 +0300
Committer: sboikov <se...@inria.fr>
Committed: Wed Jan 28 23:31:42 2015 +0300

----------------------------------------------------------------------
 .../dht/atomic/GridDhtAtomicCache.java          |   4 +
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |  12 +-
 .../local/atomic/GridLocalAtomicCache.java      |  58 +++-
 .../transactions/IgniteTxLocalAdapter.java      |  13 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java | 289 +++++++++++++++++--
 .../dht/GridCacheAtomicFullApiSelfTest.java     |  11 +-
 6 files changed, 340 insertions(+), 47 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e45b390e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
index 1076041..d2150de 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridDhtAtomicCache.java
@@ -481,6 +481,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
     @SuppressWarnings("unchecked")
     @Override public IgniteFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+        A.notNull(key, "key");
+
         return removeAllAsync0(Collections.singletonList(key), null, entry, true, false, filter);
     }
 
@@ -520,6 +522,8 @@ public class GridDhtAtomicCache<K, V> extends GridDhtCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public IgniteFuture<Boolean> removeAsync(K key, V val) {
+        A.notNull(key, "key", val, "val");
+
         return removexAsync(key, ctx.equalsPeekArray(val));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e45b390e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
index 6f6f03b..539a462 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/dht/atomic/GridNearAtomicUpdateFuture.java
@@ -549,15 +549,19 @@ public class GridNearAtomicUpdateFuture<K, V> extends GridFutureAdapter<Object>
 
             // We still can get here if user pass map with single element.
             if (key == null) {
-                onDone(new GridCacheReturn<>(null, false));
+                NullPointerException err = new NullPointerException("Null key.");
 
-                return;
+                onDone(err);
+
+                throw err;
             }
 
             if (val == null && op != GridCacheOperation.DELETE) {
-                onDone(new GridCacheReturn<>(null, false));
+                NullPointerException err = new NullPointerException("Null value.");
 
-                return;
+                onDone(err);
+
+                throw err;
             }
 
             if (cctx.portableEnabled()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e45b390e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 2c360ba..6e72c84 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -248,6 +248,8 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException {
+        A.notNull(oldVal, "oldVal");
+
         return putx(key, newVal, ctx.equalsPeekArray(oldVal));
     }
 
@@ -442,7 +444,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public boolean remove(K key, V val) throws IgniteCheckedException {
-        A.notNull(key, "key");
+        A.notNull(key, "key", val, "val");
 
         ctx.denyOnLocalRead();
 
@@ -665,9 +667,30 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(Set<? extends K> keys,
-        EntryProcessor<K, V, T> entryProcessor,
+        final EntryProcessor<K, V, T> entryProcessor,
         Object... args) throws IgniteCheckedException {
-        return invokeAllAsync(keys, entryProcessor, args).get();
+        A.notNull(keys, "keys", entryProcessor, "entryProcessor");
+
+        if (keyCheck)
+            validateCacheKeys(keys);
+
+        ctx.denyOnLocalRead();
+
+        Map<? extends K, EntryProcessor> invokeMap = F.viewAsMap(keys, new C1<K, EntryProcessor>() {
+            @Override public EntryProcessor apply(K k) {
+                return entryProcessor;
+            }
+        });
+
+        return (Map<K, EntryProcessorResult<T>>)updateAllInternal(TRANSFORM,
+            invokeMap.keySet(),
+            invokeMap.values(),
+            args,
+            expiryPerCall(),
+            true,
+            false,
+            null,
+            ctx.writeThrough());
     }
 
     /** {@inheritDoc} */
@@ -736,10 +759,26 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
+    @SuppressWarnings("unchecked")
     @Override public <T> Map<K, EntryProcessorResult<T>> invokeAll(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args) throws IgniteCheckedException {
-        return invokeAllAsync(map, args).get();
+        A.notNull(map, "map");
+
+        if (keyCheck)
+            validateCacheKeys(map.keySet());
+
+        ctx.denyOnLocalRead();
+
+        return (Map<K, EntryProcessorResult<T>>)updateAllInternal(TRANSFORM,
+            map.keySet(),
+            map.values(),
+            args,
+            expiryPerCall(),
+            true,
+            false,
+            null,
+            ctx.writeThrough());
     }
 
     /** {@inheritDoc} */
@@ -910,10 +949,13 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         boolean intercept = ctx.config().getInterceptor() != null;
 
         for (K key : keys) {
+            if (key == null)
+                throw new NullPointerException("Null key.");
+
             Object val = valsIter != null ? valsIter.next() : null;
 
-            if (key == null)
-                continue;
+            if (val == null && op != DELETE)
+                throw new NullPointerException("Null value.");
 
             while (true) {
                 GridCacheEntryEx<K, V> entry = null;
@@ -1044,7 +1086,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                 Object val = valsIter != null ? valsIter.next() : null;
 
                 if (val == null && op != DELETE)
-                    continue;
+                    throw new NullPointerException("Null value.");
 
                 try {
                     try {
@@ -1394,7 +1436,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         while (true) {
             for (K key : keys) {
                 if (key == null)
-                    continue;
+                    throw new NullPointerException("Null key.");
 
                 GridCacheEntryEx<K, V> entry = entryEx(key);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e45b390e/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
index d9c49d8..90e09d1 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/transactions/IgniteTxLocalAdapter.java
@@ -1901,6 +1901,12 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
             groupLockSanityCheck(cacheCtx, keys);
 
             for (K key : keys) {
+                if (key == null) {
+                    setRollbackOnly();
+
+                    throw new NullPointerException("Null key.");
+                }
+
                 V val = rmv || lookup == null ? null : lookup.get(key);
                 EntryProcessor entryProcessor = invokeMap == null ? null : invokeMap.get(key);
 
@@ -1930,13 +1936,10 @@ public abstract class IgniteTxLocalAdapter<K, V> extends IgniteTxAdapter<K, V>
                     drExpireTime = -1L;
                 }
 
-                if (key == null)
-                    continue;
-
                 if (!rmv && val == null && entryProcessor == null) {
-                    skipped = skip(skipped, key);
+                    setRollbackOnly();
 
-                    continue;
+                    throw new NullPointerException("Null value.");
                 }
 
                 if (cacheCtx.portableEnabled())

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e45b390e/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 905f6b4..84a9b31 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
@@ -467,7 +467,14 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         if (tx != null)
             tx.commit();
 
-        assert cache().getAll(null).isEmpty();
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache().getAll(null).isEmpty();
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
         assert cache().getAll(Collections.<String>emptyList()).isEmpty();
 
         Map<String, Integer> map1 = cache().getAll(F.asList("key1", "key2", "key9999"));
@@ -494,7 +501,6 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         if (txEnabled()) {
             tx = cache().txStart();
 
-            assert cache().getAll(null).isEmpty();
             assert cache().getAll(Collections.<String>emptyList()).isEmpty();
 
             map1 = cache().getAll(F.asList("key1", "key2", "key9999"));
@@ -660,11 +666,17 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         cache().put("key1", 1);
         cache().put("key2", 2);
 
-        IgniteFuture<Map<String, Integer>> fut1 = cache().getAllAsync(null);
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache().getAllAsync(null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
         IgniteFuture<Map<String, Integer>> fut2 = cache().getAllAsync(Collections.<String>emptyList());
         IgniteFuture<Map<String, Integer>> fut3 = cache().getAllAsync(F.asList("key1", "key2"));
 
-        assert fut1.get().isEmpty();
         assert fut2.get().isEmpty();
         assert fut3.get().size() == 2 : "Invalid map: " + fut3.get();
         assert fut3.get().get("key1") == 1;
@@ -988,25 +1000,26 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
         }, NullPointerException.class, null);
 
         {
-            Map<String, Integer> m = new HashMap<>(2);
-
-            m.put("key1", 1);
-            m.put(null, 2);
-
-            // WARN: F.asMap() doesn't work here, because it will throw NPE.
-
-            cache.putAll(m);
-        }
-
-        {
-            Set<String> keys = new HashSet<>(2);
+            final Set<String> keys = new LinkedHashSet<>(2);
 
             keys.add("key1");
             keys.add(null);
 
-            // WARN: F.asSet() doesn't work here, because it will throw NPE.
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    cache.invokeAll(keys, INCR_PROCESSOR);
 
-            cache.invokeAll(keys, INCR_PROCESSOR);
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override public Void call() throws Exception {
+                    cache.invokeAll(F.asSet("key1"), null);
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
         }
     }
 
@@ -1695,32 +1708,139 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
     /**
      * @throws Exception In case of error.
      */
+    public void testNullInTx() throws Exception {
+        if (!txEnabled())
+            return;
+
+        final IgniteCache<String, Integer> cache = jcache();
+
+        for (int i = 0; i < 100; i++) {
+            final String key = "key-" + i;
+
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                public Void call() throws Exception {
+                    IgniteTransactions txs = grid(0).transactions();
+
+                    try (IgniteTx tx = txs.txStart()) {
+                        cache.put(key, 1);
+
+                        cache.put(null, 2);
+
+                        tx.commit();
+                    }
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            assertNull(cache.get(key));
+
+            cache.put(key, 1);
+
+            assertEquals(1, (int) cache.get(key));
+
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                public Void call() throws Exception {
+                    IgniteTransactions txs = grid(0).transactions();
+
+                    try (IgniteTx tx = txs.txStart()) {
+                        cache.put(key, 2);
+
+                        cache.remove(null);
+
+                        tx.commit();
+                    }
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            assertEquals(1, (int) cache.get(key));
+
+            cache.put(key, 2);
+
+            assertEquals(2, (int)cache.get(key));
+
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                public Void call() throws Exception {
+                    IgniteTransactions txs = grid(0).transactions();
+
+                    Map<String, Integer> map = new LinkedHashMap<String, Integer>();
+
+                    map.put("k1", 1);
+                    map.put("k2", 2);
+                    map.put(null, 3);
+
+                    try (IgniteTx tx = txs.txStart()) {
+                        cache.put(key, 1);
+
+                        cache.putAll(map);
+
+                        tx.commit();
+                    }
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            assertNull(cache.get("k1"));
+            assertNull(cache.get("k2"));
+
+            assertEquals(2, (int) cache.get(key));
+
+            cache.put(key, 3);
+
+            assertEquals(3, (int)cache.get(key));
+        }
+    }
+
+    /**
+     * @throws Exception In case of error.
+     */
     public void testPutAllWithNulls() throws Exception {
-        final GridCache<String, Integer> cache = cache();
+        final IgniteCache<String, Integer> cache = jcache();
 
         {
-            Map<String, Integer> m = new HashMap<>(2);
+            final Map<String, Integer> m = new LinkedHashMap<>(2);
 
             m.put("key1", 1);
             m.put(null, 2);
 
-            // WARN: F.asMap() doesn't work here, because it will throw NPE.
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override
+                public Void call() throws Exception {
+                    cache.putAll(m);
 
-            cache.putAll(m);
+                    return null;
+                }
+            }, NullPointerException.class, null);
 
-            assertNotNull(cache.get("key1"));
+            cache.put("key1", 1);
+
+            assertEquals(1, (int)cache.get("key1"));
         }
 
         {
-            Map<String, Integer> m = new HashMap<>(2);
+            final Map<String, Integer> m = new LinkedHashMap<>(2);
 
             m.put("key3", 3);
             m.put("key4", null);
 
+            GridTestUtils.assertThrows(log, new Callable<Void>() {
+                @Override
+                public Void call() throws Exception {
+                    cache.putAll(m);
+
+                    return null;
+                }
+            }, NullPointerException.class, null);
+
+            m.put("key4", 4);
+
             cache.putAll(m);
 
-            assertNotNull(cache.get("key3"));
-            assertNull(cache.get("key4"));
+            assertEquals(3, (int) cache.get("key3"));
+            assertEquals(4, (int)cache.get("key4"));
         }
 
         assertThrows(log, new Callable<Object>() {
@@ -1730,6 +1850,78 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
                 return null;
             }
         }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.getAndPut("key1", null);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.put(null, 1);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace(null, 1);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.getAndReplace(null, 1);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace("key", null);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.getAndReplace("key", null);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace(null, 1, 2);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace("key", null, 2);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
+
+        assertThrows(log, new Callable<Object>() {
+            @Nullable @Override public Object call() throws Exception {
+                cache.replace("key", 1, null);
+
+                return null;
+            }
+        }, NullPointerException.class, A.NULL_MSG_PREFIX);
     }
 
     /**
@@ -2498,14 +2690,53 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception In case of error.
      */
     public void testRemoveAllWithNulls() throws Exception {
-        GridCache<String, Integer> cache = cache();
+        final IgniteCache<String, Integer> cache = jcache();
 
-        Collection<String> c = new LinkedList<>();
+        final Set<String> c = new LinkedHashSet<>();
 
         c.add("key1");
         c.add(null);
 
-        cache.removeAll(c);
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override
+            public Void call() throws Exception {
+                cache.removeAll(c);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.removeAll(null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.remove(null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.getAndRemove(null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache.remove("key1", null);
+
+                return null;
+            }
+        }, NullPointerException.class, null);
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/e45b390e/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicFullApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicFullApiSelfTest.java
index 4a817cf..a9cc2e7 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/distributed/dht/GridCacheAtomicFullApiSelfTest.java
@@ -20,8 +20,10 @@ package org.apache.ignite.internal.processors.cache.distributed.dht;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
 import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.testframework.*;
 
 import java.util.*;
+import java.util.concurrent.*;
 
 import static org.apache.ignite.cache.CacheAtomicWriteOrderMode.*;
 import static org.apache.ignite.cache.CacheAtomicityMode.*;
@@ -74,7 +76,14 @@ public class GridCacheAtomicFullApiSelfTest extends GridCachePartitionedFullApiS
         cache().put("key1", 1);
         cache().put("key2", 2);
 
-        assert cache().getAll((Collection<String>)null).isEmpty();
+        GridTestUtils.assertThrows(log, new Callable<Void>() {
+            @Override public Void call() throws Exception {
+                cache().getAll(null).isEmpty();
+
+                return null;
+            }
+        }, NullPointerException.class, null);
+
         assert cache().getAll(Collections.<String>emptyList()).isEmpty();
 
         Map<String, Integer> map1 = cache().getAll(F.asList("key1", "key2", "key9999"));


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

Posted by vo...@apache.org.
# sprint-1 moved existing IgniteFuture to internal package


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

Branch: refs/heads/ignite-16
Commit: d5bef132a43b2ef60afed3e454f46a54a088feb0
Parents: 87d1de1
Author: sboikov <sb...@gridgain.com>
Authored: Thu Jan 29 10:19:50 2015 +0300
Committer: sboikov <sb...@gridgain.com>
Committed: Thu Jan 29 10:20:45 2015 +0300

----------------------------------------------------------------------
 .../ComputeFibonacciContinuationExample.java    |  11 +-
 .../compute/ComputeRunnableExample.java         |   5 +-
 .../examples/ScalarContinuationExample.scala    |   9 +-
 .../ClientAbstractMultiThreadedSelfTest.java    |   6 +-
 .../rest/RestProcessorStartSelfTest.java        |   3 +-
 .../org/apache/ignite/IgniteDataLoader.java     |  14 +-
 .../java/org/apache/ignite/IgniteScheduler.java |   6 +-
 .../apache/ignite/IgniteSystemProperties.java   |   6 +-
 .../org/apache/ignite/cache/CacheEntry.java     |  27 +-
 .../apache/ignite/cache/CacheProjection.java    |  43 +--
 .../java/org/apache/ignite/cache/GridCache.java |   5 +-
 .../apache/ignite/cache/query/CacheQueries.java |   7 +-
 .../ignite/cache/query/CacheQueryFuture.java    |   4 +-
 .../ignite/compute/ComputeTaskFuture.java       |   4 +-
 .../ignite/compute/ComputeTaskSession.java      |   3 +-
 .../org/apache/ignite/hadoop/GridHadoop.java    |   6 +-
 .../java/org/apache/ignite/internal/GridEx.java |   2 +-
 .../ignite/internal/GridJobSessionImpl.java     |   2 +-
 .../org/apache/ignite/internal/GridKernal.java  |  14 +-
 .../ignite/internal/GridTaskSessionImpl.java    |   2 +-
 .../ignite/internal/IgniteInternalFuture.java   | 190 +++++++++++
 .../ignite/internal/IgniteSchedulerImpl.java    |   6 +-
 .../internal/executor/GridExecutorService.java  |  44 +--
 .../discovery/GridDiscoveryManager.java         |   2 +-
 .../eventstorage/GridEventStorageManager.java   |   4 +-
 .../affinity/GridAffinityAssignmentCache.java   |   5 +-
 .../affinity/GridAffinityProcessor.java         |   8 +-
 .../processors/cache/CacheLockImpl.java         |   4 +-
 .../processors/cache/GridCacheAdapter.java      | 316 +++++++++----------
 .../cache/GridCacheAffinityManager.java         |   8 +-
 .../processors/cache/GridCacheContext.java      |  12 +-
 .../processors/cache/GridCacheEntryImpl.java    |  27 +-
 .../cache/GridCacheEvictionEntry.java           |  27 +-
 .../cache/GridCacheEvictionManager.java         |   4 +-
 .../cache/GridCacheExplicitLockSpan.java        |   4 +-
 .../cache/GridCacheFilterEvaluationEntry.java   |  27 +-
 .../processors/cache/GridCacheFuture.java       |   3 +-
 .../processors/cache/GridCacheIoManager.java    |  15 +-
 .../cache/GridCacheMultiTxFuture.java           |   6 +-
 .../processors/cache/GridCacheMvccManager.java  |  29 +-
 .../GridCachePartitionExchangeManager.java      |   9 +-
 .../processors/cache/GridCachePreloader.java    |   7 +-
 .../cache/GridCachePreloaderAdapter.java        |   9 +-
 .../processors/cache/GridCacheProcessor.java    |   9 +-
 .../processors/cache/GridCacheProjectionEx.java |  27 +-
 .../cache/GridCacheProjectionImpl.java          |  69 ++--
 .../processors/cache/GridCacheProxyImpl.java    |  73 ++---
 .../cache/GridCacheSharedContext.java           |  13 +-
 .../processors/cache/GridCacheUtils.java        |   7 +-
 .../processors/cache/IgniteCacheProxy.java      |  13 +-
 .../GridCacheAtomicSequenceImpl.java            |   3 +-
 ...ridCacheOptimisticCheckPreparedTxFuture.java |   7 +-
 ...dCachePessimisticCheckCommittedTxFuture.java |   7 +-
 .../distributed/GridCacheTxFinishSync.java      |   8 +-
 .../GridDistributedCacheAdapter.java            |   7 +-
 .../GridDistributedTxRemoteAdapter.java         |   7 +-
 .../distributed/dht/GridDhtCacheAdapter.java    |  12 +-
 .../distributed/dht/GridDhtCacheEntry.java      |   7 +-
 .../distributed/dht/GridDhtEmbeddedFuture.java  |   8 +-
 .../cache/distributed/dht/GridDhtFuture.java    |   4 +-
 .../cache/distributed/dht/GridDhtGetFuture.java |  11 +-
 .../distributed/dht/GridDhtLocalPartition.java  |   5 +-
 .../distributed/dht/GridDhtLockFuture.java      |  11 +-
 .../distributed/dht/GridDhtTopologyFuture.java  |   4 +-
 .../dht/GridDhtTransactionalCacheAdapter.java   |  29 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |  11 +-
 .../cache/distributed/dht/GridDhtTxLocal.java   |  19 +-
 .../distributed/dht/GridDhtTxLocalAdapter.java  |  11 +-
 .../distributed/dht/GridDhtTxPrepareFuture.java |  11 +-
 .../dht/GridPartitionedGetFuture.java           |  21 +-
 .../dht/atomic/GridDhtAtomicCache.java          |  93 +++---
 .../dht/atomic/GridNearAtomicUpdateFuture.java  |   5 +-
 .../dht/colocated/GridDhtColocatedCache.java    |  21 +-
 .../colocated/GridDhtColocatedLockFuture.java   |  23 +-
 .../dht/preloader/GridDhtForceKeysFuture.java   |   7 +-
 .../preloader/GridDhtPartitionDemandPool.java   |  13 +-
 .../GridDhtPartitionsExchangeFuture.java        |  20 +-
 .../dht/preloader/GridDhtPreloader.java         |  29 +-
 .../distributed/near/GridNearAtomicCache.java   |  45 +--
 .../distributed/near/GridNearCacheAdapter.java  |  15 +-
 .../distributed/near/GridNearGetFuture.java     |  25 +-
 .../distributed/near/GridNearLockFuture.java    |  23 +-
 .../near/GridNearTransactionalCache.java        |   9 +-
 .../near/GridNearTxFinishFuture.java            |  17 +-
 .../cache/distributed/near/GridNearTxLocal.java |  58 ++--
 .../near/GridNearTxPrepareFuture.java           |  17 +-
 .../processors/cache/local/GridLocalCache.java  |   7 +-
 .../processors/cache/local/GridLocalTx.java     |  12 +-
 .../local/atomic/GridLocalAtomicCache.java      |  69 ++--
 .../GridCacheDistributedFieldsQueryFuture.java  |   4 +-
 .../query/GridCacheDistributedQueryManager.java |   9 +-
 .../query/GridCacheFieldsQueryErrorFuture.java  |   3 +-
 .../query/GridCacheLocalFieldsQueryFuture.java  |   4 +-
 .../cache/query/GridCacheLocalQueryFuture.java  |   3 +-
 .../cache/query/GridCacheQueriesEx.java         |   6 +-
 .../cache/query/GridCacheQueriesImpl.java       |  11 +-
 .../cache/query/GridCacheQueriesProxy.java      |  11 +-
 .../cache/query/GridCacheQueryManager.java      |  48 +--
 .../query/GridCacheQueryMetadataAware.java      |   4 +-
 .../GridCacheContinuousQueryEntry.java          |  27 +-
 .../transactions/IgniteTransactionsImpl.java    |   3 +-
 .../cache/transactions/IgniteTxAdapter.java     |   7 +-
 .../cache/transactions/IgniteTxEx.java          |   9 +-
 .../cache/transactions/IgniteTxHandler.java     |  48 +--
 .../transactions/IgniteTxLocalAdapter.java      |  93 +++---
 .../cache/transactions/IgniteTxLocalEx.java     |  17 +-
 .../cache/transactions/IgniteTxManager.java     |  15 +-
 .../cache/transactions/IgniteTxProxyImpl.java   |   9 +-
 .../closure/GridClosureProcessor.java           |  56 ++--
 .../continuous/GridContinuousProcessor.java     |   4 +-
 .../dataload/GridDataLoaderProcessor.java       |   5 +-
 .../dataload/IgniteDataLoaderImpl.java          |  58 ++--
 .../dr/GridDrDataLoadCacheUpdater.java          |   3 +-
 .../email/IgniteEmailProcessorAdapter.java      |   5 +-
 .../email/IgniteNoopEmailProcessor.java         |   5 +-
 .../processors/fs/GridGgfsAsyncImpl.java        |   3 +-
 .../processors/fs/GridGgfsDataManager.java      |  30 +-
 .../internal/processors/fs/GridGgfsEx.java      |   3 +-
 .../internal/processors/fs/GridGgfsImpl.java    |  16 +-
 .../processors/fs/GridGgfsInputStreamImpl.java  |  19 +-
 .../processors/fs/GridGgfsIpcHandler.java       |   4 +-
 .../processors/fs/GridGgfsMetaManager.java      |   9 +-
 .../internal/processors/fs/GridGgfsServer.java  |   8 +-
 .../processors/fs/GridGgfsServerHandler.java    |   4 +-
 .../processors/fs/IgniteFsOutputStreamImpl.java |   3 +-
 .../hadoop/IgniteHadoopNoopProcessor.java       |   5 +-
 .../hadoop/IgniteHadoopProcessorAdapter.java    |   5 +-
 .../processors/query/GridQueryIndexing.java     |   4 +-
 .../processors/query/GridQueryProcessor.java    |  12 +-
 .../processors/rest/GridRestProcessor.java      |  18 +-
 .../rest/GridRestProtocolHandler.java           |   4 +-
 .../rest/handlers/GridRestCommandHandler.java   |   4 +-
 .../handlers/cache/GridCacheCommandHandler.java |  54 ++--
 .../cache/GridCacheQueryCommandHandler.java     |  12 +-
 .../handlers/log/GridLogCommandHandler.java     |   3 +-
 .../metadata/GridPortableMetadataHandler.java   |   3 +-
 .../handlers/task/GridTaskCommandHandler.java   |   8 +-
 .../top/GridTopologyCommandHandler.java         |   3 +-
 .../version/GridVersionCommandHandler.java      |   3 +-
 .../tcp/GridTcpMemcachedNioListener.java        |  17 +-
 .../protocols/tcp/GridTcpRestNioListener.java   |   5 +-
 .../service/GridServiceProcessor.java           |  20 +-
 .../processors/streamer/IgniteStreamerImpl.java |   4 +-
 .../apache/ignite/internal/util/GridUtils.java  |   6 +-
 .../util/future/GridCompoundFuture.java         |  28 +-
 .../util/future/GridCompoundIdentityFuture.java |   2 +-
 .../util/future/GridEmbeddedFuture.java         |  38 +--
 .../util/future/GridFinishedFuture.java         |   8 +-
 .../util/future/GridFinishedFutureEx.java       |   9 +-
 .../internal/util/future/GridFutureAdapter.java |  30 +-
 .../util/future/GridFutureAdapterEx.java        |  31 +-
 .../util/future/GridFutureChainListener.java    |   8 +-
 .../ignite/internal/util/lang/GridFunc.java     |  53 ++--
 .../ignite/internal/util/nio/GridNioFuture.java |   2 +-
 .../apache/ignite/internal/util/typedef/X.java  |   6 +-
 .../visor/cache/VisorCachePreloadTask.java      |   6 +-
 .../apache/ignite/lang/IgniteAsyncSupport.java  |   4 +-
 .../ignite/lang/IgniteAsyncSupportAdapter.java  |  11 +-
 .../org/apache/ignite/lang/IgniteFuture.java    | 189 -----------
 .../ignite/scheduler/SchedulerFuture.java       |   4 +-
 .../spi/discovery/tcp/TcpDiscoverySpi.java      |   4 +-
 .../org/apache/ignite/GridTestTaskSession.java  |   4 +-
 .../store/GridCacheBalancingStoreSelfTest.java  |   3 +-
 ...CacheJdbcBlobStoreMultithreadedSelfTest.java |   5 +-
 .../GridJobMasterLeaveAwareSelfTest.java        |  56 ++--
 .../internal/GridMultipleJobsSelfTest.java      |   6 +-
 .../internal/GridProjectionAbstractTest.java    |  20 +-
 .../ignite/internal/GridReduceSelfTest.java     |   2 +-
 .../GridTaskFutureImplStopGridSelfTest.java     |   5 +-
 .../internal/GridTaskListenerSelfTest.java      |   4 +-
 .../GridCheckpointManagerAbstractSelfTest.java  |   2 +-
 .../events/GridEventStorageManagerSelfTest.java |   3 +-
 .../GridCacheAbstractFailoverSelfTest.java      |   3 +-
 .../cache/GridCacheAbstractFlagsTest.java       |   4 +-
 ...cheAbstractFullApiMultithreadedSelfTest.java |   5 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  76 ++---
 .../GridCacheAbstractIteratorsSelfTest.java     |  10 +-
 .../cache/GridCacheAbstractMetricsSelfTest.java |  18 +-
 .../GridCacheAbstractRemoveFailureTest.java     |   6 +-
 .../GridCacheAsyncOperationsLimitSelfTest.java  |   8 +-
 .../cache/GridCacheConcurrentMapTest.java       |   6 +-
 .../GridCacheFinishPartitionsSelfTest.java      |  23 +-
 ...CacheFullTextQueryMultithreadedSelfTest.java |   6 +-
 ...idCacheGetAndTransformStoreAbstractTest.java |   4 +-
 .../GridCacheGroupLockAbstractSelfTest.java     |   2 +-
 .../GridCacheGroupLockFailoverSelfTest.java     |   5 +-
 .../cache/GridCacheIncrementTransformTest.java  |   4 +-
 .../cache/GridCacheLuceneQueryIndexTest.java    |  18 +-
 .../GridCacheMissingCommitVersionSelfTest.java  |   3 +-
 ...GridCacheMixedPartitionExchangeSelfTest.java |   4 +-
 .../cache/GridCacheMultiUpdateLockSelfTest.java |   3 +-
 ...HeapMultiThreadedUpdateAbstractSelfTest.java |   5 +-
 .../GridCachePreloadingEvictionsSelfTest.java   |   2 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |  13 +-
 .../GridCacheReferenceCleanupSelfTest.java      |   6 +-
 ...ridCacheReplicatedSynchronousCommitTest.java |   4 +-
 .../processors/cache/GridCacheStopSelfTest.java |   6 +-
 .../cache/GridCacheSwapPreloadSelfTest.java     |   4 +-
 .../cache/GridCacheTtlManagerLoadTest.java      |   3 +-
 .../GridCacheVariableTopologySelfTest.java      |   4 +-
 ...idCacheWriteBehindStoreAbstractSelfTest.java |   4 +-
 .../GridCacheWriteBehindStoreAbstractTest.java  |   4 +-
 .../GridCacheWriteBehindStoreSelfTest.java      |   6 +-
 .../IgniteCacheEntryListenerAbstractTest.java   |   5 +-
 .../cache/IgniteCacheInvokeAbstractTest.java    |   6 +-
 .../IgniteTxMultiThreadedAbstractTest.java      |   6 +-
 ...eAbstractDataStructuresFailoverSelfTest.java |  27 +-
 ...actQueueFailoverDataConsistencySelfTest.java |   7 +-
 .../GridCacheCountDownLatchSelfTest.java        |   6 +-
 .../GridCacheQueueCleanupSelfTest.java          |   3 +-
 ...ridCacheQueueJoinedNodeSelfAbstractTest.java |   7 +-
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |  17 +-
 .../GridCacheSetAbstractSelfTest.java           |   7 +-
 .../GridCacheSetFailoverAbstractSelfTest.java   |   4 +-
 ...PartitionedQueueCreateMultiNodeSelfTest.java |   6 +-
 ...dCachePartitionedQueueEntryMoveSelfTest.java |   6 +-
 .../GridCacheAbstractJobExecutionTest.java      |   6 +-
 .../GridCacheAtomicTimeoutSelfTest.java         |   9 +-
 .../GridCacheBasicOpAbstractTest.java           |   7 +-
 .../distributed/GridCacheLockAbstractTest.java  |   6 +-
 .../GridCacheMultiNodeAbstractTest.java         |  15 +-
 ...dCacheMultithreadedFailoverAbstractTest.java |   4 +-
 .../GridCacheNodeFailureAbstractTest.java       |   6 +-
 ...cOriginatingNodeFailureAbstractSelfTest.java |   2 +-
 .../IgniteTxPreloadAbstractTest.java            |   4 +-
 .../dht/GridCacheColocatedDebugTest.java        |   5 +-
 ...GridCacheDhtEvictionNearReadersSelfTest.java |   6 +-
 .../dht/GridCacheDhtEvictionSelfTest.java       |   8 +-
 .../dht/GridCacheDhtPreloadPutGetSelfTest.java  |   6 +-
 .../dht/GridCacheDhtPreloadSelfTest.java        |   5 +-
 .../GridCacheDhtPreloadStartStopSelfTest.java   |   3 +-
 ...dCachePartitionedTopologyChangeSelfTest.java |  28 +-
 ...eAtomicInvalidPartitionHandlingSelfTest.java |   3 +-
 .../GridCachePartitionedEntryLockSelfTest.java  |   5 +-
 ...titionedExplicitLockNodeFailureSelfTest.java |   6 +-
 ...achePartitionedMultiNodeFullApiSelfTest.java |   5 +-
 ...hePartitionedQueryMultiThreadedSelfTest.java |   6 +-
 .../GridCachePartitionedTxSalvageSelfTest.java  |   3 +-
 .../GridCacheReplicatedEvictionSelfTest.java    |   5 +-
 .../GridCacheBatchEvictUnswapSelfTest.java      |   5 +-
 ...heConcurrentEvictionConsistencySelfTest.java |   4 +-
 .../GridCacheConcurrentEvictionsSelfTest.java   |   4 +-
 .../cache/eviction/GridCacheMockEntry.java      |  27 +-
 ...cheSynchronousEvictionsFailoverSelfTest.java |   4 +-
 .../closure/GridClosureProcessorSelfTest.java   |  27 +-
 .../continuous/GridEventConsumeSelfTest.java    |   8 +-
 .../GridDataLoaderProcessorSelfTest.java        |  23 +-
 .../processors/fs/GridGgfsAbstractSelfTest.java |   4 +-
 .../fs/GridGgfsDataManagerSelfTest.java         |   9 +-
 .../processors/fs/GridGgfsTaskSelfTest.java     |   5 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |   3 +-
 .../GridServiceProcessorAbstractSelfTest.java   |  20 +-
 .../GridServiceProcessorMultiNodeSelfTest.java  |   8 +-
 .../streamer/GridStreamerFailoverSelfTest.java  |   4 +-
 .../timeout/GridTimeoutProcessorSelfTest.java   |   5 +-
 .../util/future/GridCompoundFutureSelfTest.java |   8 +-
 .../util/future/GridEmbeddedFutureSelfTest.java |   7 +-
 .../util/future/GridFinishedFutureSelfTest.java |   7 +-
 .../util/future/GridFutureAdapterSelfTest.java  |  23 +-
 .../future/GridFutureListenPerformanceTest.java |   5 +-
 .../shmem/GridIpcSharedMemorySpaceSelfTest.java |   6 +-
 .../internal/util/nio/GridNioSelfTest.java      |   8 +-
 .../offheap/GridOffHeapMapAbstractSelfTest.java |   7 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |   3 +-
 ...apPartitionedMapPerformanceAbstractTest.java |  10 +-
 .../unsafe/GridUnsafeMemorySelfTest.java        |   6 +-
 .../ignite/jvmtest/ConcurrentMapTest.java       |   4 +-
 .../ignite/jvmtest/NetworkFailureTest.java      |  16 +-
 .../QueueSizeCounterMultiThreadedTest.java      |   4 +-
 .../jvmtest/ReadWriteLockMultiThreadedTest.java |  10 +-
 .../ignite/lang/GridBasicPerformanceTest.java   |   5 +-
 .../apache/ignite/lang/GridFuncSelfTest.java    |  13 +-
 .../lang/GridFutureListenPerformanceTest.java   |   5 +-
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |   6 +-
 .../cache/GridCacheAbstractLoadTest.java        |   6 +-
 .../loadtests/cache/GridCacheSwapLoadTest.java  |  13 +-
 .../GridCacheWriteBehindStoreLoadTest.java      |   4 +-
 .../loadtests/colocation/GridTestMain.java      |  10 +-
 .../communication/GridIoManagerBenchmark0.java  |  10 +-
 .../GridContinuousOperationsLoadTest.java       |   4 +-
 ...GridJobExecutionLoadTestClientSemaphore.java |   5 +-
 ...JobExecutionSingleNodeSemaphoreLoadTest.java |   7 +-
 .../mergesort/GridMergeSortLoadTask.java        |   6 +-
 ...apPartitionedMapPerformanceAbstractTest.java |  10 +-
 .../streamer/GridStreamerIndexLoadTest.java     |   8 +-
 .../marshaller/GridMarshallerAbstractTest.java  |  14 +-
 .../ignite/messaging/GridMessagingSelfTest.java |   6 +-
 ...redFsCheckpointSpiMultiThreadedSelfTest.java |   8 +-
 .../collision/GridTestCollisionTaskSession.java |   3 +-
 ...mmunicationSpiConcurrentConnectSelfTest.java |   3 +-
 .../tcp/GridTcpCommunicationSpiLanTest.java     |   3 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |   9 +-
 ...GridTcpCommunicationSpiRecoverySelfTest.java |   7 +-
 .../tcp/GridTcpDiscoveryMultiThreadedTest.java  |   5 +-
 .../file/GridFileSwapSpaceSpiSelfTest.java      |   7 +-
 .../GridCacheStoreValueBytesTest.java           |   7 +-
 .../index/GridStreamerIndexSelfTest.java        |   8 +-
 .../window/GridStreamerWindowSelfTest.java      |   4 +-
 .../ignite/testframework/GridTestUtils.java     |   8 +-
 .../testframework/junits/GridAbstractTest.java  |   8 +-
 .../cache/GridAbstractCacheStoreSelfTest.java   |   3 +-
 .../junits/common/GridCommonAbstractTest.java   |   4 +-
 ...dConcurrentLinkedDequeMultiThreadedTest.java |   6 +-
 ...rrentLinkedHashMapMultiThreadedSelfTest.java |  10 +-
 .../apache/ignite/util/GridIndexFillTest.java   |   4 +-
 .../ignite/util/GridSnapshotLockSelfTest.java   |   6 +-
 .../util/GridSpinReadWriteLockSelfTest.java     |   4 +-
 .../processors/email/IgniteEmailProcessor.java  |   5 +-
 .../GridHadoopClientProtocolProvider.java       |   8 +-
 .../processors/hadoop/GridHadoopImpl.java       |   6 +-
 .../hadoop/IgniteHadoopProcessor.java           |   5 +-
 .../hadoop/jobtracker/GridHadoopJobTracker.java |  19 +-
 .../proto/GridHadoopProtocolJobStatusTask.java  |   7 +-
 .../hadoop/shuffle/GridHadoopShuffle.java       |   2 +-
 .../hadoop/shuffle/GridHadoopShuffleJob.java    |   7 +-
 .../GridHadoopExternalTaskExecutor.java         |  16 +-
 .../child/GridHadoopChildProcessRunner.java     |  18 +-
 ...idHadoopDefaultMapReducePlannerSelfTest.java |   8 +-
 .../hadoop/GridHadoopJobTrackerSelfTest.java    |   3 +-
 .../GridHadoopMapReduceEmbeddedSelfTest.java    |   4 +-
 .../hadoop/GridHadoopMapReduceTest.java         |   4 +-
 .../hadoop/GridHadoopTaskExecutionSelfTest.java |  12 +-
 .../GridHadoopExecutorServiceTest.java          |   4 +-
 ...GridHadoopExternalTaskExecutionSelfTest.java |   6 +-
 .../ggfs/GridGgfsPerformanceBenchmark.java      |   4 +-
 .../processors/query/h2/IgniteH2Indexing.java   |   4 +-
 .../h2/twostep/GridReduceQueryExecutor.java     |   2 +-
 .../cache/GridCacheOffHeapAndSwapSelfTest.java  |   4 +-
 .../GridCacheQueryMultiThreadedSelfTest.java    |  16 +-
 ...idCacheReduceQueryMultithreadedSelfTest.java |   6 +-
 .../near/GridCacheQueryNodeRestartSelfTest.java |   5 +-
 .../GridCacheReplicatedQuerySelfTest.java       |   2 +-
 .../query/h2/GridH2IndexRebuildTest.java        |   7 +-
 .../query/h2/GridH2IndexingGeoSelfTest.java     |   6 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |   2 +-
 .../scalar/pimps/ScalarProjectionPimp.scala     |  25 +-
 .../processors/schedule/ScheduleFutureImpl.java |  22 +-
 .../schedule/GridScheduleSelfTest.java          |  13 +-
 .../GridTaskUriDeploymentDeadlockSelfTest.java  |   2 +-
 .../internal/websession/WebSessionSelfTest.java |   4 +-
 340 files changed, 2289 insertions(+), 2197 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/examples/src/main/java/org/apache/ignite/examples/compute/ComputeFibonacciContinuationExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/compute/ComputeFibonacciContinuationExample.java b/examples/src/main/java/org/apache/ignite/examples/compute/ComputeFibonacciContinuationExample.java
index 7309f7e..9301686 100644
--- a/examples/src/main/java/org/apache/ignite/examples/compute/ComputeFibonacciContinuationExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/compute/ComputeFibonacciContinuationExample.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.examples.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.jetbrains.annotations.*;
@@ -86,10 +87,10 @@ public final class ComputeFibonacciContinuationExample {
      */
     private static class FibonacciClosure implements IgniteClosure<Long, BigInteger> {
         /** Future for spawned task. */
-        private IgniteFuture<BigInteger> fut1;
+        private IgniteInternalFuture<BigInteger> fut1;
 
         /** Future for spawned task. */
-        private IgniteFuture<BigInteger> fut2;
+        private IgniteInternalFuture<BigInteger> fut2;
 
         /** Auto-inject job context. */
         @IgniteJobContextResource
@@ -123,7 +124,7 @@ public final class ComputeFibonacciContinuationExample {
                         return n == 0 ? BigInteger.ZERO : BigInteger.ONE;
 
                     // Node-local storage.
-                    ClusterNodeLocalMap<Long, IgniteFuture<BigInteger>> locMap = g.cluster().nodeLocalMap();
+                    ClusterNodeLocalMap<Long, IgniteInternalFuture<BigInteger>> locMap = g.cluster().nodeLocalMap();
 
                     // Check if value is cached in node-local-map first.
                     fut1 = locMap.get(n - 1);
@@ -149,8 +150,8 @@ public final class ComputeFibonacciContinuationExample {
 
                     // If futures are not done, then wait asynchronously for the result
                     if (!fut1.isDone() || !fut2.isDone()) {
-                        IgniteInClosure<IgniteFuture<BigInteger>> lsnr = new IgniteInClosure<IgniteFuture<BigInteger>>() {
-                            @Override public void apply(IgniteFuture<BigInteger> f) {
+                        IgniteInClosure<IgniteInternalFuture<BigInteger>> lsnr = new IgniteInClosure<IgniteInternalFuture<BigInteger>>() {
+                            @Override public void apply(IgniteInternalFuture<BigInteger> f) {
                                 // If both futures are done, resume the continuation.
                                 if (fut1.isDone() && fut2.isDone())
                                     // CONTINUATION:

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/examples/src/main/java/org/apache/ignite/examples/compute/ComputeRunnableExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/apache/ignite/examples/compute/ComputeRunnableExample.java b/examples/src/main/java/org/apache/ignite/examples/compute/ComputeRunnableExample.java
index c70360f..a7dfad0 100644
--- a/examples/src/main/java/org/apache/ignite/examples/compute/ComputeRunnableExample.java
+++ b/examples/src/main/java/org/apache/ignite/examples/compute/ComputeRunnableExample.java
@@ -19,6 +19,7 @@ package org.apache.ignite.examples.compute;
 
 import org.apache.ignite.*;
 import org.apache.ignite.examples.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 
 import java.util.*;
@@ -44,7 +45,7 @@ public class ComputeRunnableExample {
             System.out.println();
             System.out.println("Compute runnable example started.");
 
-            Collection<IgniteFuture> futs = new ArrayList<>();
+            Collection<IgniteInternalFuture> futs = new ArrayList<>();
 
             // Enable asynchronous mode.
             IgniteCompute compute = g.compute().withAsync();
@@ -63,7 +64,7 @@ public class ComputeRunnableExample {
             }
 
             // Wait for all futures to complete.
-            for (IgniteFuture<?> f : futs)
+            for (IgniteInternalFuture<?> f : futs)
                 f.get();
 
             System.out.println();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarContinuationExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarContinuationExample.scala b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarContinuationExample.scala
index 6ac2b91..7218490 100644
--- a/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarContinuationExample.scala
+++ b/examples/src/main/scala/org/apache/ignite/scalar/examples/ScalarContinuationExample.scala
@@ -18,7 +18,8 @@
 package org.apache.ignite.scalar.examples
 
 import org.apache.ignite.compute.ComputeJobContext
-import org.apache.ignite.lang.{IgniteClosure, IgniteFuture}
+import org.apache.ignite.internal.IgniteInternalFuture
+import org.apache.ignite.lang.IgniteClosure
 import org.apache.ignite.resources.IgniteJobContextResource
 import org.apache.ignite.scalar.scalar
 import org.apache.ignite.scalar.scalar._
@@ -81,7 +82,7 @@ class FibonacciClosure (
     // However, these fields will be preserved locally while
     // this closure is being "held", i.e. while it is suspended
     // and is waiting to be continued.
-    @transient private var fut1, fut2: IgniteFuture[BigInteger] = null
+    @transient private var fut1, fut2: IgniteInternalFuture[BigInteger] = null
 
     // Auto-inject job context.
     @IgniteJobContextResource
@@ -103,7 +104,7 @@ class FibonacciClosure (
                     BigInteger.ONE
 
             // Get properly typed node-local storage.
-            val store = g.cluster().nodeLocalMap[Long, IgniteFuture[BigInteger]]()
+            val store = g.cluster().nodeLocalMap[Long, IgniteInternalFuture[BigInteger]]()
 
             // Check if value is cached in node-local store first.
             fut1 = store.get(n - 1)
@@ -133,7 +134,7 @@ class FibonacciClosure (
 
             // If futures are not done, then wait asynchronously for the result
             if (!fut1.isDone || !fut2.isDone) {
-                val lsnr = (fut: IgniteFuture[BigInteger]) => {
+                val lsnr = (fut: IgniteInternalFuture[BigInteger]) => {
                     // This method will be called twice, once for each future.
                     // On the second call - we have to have both futures to be done
                     // - therefore we can call the continuation.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java
index 6d7d046..b5c5a85 100644
--- a/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/client/ClientAbstractMultiThreadedSelfTest.java
@@ -305,7 +305,7 @@ public abstract class ClientAbstractMultiThreadedSelfTest extends GridCommonAbst
 
             final String val = "v" + i;
 
-            IgniteFuture<?> f = multithreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> f = multithreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     l.await();
 
@@ -333,7 +333,7 @@ public abstract class ClientAbstractMultiThreadedSelfTest extends GridCommonAbst
 
         final ConcurrentLinkedQueue<String> execQueue = new ConcurrentLinkedQueue<>();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override
             public void run() {
                 long processed;
@@ -412,7 +412,7 @@ public abstract class ClientAbstractMultiThreadedSelfTest extends GridCommonAbst
 
         assertEquals(NODES_CNT, client.compute().refreshTopology(false, false).size());
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @SuppressWarnings("OverlyStrongTypeCast")
             @Override public void run() {
                 try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorStartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorStartSelfTest.java b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorStartSelfTest.java
index 871e592..efae923 100644
--- a/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorStartSelfTest.java
+++ b/modules/clients/src/test/java/org/apache/ignite/internal/processors/rest/RestProcessorStartSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.rest;
 
 import org.apache.ignite.configuration.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.client.*;
@@ -119,7 +120,7 @@ public class RestProcessorStartSelfTest extends GridCommonAbstractTest {
         try {
             gridReady.await();
 
-            IgniteFuture<GridClient> c = GridTestUtils.runAsync(new Callable<GridClient>() {
+            IgniteInternalFuture<GridClient> c = GridTestUtils.runAsync(new Callable<GridClient>() {
                 @Override public GridClient call() throws Exception {
                     return GridClientFactory.start(cfg);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
index a4b41dd..aa77747 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteDataLoader.java
@@ -18,7 +18,7 @@
 package org.apache.ignite;
 
 import org.apache.ignite.dataload.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -209,7 +209,7 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      *
      * @return Future for this loading process.
      */
-    public IgniteFuture<?> future();
+    public IgniteInternalFuture<?> future();
 
     /**
      * Optional deploy class for peer deployment. All classes loaded by a data loader
@@ -240,7 +240,7 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteFuture<?> removeData(K key)  throws IgniteCheckedException, IgniteInterruptedException, IllegalStateException;
+    public IgniteInternalFuture<?> removeData(K key)  throws IgniteCheckedException, IgniteInterruptedException, IllegalStateException;
 
     /**
      * Adds data for loading on remote node. This method can be called from multiple
@@ -258,7 +258,7 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteFuture<?> addData(K key, @Nullable V val) throws IgniteCheckedException, IgniteInterruptedException,
+    public IgniteInternalFuture<?> addData(K key, @Nullable V val) throws IgniteCheckedException, IgniteInterruptedException,
         IllegalStateException;
 
     /**
@@ -276,7 +276,7 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      * @throws IllegalStateException If grid has been concurrently stopped or
      *      {@link #close(boolean)} has already been called on loader.
      */
-    public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteCheckedException, IgniteInterruptedException,
+    public IgniteInternalFuture<?> addData(Map.Entry<K, V> entry) throws IgniteCheckedException, IgniteInterruptedException,
         IllegalStateException;
 
     /**
@@ -292,7 +292,7 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      *      {@link #close(boolean)} has already been called on loader.
      * @return Future for this load operation.
      */
-    public IgniteFuture<?> addData(Collection<? extends Map.Entry<K, V>> entries) throws IllegalStateException;
+    public IgniteInternalFuture<?> addData(Collection<? extends Map.Entry<K, V>> entries) throws IllegalStateException;
 
     /**
      * Adds data for loading on remote node. This method can be called from multiple
@@ -307,7 +307,7 @@ public interface IgniteDataLoader<K, V> extends AutoCloseable {
      *      {@link #close(boolean)} has already been called on loader.
      * @return Future for this load operation.
      */
-    public IgniteFuture<?> addData(Map<K, V> entries) throws IllegalStateException;
+    public IgniteInternalFuture<?> addData(Map<K, V> entries) throws IllegalStateException;
 
     /**
      * Loads any remaining data, but doesn't close the loader. Data can be still added after

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java b/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java
index 1f0a7c7..26fd0f3 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteScheduler.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite;
 
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.scheduler.*;
 import org.jetbrains.annotations.*;
 
@@ -57,7 +57,7 @@ public interface IgniteScheduler {
      * @see #callLocal(Callable)
      * @see org.apache.ignite.lang.IgniteClosure
      */
-    public IgniteFuture<?> runLocal(@Nullable Runnable r);
+    public IgniteInternalFuture<?> runLocal(@Nullable Runnable r);
 
     /**
      * Executes given callable on internal system thread pool asynchronously.
@@ -71,7 +71,7 @@ public interface IgniteScheduler {
      * @see #runLocal(Runnable)
      * @see org.apache.ignite.lang.IgniteOutClosure
      */
-    public <R> IgniteFuture<R> callLocal(@Nullable Callable<R> c);
+    public <R> IgniteInternalFuture<R> callLocal(@Nullable Callable<R> c);
 
     /**
      * Schedules job for execution using local <b>cron-based</b> scheduling.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
index d35f8f9..30db44c 100644
--- a/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
+++ b/modules/core/src/main/java/org/apache/ignite/IgniteSystemProperties.java
@@ -272,10 +272,10 @@ public final class IgniteSystemProperties {
     /**
      * Name of the system property or environment variable to activate synchronous
      * listener notification for future objects implemented in GridGain. I.e.
-     * closure passed into method {@link org.apache.ignite.lang.IgniteFuture#listenAsync(org.apache.ignite.lang.IgniteInClosure)} will
+     * closure passed into method {@link org.apache.ignite.internal.IgniteInternalFuture#listenAsync(org.apache.ignite.lang.IgniteInClosure)} will
      * be evaluated in the same thread that will end the future.
      *
-     * @see org.apache.ignite.lang.IgniteFuture#syncNotify()
+     * @see org.apache.ignite.internal.IgniteInternalFuture#syncNotify()
      */
     public static final String GG_FUT_SYNC_NOTIFICATION = "GRIDGAIN_FUTURE_SYNC_NOTIFICATION";
 
@@ -285,7 +285,7 @@ public final class IgniteSystemProperties {
      * upon future completion every listener will be notified concurrently in a
      * separate thread.
      *
-     * @see org.apache.ignite.lang.IgniteFuture#concurrentNotify()
+     * @see org.apache.ignite.internal.IgniteInternalFuture#concurrentNotify()
      */
     public static final String GG_FUT_CONCURRENT_NOTIFICATION = "GRIDGAIN_FUTURE_CONCURRENT_NOTIFICATION";
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java
index 12ff0f4..4211b9b 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheEntry.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.jetbrains.annotations.*;
@@ -128,7 +129,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      *
      * @return See {@link CacheProjection#reloadAsync(Object)}.
      */
-    public IgniteFuture<V> reloadAsync();
+    public IgniteInternalFuture<V> reloadAsync();
 
     /**
      * This method has the same semantic as
@@ -228,7 +229,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      *
      * @return See {@link CacheProjection#getAsync(Object)}.
      */
-    public IgniteFuture<V> getAsync();
+    public IgniteInternalFuture<V> getAsync();
 
     /**
      * This method has the same semantic as {@link #set(Object, org.apache.ignite.lang.IgnitePredicate[])} method, however it
@@ -258,7 +259,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param filter See {@link CacheProjection#putAsync(Object, Object, org.apache.ignite.lang.IgnitePredicate[])}.
      * @return See {@link CacheProjection#putAsync(Object, Object, org.apache.ignite.lang.IgnitePredicate[])}.
      */
-    public IgniteFuture<V> setAsync(V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
+    public IgniteInternalFuture<V> setAsync(V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
      * This method has the same semantic as
@@ -277,7 +278,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param val See {@link CacheProjection#putIfAbsentAsync(Object, Object)}
      * @return See {@link CacheProjection#putIfAbsentAsync(Object, Object)}.
      */
-    public IgniteFuture<V> setIfAbsentAsync(V val);
+    public IgniteInternalFuture<V> setIfAbsentAsync(V val);
 
     /**
      * This method has the same semantic as
@@ -299,7 +300,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param filter See {@link CacheProjection#putxAsync(Object, Object, org.apache.ignite.lang.IgnitePredicate[])}.
      * @return See {@link CacheProjection#putxAsync(Object, Object, org.apache.ignite.lang.IgnitePredicate[])}.
      */
-    public IgniteFuture<Boolean> setxAsync(V val,
+    public IgniteInternalFuture<Boolean> setxAsync(V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
@@ -319,7 +320,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param val See {@link CacheProjection#putxIfAbsentAsync(Object, Object)}
      * @return See {@link CacheProjection#putxIfAbsentAsync(Object, Object)}.
      */
-    public IgniteFuture<Boolean> setxIfAbsentAsync(V val);
+    public IgniteInternalFuture<Boolean> setxIfAbsentAsync(V val);
 
     /**
      * This method has the same semantic as
@@ -338,7 +339,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param val See {@link CacheProjection#replaceAsync(Object, Object)}
      * @return See {@link CacheProjection#replaceAsync(Object, Object)}.
      */
-    public IgniteFuture<V> replaceAsync(V val);
+    public IgniteInternalFuture<V> replaceAsync(V val);
 
     /**
      * This method has the same semantic as
@@ -357,7 +358,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param val See {@link CacheProjection#replacexAsync(Object, Object)}
      * @return See {@link CacheProjection#replacexAsync(Object, Object)}.
      */
-    public IgniteFuture<Boolean> replacexAsync(V val);
+    public IgniteInternalFuture<Boolean> replacexAsync(V val);
 
     /**
      * This method has the same semantic as
@@ -378,7 +379,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param newVal See {@link CacheProjection#replaceAsync(Object, Object, Object)}
      * @return See {@link CacheProjection#replaceAsync(Object, Object)}.
      */
-    public IgniteFuture<Boolean> replaceAsync(V oldVal, V newVal);
+    public IgniteInternalFuture<Boolean> replaceAsync(V oldVal, V newVal);
 
     /**
      * This method has the same semantic as
@@ -397,7 +398,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param filter See {@link CacheProjection#removeAsync(Object, org.apache.ignite.lang.IgnitePredicate[])}.
      * @return See {@link CacheProjection#removeAsync(Object, org.apache.ignite.lang.IgnitePredicate[])}.
      */
-    public IgniteFuture<V> removeAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
+    public IgniteInternalFuture<V> removeAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
      * This method has the same semantic as
@@ -416,7 +417,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param filter See {@link CacheProjection#removexAsync(Object, org.apache.ignite.lang.IgnitePredicate[])}.
      * @return See {@link CacheProjection#removexAsync(Object, org.apache.ignite.lang.IgnitePredicate[])}.
      */
-    public IgniteFuture<Boolean> removexAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
+    public IgniteInternalFuture<Boolean> removexAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
      * This method has the same semantic as
@@ -435,7 +436,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      * @param val See {@link CacheProjection#removeAsync(Object, Object)}.
      * @return See {@link CacheProjection#removeAsync(Object, Object)}.
      */
-    public IgniteFuture<Boolean> removeAsync(V val);
+    public IgniteInternalFuture<Boolean> removeAsync(V val);
 
     /**
      * This method has the same semantic as
@@ -507,7 +508,7 @@ public interface CacheEntry<K, V> extends Map.Entry<K, V>, javax.cache.Cache.Ent
      *      {@code false} otherwise.
      * @throws org.apache.ignite.internal.processors.cache.CacheFlagException If flags validation failed.
      */
-    public IgniteFuture<Boolean> lockAsync(long timeout,
+    public IgniteInternalFuture<Boolean> lockAsync(long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java b/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java
index 4d61d1d..2457fcd 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/CacheProjection.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cache.store.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
@@ -347,7 +348,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @param key Key.
      * @return Future.
      */
-    public IgniteFuture<Boolean> containsKeyAsync(K key);
+    public IgniteInternalFuture<Boolean> containsKeyAsync(K key);
 
     /**
      * Returns {@code true} if this cache contains given value.
@@ -407,7 +408,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @param key Key to reload.
      * @return Future to be completed whenever the entry is reloaded.
      */
-    public IgniteFuture<V> reloadAsync(K key);
+    public IgniteInternalFuture<V> reloadAsync(K key);
 
     /**
      * Reloads all currently cached keys form persistent storage.
@@ -428,7 +429,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      *
      * @return Future which will complete whenever {@code reload} completes.
      */
-    public IgniteFuture<?> reloadAllAsync();
+    public IgniteInternalFuture<?> reloadAllAsync();
 
     /**
      * Reloads specified entries from underlying persistent storage.
@@ -451,7 +452,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @param keys Keys to reload.
      * @return Future which will complete whenever {@code reload} completes.
      */
-    public IgniteFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys);
+    public IgniteInternalFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys);
 
     /**
      * Peeks at in-memory cached value using default {@link GridCachePeekMode#SMART}
@@ -536,7 +537,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException if the key is {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<V> getAsync(K key);
+    public IgniteInternalFuture<V> getAsync(K key);
 
     /**
      * Retrieves values mapped to the specified keys from cache. Value will only be returned if
@@ -583,7 +584,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @return Future for the get operation.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys);
+    public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys);
 
     /**
      * Stores given key-value pair in cache. If filters are provided, then entries will
@@ -655,7 +656,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException If either key or value are {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<V> putAsync(K key, V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
+    public IgniteInternalFuture<V> putAsync(K key, V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
      * Stores given key-value pair in cache. If filters are provided, then entries will
@@ -719,7 +720,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException If either key or value are {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<Boolean> putxAsync(K key, V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
+    public IgniteInternalFuture<Boolean> putxAsync(K key, V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
      * Stores given key-value pair in cache only if cache had no previous mapping for it. If cache
@@ -781,7 +782,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException If either key or value are {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<V> putIfAbsentAsync(K key, V val);
+    public IgniteInternalFuture<V> putIfAbsentAsync(K key, V val);
 
     /**
      * Stores given key-value pair in cache only if cache had no previous mapping for it.
@@ -832,7 +833,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException If either key or value are {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<Boolean> putxIfAbsentAsync(K key, V val);
+    public IgniteInternalFuture<Boolean> putxIfAbsentAsync(K key, V val);
 
     /**
      * Stores given key-value pair in cache only if there is a previous mapping for it.
@@ -891,7 +892,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException If either key or value are {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<V> replaceAsync(K key, V val);
+    public IgniteInternalFuture<V> replaceAsync(K key, V val);
 
     /**
      * Stores given key-value pair in cache only if only if there is a previous mapping for it.
@@ -942,7 +943,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException If either key or value are {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<Boolean> replacexAsync(K key, V val);
+    public IgniteInternalFuture<Boolean> replacexAsync(K key, V val);
 
     /**
      * Stores given key-value pair in cache only if only if the previous value is equal to the
@@ -991,7 +992,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException If either key or value are {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal);
+    public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal);
 
     /**
      * Stores given key-value pairs in cache. If filters are provided, then entries will
@@ -1036,7 +1037,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @return Future for putAll operation.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<?> putAllAsync(@Nullable Map<? extends K, ? extends V> m,
+    public IgniteInternalFuture<?> putAllAsync(@Nullable Map<? extends K, ? extends V> m,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
@@ -1465,7 +1466,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException if the key is {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<V> removeAsync(K key, IgnitePredicate<CacheEntry<K, V>>... filter);
+    public IgniteInternalFuture<V> removeAsync(K key, IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
      * Removes given key mapping from cache.
@@ -1518,7 +1519,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException if the key is {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<Boolean> removexAsync(K key,
+    public IgniteInternalFuture<Boolean> removexAsync(K key,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
@@ -1565,7 +1566,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      * @throws NullPointerException if the key or value is {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<Boolean> removeAsync(K key, V val);
+    public IgniteInternalFuture<Boolean> removeAsync(K key, V val);
 
     /**
      * Removes given key mappings from cache for entries for which the optionally passed in filters do
@@ -1609,7 +1610,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      *      remove operation completes.
      * @throws CacheFlagException If flags validation failed.
      */
-    public IgniteFuture<?> removeAllAsync(@Nullable Collection<? extends K> keys,
+    public IgniteInternalFuture<?> removeAllAsync(@Nullable Collection<? extends K> keys,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
@@ -1662,7 +1663,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      *      remove operation completes.
      * @throws CacheFlagException If flags validation failed.
      */
-    public IgniteFuture<?> removeAllAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
+    public IgniteInternalFuture<?> removeAllAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
      * Synchronously acquires lock on a cached object with given
@@ -1713,7 +1714,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      *      {@code false} otherwise.
      * @throws CacheFlagException If flags validation failed.
      */
-    public IgniteFuture<Boolean> lockAsync(K key, long timeout,
+    public IgniteInternalFuture<Boolean> lockAsync(K key, long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
@@ -1765,7 +1766,7 @@ public interface CacheProjection<K, V> extends Iterable<CacheEntry<K, V>> {
      *      timeout has expired, {@code false} otherwise.
      * @throws CacheFlagException If flags validation failed.
      */
-    public IgniteFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys, long timeout,
+    public IgniteInternalFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys, long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java b/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
index fdddd9f..640a70c 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/GridCache.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.datastructures.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.mxbean.*;
 import org.apache.ignite.transactions.*;
@@ -239,7 +240,7 @@ public interface GridCache<K, V> extends CacheProjection<K, V> {
      *      {@link CacheStore#loadCache(org.apache.ignite.lang.IgniteBiInClosure,Object...)} method.
      * @return Future to be completed whenever loading completes.
      */
-    public IgniteFuture<?> loadCacheAsync(@Nullable IgniteBiPredicate<K, V> p, long ttl, @Nullable Object... args);
+    public IgniteInternalFuture<?> loadCacheAsync(@Nullable IgniteBiPredicate<K, V> p, long ttl, @Nullable Object... args);
 
     /**
      * Gets a random entry out of cache. In the worst cache scenario this method
@@ -277,5 +278,5 @@ public interface GridCache<K, V> extends CacheProjection<K, V> {
      * <p>
      * @return Future that will be completed when preloading is finished.
      */
-    public IgniteFuture<?> forceRepartition();
+    public IgniteInternalFuture<?> forceRepartition();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueries.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueries.java b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueries.java
index 1cc59bb..f14f869 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueries.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueries.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.cache.query;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
@@ -111,7 +112,7 @@ public interface CacheQueries<K, V> {
      *
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
-    public IgniteFuture<?> rebuildIndexes(Class<?> cls);
+    public IgniteInternalFuture<?> rebuildIndexes(Class<?> cls);
 
     /**
      * Forces this cache to rebuild all search indexes of given value type. Sometimes indexes
@@ -124,7 +125,7 @@ public interface CacheQueries<K, V> {
      *
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
-    public IgniteFuture<?> rebuildIndexes(String typeName);
+    public IgniteInternalFuture<?> rebuildIndexes(String typeName);
 
     /**
      * Forces this cache to rebuild search indexes of all types. Sometimes indexes
@@ -135,7 +136,7 @@ public interface CacheQueries<K, V> {
      *
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
-    public IgniteFuture<?> rebuildAllIndexes();
+    public IgniteInternalFuture<?> rebuildAllIndexes();
 
     /**
      * Accumulated metrics for all queries executed for this cache.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryFuture.java b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryFuture.java
index b0c930c..6f68f29 100644
--- a/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/cache/query/CacheQueryFuture.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.cache.query;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -27,7 +27,7 @@ import java.util.*;
  * Cache query future returned by query execution.
  * Refer to {@link CacheQuery} documentation for more information.
  */
-public interface CacheQueryFuture<T> extends IgniteFuture<Collection<T>> {
+public interface CacheQueryFuture<T> extends IgniteInternalFuture<Collection<T>> {
     /**
      * Returns number of elements that are already fetched and can
      * be returned from {@link #next()} method without blocking.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
index 6bb2325..8476fc6 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskFuture.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.compute;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 
 import java.util.*;
 import java.util.concurrent.*;
@@ -29,7 +29,7 @@ import java.util.concurrent.*;
  * hierarchy.
  * @param <R> Type of the task result returning from {@link ComputeTask#reduce(List)} method.
  */
-public interface ComputeTaskFuture<R> extends IgniteFuture<R> {
+public interface ComputeTaskFuture<R> extends IgniteInternalFuture<R> {
     /**
      * {@inheritDoc}
      *

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSession.java b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSession.java
index cae75a7..c17eee9 100644
--- a/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSession.java
+++ b/modules/core/src/main/java/org/apache/ignite/compute/ComputeTaskSession.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.compute;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
@@ -439,5 +440,5 @@ public interface ComputeTaskSession {
      *
      * @return Future that will be completed when task "<tt>map</tt>" step has completed.
      */
-    public IgniteFuture<?> mapFuture();
+    public IgniteInternalFuture<?> mapFuture();
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoop.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoop.java b/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoop.java
index 64d2dec..bab5496 100644
--- a/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoop.java
+++ b/modules/core/src/main/java/org/apache/ignite/hadoop/GridHadoop.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.hadoop;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -46,7 +46,7 @@ public interface GridHadoop {
      * @param jobInfo Job info to submit.
      * @return Execution future.
      */
-    public IgniteFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo);
+    public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo);
 
     /**
      * Gets Hadoop job execution status.
@@ -73,7 +73,7 @@ public interface GridHadoop {
      * @return Job finish future or {@code null} in case job with the given ID is not found.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgniteFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException;
+    @Nullable public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException;
 
     /**
      * Kills job.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/GridEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridEx.java b/modules/core/src/main/java/org/apache/ignite/internal/GridEx.java
index 1eea09a..6e004fc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridEx.java
@@ -124,7 +124,7 @@ public interface GridEx extends Ignite, ClusterGroupEx, IgniteCluster {
     /**
      * Schedule sending of given email to all configured admin emails.
      */
-    IgniteFuture<Boolean> sendAdminEmailAsync(String subj, String body, boolean html);
+    IgniteInternalFuture<Boolean> sendAdminEmailAsync(String subj, String body, boolean html);
 
     /**
      * Get GGFS instance returning null if it doesn't exist.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/GridJobSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSessionImpl.java
index 3b261d0..abec4f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridJobSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridJobSessionImpl.java
@@ -277,7 +277,7 @@ public class GridJobSessionImpl implements GridTaskSessionInternal {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> mapFuture() {
+    @Override public IgniteInternalFuture<?> mapFuture() {
         return new GridFinishedFuture<>(ctx);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java b/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
index 35484a8..8ffafac 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridKernal.java
@@ -2534,7 +2534,7 @@ public class GridKernal extends ClusterGroupAdapter implements GridEx, IgniteMXB
      * @see #isSmtpEnabled()
      * @see org.apache.ignite.configuration.IgniteConfiguration#getAdminEmails()
      */
-    @Override public IgniteFuture<Boolean> sendAdminEmailAsync(String subj, String body, boolean html) {
+    @Override public IgniteInternalFuture<Boolean> sendAdminEmailAsync(String subj, String body, boolean html) {
         A.notNull(subj, "subj");
         A.notNull(body, "body");
 
@@ -2626,7 +2626,7 @@ public class GridKernal extends ClusterGroupAdapter implements GridEx, IgniteMXB
      * @throws IgniteCheckedException In case of error.
      * @see {@link org.apache.ignite.IgniteCluster#startNodes(java.io.File, boolean, int, int)}.
      */
-    IgniteFuture<Collection<GridTuple3<String, Boolean, String>>> startNodesAsync(File file, boolean restart,                                                                                            int timeout, int maxConn) throws IgniteCheckedException {
+    IgniteInternalFuture<Collection<GridTuple3<String, Boolean, String>>> startNodesAsync(File file, boolean restart,                                                                                            int timeout, int maxConn) throws IgniteCheckedException {
         A.notNull(file, "file");
         A.ensure(file.exists(), "file doesn't exist.");
         A.ensure(file.isFile(), "file is a directory.");
@@ -2647,7 +2647,7 @@ public class GridKernal extends ClusterGroupAdapter implements GridEx, IgniteMXB
     }
 
     /** {@inheritDoc} */
-    @Override public <R> IgniteFuture<R> future() {
+    @Override public <R> IgniteInternalFuture<R> future() {
         throw new IllegalStateException("Asynchronous mode is not enabled.");
     }
 
@@ -2668,7 +2668,7 @@ public class GridKernal extends ClusterGroupAdapter implements GridEx, IgniteMXB
      * @throws IgniteCheckedException In case of error.
      * @see {@link org.apache.ignite.IgniteCluster#startNodes(java.util.Collection, java.util.Map, boolean, int, int)}.
      */
-    IgniteFuture<Collection<GridTuple3<String, Boolean, String>>> startNodesAsync(
+    IgniteInternalFuture<Collection<GridTuple3<String, Boolean, String>>> startNodesAsync(
         Collection<Map<String, Object>> hosts, @Nullable Map<String, Object> dflts, boolean restart, int timeout,
         int maxConn) throws IgniteCheckedException {
         A.notNull(hosts, "hosts");
@@ -2809,15 +2809,15 @@ public class GridKernal extends ClusterGroupAdapter implements GridEx, IgniteMXB
         if (call == null)
             return false;
 
-        IgniteFuture<GridTuple3<String, Boolean, String>> fut = ctx.closure().callLocalSafe(call, true);
+        IgniteInternalFuture<GridTuple3<String, Boolean, String>> fut = ctx.closure().callLocalSafe(call, true);
 
         comp.add(fut);
 
         if (cnt.decrementAndGet() == 0)
             comp.markInitialized();
 
-        fut.listenAsync(new CI1<IgniteFuture<GridTuple3<String, Boolean, String>>>() {
-            @Override public void apply(IgniteFuture<GridTuple3<String, Boolean, String>> f) {
+        fut.listenAsync(new CI1<IgniteInternalFuture<GridTuple3<String, Boolean, String>>>() {
+            @Override public void apply(IgniteInternalFuture<GridTuple3<String, Boolean, String>> f) {
                 runNextNodeCallable(queue, comp, cnt);
             }
         });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionImpl.java
index f5f0b33..e11c7d0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/GridTaskSessionImpl.java
@@ -828,7 +828,7 @@ public class GridTaskSessionImpl implements GridTaskSessionInternal {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> mapFuture() {
+    @Override public IgniteInternalFuture<?> mapFuture() {
         return mapFut;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
new file mode 100644
index 0000000..cc376fb
--- /dev/null
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteInternalFuture.java
@@ -0,0 +1,190 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.lang.*;
+import org.jetbrains.annotations.*;
+
+import java.util.concurrent.*;
+
+/**
+ * Extension for standard {@link Future} interface. It adds simplified exception handling,
+ * functional programming support and ability to listen for future completion via functional
+ * callback.
+ * @param <R> Type of the result for the future.
+ */
+public interface IgniteInternalFuture<R> {
+    /**
+     * Synchronously waits for completion of the computation and
+     * returns computation result.
+     *
+     * @return Computation result.
+     * @throws IgniteInterruptedException Subclass of {@link IgniteCheckedException} thrown if the wait was interrupted.
+     * @throws IgniteFutureCancelledException Subclass of {@link IgniteCheckedException} throws if computation was cancelled.
+     * @throws IgniteCheckedException If computation failed.
+     */
+    public R get() throws IgniteCheckedException;
+
+    /**
+     * Synchronously waits for completion of the computation for
+     * up to the timeout specified and returns computation result.
+     * This method is equivalent to calling {@link #get(long, TimeUnit) get(long, TimeUnit.MILLISECONDS)}.
+     *
+     * @param timeout The maximum time to wait in milliseconds.
+     * @return Computation result.
+     * @throws IgniteInterruptedException Subclass of {@link IgniteCheckedException} thrown if the wait was interrupted.
+     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException Subclass of {@link IgniteCheckedException} thrown if the wait was timed out.
+     * @throws IgniteFutureCancelledException Subclass of {@link IgniteCheckedException} throws if computation was cancelled.
+     * @throws IgniteCheckedException If computation failed.
+     */
+    public R get(long timeout) throws IgniteCheckedException;
+
+    /**
+     * Synchronously waits for completion of the computation for
+     * up to the timeout specified and returns computation result.
+     *
+     * @param timeout The maximum time to wait.
+     * @param unit The time unit of the {@code timeout} argument.
+     * @return Computation result.
+     * @throws IgniteInterruptedException Subclass of {@link IgniteCheckedException} thrown if the wait was interrupted.
+     * @throws org.apache.ignite.lang.IgniteFutureTimeoutException Subclass of {@link IgniteCheckedException} thrown if the wait was timed out.
+     * @throws IgniteFutureCancelledException Subclass of {@link IgniteCheckedException} throws if computation was cancelled.
+     * @throws IgniteCheckedException If computation failed.
+     */
+    public R get(long timeout, TimeUnit unit) throws IgniteCheckedException;
+
+    /**
+     * Cancels this future.
+     *
+     * @return {@code True} if future was canceled (i.e. was not finished prior to this call).
+     * @throws IgniteCheckedException If cancellation failed.
+     */
+    public boolean cancel() throws IgniteCheckedException;
+
+    /**
+     * Checks if computation is done.
+     *
+     * @return {@code True} if computation is done, {@code false} otherwise.
+     */
+    public boolean isDone();
+
+    /**
+     * Returns {@code true} if this computation was cancelled before it completed normally.
+     *
+     * @return {@code True} if this computation was cancelled before it completed normally.
+     */
+    public boolean isCancelled();
+
+    /**
+     * Gets start time for this future.
+     *
+     * @return Start time for this future.
+     */
+    public long startTime();
+
+    /**
+     * Gets duration in milliseconds between start of the future and current time if future
+     * is not finished, or between start and finish of this future.
+     *
+     * @return Time in milliseconds this future has taken to execute.
+     */
+    public long duration();
+
+    /**
+     * Flag to turn on or off synchronous listener notification. If this flag is {@code true}, then
+     * upon future completion the notification may happen in the same thread that created
+     * the future. This becomes especially important when adding listener to a future that
+     * is already {@code done} - if this flag is {@code true}, then listener will be
+     * immediately notified within the same thread.
+     * <p>
+     * Default value is {@code false}. To change the default, set
+     * {@link IgniteSystemProperties#GG_FUT_SYNC_NOTIFICATION} system property to {@code true}.
+     *
+     * @param syncNotify Flag to turn on or off synchronous listener notification.
+     */
+    public void syncNotify(boolean syncNotify);
+
+    /**
+     * Gets value of synchronous listener notification flag. If this flag is {@code true}, then
+     * upon future completion the notification may happen in the same thread that created
+     * the future. This becomes especially important when adding listener to a future that
+     * is already {@code done} - if this flag is {@code true}, then listener will be
+     * immediately notified within the same thread.
+     * <p>
+     * Default value is {@code false}. To change the default, set
+     * {@link IgniteSystemProperties#GG_FUT_SYNC_NOTIFICATION} system property to {@code true}.
+     *
+     * @return Synchronous listener notification flag.
+     */
+    public boolean syncNotify();
+
+    /**
+     * Flag to turn on or off concurrent listener notification. This flag comes into play only
+     * when a future has more than one listener subscribed to it. If this flag is {@code true},
+     * then all listeners will be notified concurrently by different threads; otherwise,
+     * listeners will be notified one after another within one thread (depending on
+     * {@link #syncNotify()} flag, these notifications may happen either in the same thread which
+     * started the future, or in a different thread).
+     * <p>
+     * Default value is {@code false}. To change the default, set
+     * {@link IgniteSystemProperties#GG_FUT_CONCURRENT_NOTIFICATION} system property to {@code true}.
+     *
+     * @param concurNotify Flag to turn on or off concurrent listener notification.
+     */
+    public void concurrentNotify(boolean concurNotify);
+
+    /**
+     * Gets value concurrent listener notification flag. This flag comes into play only
+     * when a future has more than one listener subscribed to it. If this flag is {@code true},
+     * then all listeners will be notified concurrently by different threads; otherwise,
+     * listeners will be notified one after another within one thread (depending on
+     * {@link #syncNotify()} flag, these notifications may happen either in the same thread which
+     * started the future, or in a different thread).
+     * <p>
+     * Default value is {@code false}. To change the default, set
+     * {@link IgniteSystemProperties#GG_FUT_CONCURRENT_NOTIFICATION} system property to {@code true}.
+     *
+     * @return Concurrent listener notification flag
+     */
+    public boolean concurrentNotify();
+
+    /**
+     * Registers listener closure to be asynchronously notified whenever future completes.
+     *
+     * @param lsnr Listener closure to register. If not provided - this method is no-op.
+     */
+    public void listenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<R>> lsnr);
+
+    /**
+     * Removes given listeners from the future. If no listener is passed in, then all listeners
+     * will be removed.
+     *
+     * @param lsnr Listeners to remove.
+     */
+    public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<R>>... lsnr);
+
+    /**
+     * Make a chained future to convert result of this future (when complete) into a new format.
+     * It is guaranteed that done callback will be called only ONCE.
+     *
+     * @param doneCb Done callback that is applied to this future when it finishes to produce chained future result.
+     * @return Chained future that finishes after this future completes and done callback is called.
+     */
+    public <T> IgniteInternalFuture<T> chain(IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb);
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/IgniteSchedulerImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/IgniteSchedulerImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/IgniteSchedulerImpl.java
index 24a3be0..2a7e1c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/IgniteSchedulerImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/IgniteSchedulerImpl.java
@@ -18,8 +18,6 @@
 package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.scheduler.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
@@ -51,7 +49,7 @@ public class IgniteSchedulerImpl implements IgniteScheduler, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> runLocal(Runnable r) {
+    @Override public IgniteInternalFuture<?> runLocal(Runnable r) {
         A.notNull(r, "r");
 
         guard();
@@ -65,7 +63,7 @@ public class IgniteSchedulerImpl implements IgniteScheduler, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public <R> IgniteFuture<R> callLocal(Callable<R> c) {
+    @Override public <R> IgniteInternalFuture<R> callLocal(Callable<R> c) {
         A.notNull(c, "c");
 
         guard();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java b/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
index 0f2b6eb..019fd59 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/executor/GridExecutorService.java
@@ -87,7 +87,7 @@ public class GridExecutorService implements ExecutorService, Externalizable {
     private boolean isBeingShutdown;
 
     /** List of executing or scheduled for execution tasks. */
-    private List<IgniteFuture<?>> futs = new ArrayList<>();
+    private List<IgniteInternalFuture<?>> futs = new ArrayList<>();
 
     /** Rejected or completed tasks listener. */
     private TaskTerminateListener lsnr = new TaskTerminateListener<>();
@@ -151,7 +151,7 @@ public class GridExecutorService implements ExecutorService, Externalizable {
 
     /** {@inheritDoc} */
     @Override public List<Runnable> shutdownNow() {
-        List<IgniteFuture<?>> cpFuts;
+        List<IgniteInternalFuture<?>> cpFuts;
 
         // Cancel all tasks.
         synchronized (mux) {
@@ -160,7 +160,7 @@ public class GridExecutorService implements ExecutorService, Externalizable {
             isBeingShutdown = true;
         }
 
-        for (IgniteFuture<?> task : cpFuts) {
+        for (IgniteInternalFuture<?> task : cpFuts) {
             try {
                 task.cancel();
             }
@@ -198,17 +198,17 @@ public class GridExecutorService implements ExecutorService, Externalizable {
         if (end < 0)
             end = Long.MAX_VALUE;
 
-        List<IgniteFuture<?>> locTasks;
+        List<IgniteInternalFuture<?>> locTasks;
 
         // Cancel all tasks.
         synchronized (mux) {
             locTasks = new ArrayList<>(futs);
         }
 
-        Iterator<IgniteFuture<?>> iter = locTasks.iterator();
+        Iterator<IgniteInternalFuture<?>> iter = locTasks.iterator();
 
         while (iter.hasNext() && now < end) {
-            IgniteFuture<?> fut = iter.next();
+            IgniteInternalFuture<?> fut = iter.next();
 
             try {
                 fut.get(end - now);
@@ -261,8 +261,8 @@ public class GridExecutorService implements ExecutorService, Externalizable {
         try {
             comp.run(task);
 
-            IgniteFuture<T> fut = comp.future().chain(new CX1<IgniteFuture<?>, T>() {
-                @Override public T applyx(IgniteFuture<?> fut) throws IgniteCheckedException {
+            IgniteInternalFuture<T> fut = comp.future().chain(new CX1<IgniteInternalFuture<?>, T>() {
+                @Override public T applyx(IgniteInternalFuture<?> fut) throws IgniteCheckedException {
                     fut.get();
 
                     return res;
@@ -345,14 +345,14 @@ public class GridExecutorService implements ExecutorService, Externalizable {
 
         checkShutdown();
 
-        Collection<IgniteFuture<T>> taskFuts = new ArrayList<>();
+        Collection<IgniteInternalFuture<T>> taskFuts = new ArrayList<>();
 
         assert comp.isAsync();
 
         for (Callable<T> task : tasks) {
             // Execute task without predefined timeout.
             // GridFuture.cancel() will be called if timeout elapsed.
-            IgniteFuture<T> fut;
+            IgniteInternalFuture<T> fut;
 
             try {
                 comp.call(task);
@@ -371,7 +371,7 @@ public class GridExecutorService implements ExecutorService, Externalizable {
 
         boolean isInterrupted = false;
 
-        for (IgniteFuture<T> fut : taskFuts) {
+        for (IgniteInternalFuture<T> fut : taskFuts) {
             if (!isInterrupted && now < end) {
                 try {
                     fut.get(end - now);
@@ -403,7 +403,7 @@ public class GridExecutorService implements ExecutorService, Externalizable {
         List<Future<T>> futs = new ArrayList<>(taskFuts.size());
 
         // Convert futures.
-        for (IgniteFuture<T> fut : taskFuts) {
+        for (IgniteInternalFuture<T> fut : taskFuts) {
             // Per executor service contract any task that was not completed
             // should be cancelled upon return.
             if (!fut.isDone())
@@ -420,7 +420,7 @@ public class GridExecutorService implements ExecutorService, Externalizable {
      *
      * @param fut Future to cancel.
      */
-    private void cancelFuture(IgniteFuture<?> fut) {
+    private void cancelFuture(IgniteInternalFuture<?> fut) {
         try {
             fut.cancel();
         }
@@ -485,13 +485,13 @@ public class GridExecutorService implements ExecutorService, Externalizable {
 
         checkShutdown();
 
-        Collection<IgniteFuture<T>> taskFuts = new ArrayList<>();
+        Collection<IgniteInternalFuture<T>> taskFuts = new ArrayList<>();
 
         assert comp.isAsync();
 
         for (Callable<T> cmd : tasks) {
             // Execute task with predefined timeout.
-            IgniteFuture<T> fut;
+            IgniteInternalFuture<T> fut;
 
             try
             {
@@ -514,7 +514,7 @@ public class GridExecutorService implements ExecutorService, Externalizable {
 
         int errCnt = 0;
 
-        for (IgniteFuture<T> fut : taskFuts) {
+        for (IgniteInternalFuture<T> fut : taskFuts) {
             now = U.currentTimeMillis();
 
             boolean cancel = false;
@@ -601,7 +601,7 @@ public class GridExecutorService implements ExecutorService, Externalizable {
      * @return Future for command.
      */
     @SuppressWarnings("unchecked")
-    private <T> Future<T> addFuture(IgniteFuture<T> fut) {
+    private <T> Future<T> addFuture(IgniteInternalFuture<T> fut) {
         synchronized (mux) {
             if (!fut.isDone()) {
                 fut.listenAsync(lsnr);
@@ -616,12 +616,12 @@ public class GridExecutorService implements ExecutorService, Externalizable {
     /**
      * Listener to track tasks.
      */
-    private class TaskTerminateListener<T> implements IgniteInClosure<IgniteFuture<T>> {
+    private class TaskTerminateListener<T> implements IgniteInClosure<IgniteInternalFuture<T>> {
         /** */
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
-        @Override public void apply(IgniteFuture<T> taskFut) {
+        @Override public void apply(IgniteInternalFuture<T> taskFut) {
             synchronized (mux) {
                 futs.remove(taskFut);
             }
@@ -629,20 +629,20 @@ public class GridExecutorService implements ExecutorService, Externalizable {
     }
 
     /**
-     * Wrapper for {@link org.apache.ignite.lang.IgniteFuture}.
+     * Wrapper for {@link org.apache.ignite.internal.IgniteInternalFuture}.
      * Used for compatibility {@link Future} interface.
      * @param <T> The result type of the {@link Future} argument.
      */
     private class TaskFutureWrapper<T> implements Future<T> {
         /** */
-        private final IgniteFuture<T> fut;
+        private final IgniteInternalFuture<T> fut;
 
         /**
          * Creates wrapper.
          *
          * @param fut Grid future.
          */
-        TaskFutureWrapper(IgniteFuture<T> fut) {
+        TaskFutureWrapper(IgniteInternalFuture<T> fut) {
             assert fut != null;
 
             this.fut = fut;


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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java
index fd9ab76..3e8abd8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/dataload/IgniteDataLoaderImpl.java
@@ -114,12 +114,12 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
     /** Active futures of this data loader. */
     @GridToStringInclude
-    private final Collection<IgniteFuture<?>> activeFuts = new GridConcurrentHashSet<>();
+    private final Collection<IgniteInternalFuture<?>> activeFuts = new GridConcurrentHashSet<>();
 
     /** Closure to remove from active futures. */
     @GridToStringExclude
-    private final IgniteInClosure<IgniteFuture<?>> rmvActiveFut = new IgniteInClosure<IgniteFuture<?>>() {
-        @Override public void apply(IgniteFuture<?> t) {
+    private final IgniteInClosure<IgniteInternalFuture<?>> rmvActiveFut = new IgniteInClosure<IgniteInternalFuture<?>>() {
+        @Override public void apply(IgniteInternalFuture<?> t) {
             boolean rmv = activeFuts.remove(t);
 
             assert rmv;
@@ -257,7 +257,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> future() {
+    @Override public IgniteInternalFuture<?> future() {
         return fut;
     }
 
@@ -356,14 +356,14 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> addData(Map<K, V> entries) throws IllegalStateException {
+    @Override public IgniteInternalFuture<?> addData(Map<K, V> entries) throws IllegalStateException {
         A.notNull(entries, "entries");
 
         return addData(entries.entrySet());
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> addData(Collection<? extends Map.Entry<K, V>> entries) {
+    @Override public IgniteInternalFuture<?> addData(Collection<? extends Map.Entry<K, V>> entries) {
         A.notEmpty(entries, "entries");
 
         enterBusy();
@@ -397,21 +397,21 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteCheckedException, IllegalStateException {
+    @Override public IgniteInternalFuture<?> addData(Map.Entry<K, V> entry) throws IgniteCheckedException, IllegalStateException {
         A.notNull(entry, "entry");
 
         return addData(F.asList(entry));
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> addData(K key, V val) throws IgniteCheckedException, IllegalStateException {
+    @Override public IgniteInternalFuture<?> addData(K key, V val) throws IgniteCheckedException, IllegalStateException {
         A.notNull(key, "key");
 
         return addData(new Entry0<>(key, val));
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeData(K key) throws IgniteCheckedException, IllegalStateException {
+    @Override public IgniteInternalFuture<?> removeData(K key) throws IgniteCheckedException, IllegalStateException {
         return addData(key, null);
     }
 
@@ -491,8 +491,8 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
             final Collection<Map.Entry<K, V>> entriesForNode = e.getValue();
 
-            IgniteInClosure<IgniteFuture<?>> lsnr = new IgniteInClosure<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> t) {
+            IgniteInClosure<IgniteInternalFuture<?>> lsnr = new IgniteInClosure<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> t) {
                     try {
                         t.get();
 
@@ -555,11 +555,11 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     private void doFlush() throws IgniteCheckedException {
         lastFlushTime = U.currentTimeMillis();
 
-        List<IgniteFuture> activeFuts0 = null;
+        List<IgniteInternalFuture> activeFuts0 = null;
 
         int doneCnt = 0;
 
-        for (IgniteFuture<?> f : activeFuts) {
+        for (IgniteInternalFuture<?> f : activeFuts) {
             if (!f.isDone()) {
                 if (activeFuts0 == null)
                     activeFuts0 = new ArrayList<>((int)(activeFuts.size() * 1.2));
@@ -577,10 +577,10 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
             return;
 
         while (true) {
-            Queue<IgniteFuture<?>> q = null;
+            Queue<IgniteInternalFuture<?>> q = null;
 
             for (Buffer buf : bufMappings.values()) {
-                IgniteFuture<?> flushFut = buf.flush();
+                IgniteInternalFuture<?> flushFut = buf.flush();
 
                 if (flushFut != null) {
                     if (q == null)
@@ -595,7 +595,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
                 boolean err = false;
 
-                for (IgniteFuture fut = q.poll(); fut != null; fut = q.poll()) {
+                for (IgniteInternalFuture fut = q.poll(); fut != null; fut = q.poll()) {
                     try {
                         fut.get();
                     }
@@ -615,7 +615,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
             doneCnt = 0;
 
             for (int i = 0; i < activeFuts0.size(); i++) {
-                IgniteFuture f = activeFuts0.get(i);
+                IgniteInternalFuture f = activeFuts0.get(i);
 
                 if (f == null)
                     doneCnt++;
@@ -752,7 +752,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
         private final ClusterNode node;
 
         /** Active futures. */
-        private final Collection<IgniteFuture<Object>> locFuts;
+        private final Collection<IgniteInternalFuture<Object>> locFuts;
 
         /** Buffered entries. */
         private List<Map.Entry<K, V>> entries;
@@ -775,8 +775,8 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
         /** Closure to signal on task finish. */
         @GridToStringExclude
-        private final IgniteInClosure<IgniteFuture<Object>> signalC = new IgniteInClosure<IgniteFuture<Object>>() {
-            @Override public void apply(IgniteFuture<Object> t) {
+        private final IgniteInClosure<IgniteInternalFuture<Object>> signalC = new IgniteInClosure<IgniteInternalFuture<Object>>() {
+            @Override public void apply(IgniteInternalFuture<Object> t) {
                 signalTaskFinished(t);
             }
         };
@@ -809,7 +809,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
          * @return Future for operation.
          */
         @Nullable GridFutureAdapter<?> update(Iterable<Map.Entry<K, V>> newEntries,
-            IgniteInClosure<IgniteFuture<?>> lsnr) throws IgniteInterruptedException {
+            IgniteInClosure<IgniteInternalFuture<?>> lsnr) throws IgniteInterruptedException {
             List<Map.Entry<K, V>> entries0 = null;
             GridFutureAdapter<Object> curFut0;
 
@@ -853,7 +853,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
          * @throws org.apache.ignite.IgniteInterruptedException If thread has been interrupted.
          */
         @Nullable
-        IgniteFuture<?> flush() throws IgniteInterruptedException {
+        IgniteInternalFuture<?> flush() throws IgniteInterruptedException {
             List<Map.Entry<K, V>> entries0 = null;
             GridFutureAdapter<Object> curFut0 = null;
 
@@ -874,14 +874,14 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
             // Create compound future for this flush.
             GridCompoundFuture<Object, Object> res = null;
 
-            for (IgniteFuture<Object> f : locFuts) {
+            for (IgniteInternalFuture<Object> f : locFuts) {
                 if (res == null)
                     res = new GridCompoundFuture<>(ctx);
 
                 res.add(f);
             }
 
-            for (IgniteFuture<Object> f : reqs.values()) {
+            for (IgniteInternalFuture<Object> f : reqs.values()) {
                 if (res == null)
                     res = new GridCompoundFuture<>(ctx);
 
@@ -906,7 +906,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
         /**
          * @param f Future that finished.
          */
-        private void signalTaskFinished(IgniteFuture<Object> f) {
+        private void signalTaskFinished(IgniteInternalFuture<Object> f) {
             assert f != null;
 
             sem.release();
@@ -925,7 +925,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
             incrementActiveTasks();
 
-            IgniteFuture<Object> fut;
+            IgniteInternalFuture<Object> fut;
 
             if (isLocNode) {
                 fut = ctx.closure().callLocalSafe(
@@ -933,8 +933,8 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
                 locFuts.add(fut);
 
-                fut.listenAsync(new IgniteInClosure<IgniteFuture<Object>>() {
-                    @Override public void apply(IgniteFuture<Object> t) {
+                fut.listenAsync(new IgniteInClosure<IgniteInternalFuture<Object>>() {
+                    @Override public void apply(IgniteInternalFuture<Object> t) {
                         try {
                             boolean rmv = locFuts.remove(t);
 
@@ -1123,7 +1123,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
         void cancelAll() {
             IgniteCheckedException err = new IgniteCheckedException("Data loader has been cancelled: " + IgniteDataLoaderImpl.this);
 
-            for (IgniteFuture<?> f : locFuts) {
+            for (IgniteInternalFuture<?> f : locFuts) {
                 try {
                     f.cancel();
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java
index 2ba9782..03216a2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java
@@ -23,7 +23,6 @@ import org.apache.ignite.dataload.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.cache.dr.*;
 import org.apache.ignite.internal.util.typedef.*;
 
@@ -50,7 +49,7 @@ public class GridDrDataLoadCacheUpdater<K, V> implements IgniteDataLoadCacheUpda
         if (log.isDebugEnabled())
             log.debug("Running DR put job [nodeId=" + ctx.localNodeId() + ", cacheName=" + cacheName + ']');
 
-        IgniteFuture<?> f = cache.context().preloader().startFuture();
+        IgniteInternalFuture<?> f = cache.context().preloader().startFuture();
 
         if (!f.isDone())
             f.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessorAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessorAdapter.java
index 3a383d1..5d8c179 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessorAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessorAdapter.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.email;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
-import org.apache.ignite.lang.*;
 
 import java.util.*;
 
@@ -68,7 +67,7 @@ public abstract class IgniteEmailProcessorAdapter extends GridProcessorAdapter {
      * @param html HTML format flag.
      * @return Future for scheduled email.
      */
-    public abstract IgniteFuture<Boolean> schedule(String subj, String body, boolean html);
+    public abstract IgniteInternalFuture<Boolean> schedule(String subj, String body, boolean html);
 
     /**
      * Schedules sending of given email. If SMTP is disabled - this method is no-op. Emails will be send
@@ -81,5 +80,5 @@ public abstract class IgniteEmailProcessorAdapter extends GridProcessorAdapter {
      * @param addrs Addresses.
      * @return Future for scheduled email.
      */
-    public abstract IgniteFuture<Boolean> schedule(String subj, String body, boolean html, Collection<String> addrs);
+    public abstract IgniteInternalFuture<Boolean> schedule(String subj, String body, boolean html, Collection<String> addrs);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteNoopEmailProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteNoopEmailProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteNoopEmailProcessor.java
index 5d97490..4812f57 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteNoopEmailProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/email/IgniteNoopEmailProcessor.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.email;
 
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.future.*;
 
 import java.util.*;
@@ -45,12 +44,12 @@ public class IgniteNoopEmailProcessor extends IgniteEmailProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> schedule(String subj, String body, boolean html) {
+    @Override public IgniteInternalFuture<Boolean> schedule(String subj, String body, boolean html) {
         return new GridFinishedFuture<>(ctx, true);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> schedule(String subj, String body, boolean html, Collection<String> addrs) {
+    @Override public IgniteInternalFuture<Boolean> schedule(String subj, String body, boolean html, Collection<String> addrs) {
         return new GridFinishedFuture<>(ctx, true);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsAsyncImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsAsyncImpl.java
index 58de369..79cbad2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsAsyncImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsAsyncImpl.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.fs;
 import org.apache.ignite.*;
 import org.apache.ignite.fs.*;
 import org.apache.ignite.fs.mapreduce.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
@@ -130,7 +131,7 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter<IgniteFs> imple
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
         return ggfs.awaitDeletesAsync();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java
index 44623fb..71c018a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsDataManager.java
@@ -71,7 +71,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
     private GridCache<Object, Object> dataCache;
 
     /** */
-    private IgniteFuture<?> dataCacheStartFut;
+    private IgniteInternalFuture<?> dataCacheStartFut;
 
     /** Local GGFS metrics. */
     private GridGgfsLocalMetrics metrics;
@@ -110,7 +110,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
     private long trashPurgeTimeout;
 
     /** On-going remote reads futures. */
-    private final ConcurrentHashMap8<GridGgfsBlockKey, IgniteFuture<byte[]>> rmtReadFuts =
+    private final ConcurrentHashMap8<GridGgfsBlockKey, IgniteInternalFuture<byte[]>> rmtReadFuts =
         new ConcurrentHashMap8<>();
 
     /** Executor service for puts in dual mode */
@@ -379,7 +379,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @return Requested data block or {@code null} if nothing found.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgniteFuture<byte[]> dataBlock(final GridGgfsFileInfo fileInfo, final IgniteFsPath path,
+    @Nullable public IgniteInternalFuture<byte[]> dataBlock(final GridGgfsFileInfo fileInfo, final IgniteFsPath path,
         final long blockIdx, @Nullable final IgniteFsReader secReader)
         throws IgniteCheckedException {
         //assert validTxState(any); // Allow this method call for any transaction state.
@@ -400,17 +400,17 @@ public class GridGgfsDataManager extends GridGgfsManager {
                     ", blockIdx=" + blockIdx + ']');
         }
 
-        IgniteFuture<byte[]> fut = dataCachePrj.getAsync(key);
+        IgniteInternalFuture<byte[]> fut = dataCachePrj.getAsync(key);
 
         if (secReader != null) {
-            fut = fut.chain(new CX1<IgniteFuture<byte[]>, byte[]>() {
-                @Override public byte[] applyx(IgniteFuture<byte[]> fut) throws IgniteCheckedException {
+            fut = fut.chain(new CX1<IgniteInternalFuture<byte[]>, byte[]>() {
+                @Override public byte[] applyx(IgniteInternalFuture<byte[]> fut) throws IgniteCheckedException {
                     byte[] res = fut.get();
 
                     if (res == null) {
                         GridFutureAdapter<byte[]> rmtReadFut = new GridFutureAdapter<>(ggfsCtx.kernalContext());
 
-                        IgniteFuture<byte[]> oldRmtReadFut = rmtReadFuts.putIfAbsent(key, rmtReadFut);
+                        IgniteInternalFuture<byte[]> oldRmtReadFut = rmtReadFuts.putIfAbsent(key, rmtReadFut);
 
                         if (oldRmtReadFut == null) {
                             try {
@@ -491,7 +491,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param fileInfo File info of file opened to write.
      * @return Future that will be completed when all ack messages are received or when write failed.
      */
-    public IgniteFuture<Boolean> writeStart(GridGgfsFileInfo fileInfo) {
+    public IgniteInternalFuture<Boolean> writeStart(GridGgfsFileInfo fileInfo) {
         WriteCompletionFuture fut = new WriteCompletionFuture(ggfsCtx.kernalContext(), fileInfo.id());
 
         WriteCompletionFuture oldFut = pendingWrites.putIfAbsent(fileInfo.id(), fut);
@@ -594,7 +594,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param fileInfo File details to remove data for.
      * @return Delete future that will be completed when file is actually erased.
      */
-    public IgniteFuture<Object> delete(GridGgfsFileInfo fileInfo) {
+    public IgniteInternalFuture<Object> delete(GridGgfsFileInfo fileInfo) {
         //assert validTxState(any); // Allow this method call for any transaction state.
 
         if (!fileInfo.isFile()) {
@@ -1048,8 +1048,8 @@ public class GridGgfsDataManager extends GridGgfsManager {
         else {
             callGgfsLocalSafe(new GridPlainCallable<Object>() {
                 @Override @Nullable public Object call() throws Exception {
-                    storeBlocksAsync(blocks).listenAsync(new CI1<IgniteFuture<?>>() {
-                        @Override public void apply(IgniteFuture<?> fut) {
+                    storeBlocksAsync(blocks).listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> fut) {
                             try {
                                 fut.get();
 
@@ -1242,7 +1242,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @return Future that will be completed after put is done.
      */
     @SuppressWarnings("unchecked")
-    private IgniteFuture<?> storeBlocksAsync(Map<GridGgfsBlockKey, byte[]> blocks) {
+    private IgniteInternalFuture<?> storeBlocksAsync(Map<GridGgfsBlockKey, byte[]> blocks) {
         assert !blocks.isEmpty();
 
         if (dataCachePrj.ggfsDataSpaceUsed() >= dataCachePrj.ggfsDataSpaceMax()) {
@@ -1276,8 +1276,8 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param blocksMsg Write request message.
      */
     private void processBlocksMessage(final UUID nodeId, final GridGgfsBlocksMessage blocksMsg) {
-        storeBlocksAsync(blocksMsg.blocks()).listenAsync(new CI1<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> fut) {
+        storeBlocksAsync(blocksMsg.blocks()).listenAsync(new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> fut) {
                 IgniteCheckedException err = null;
 
                 try {
@@ -1681,7 +1681,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
          * @param info File info to delete.
          * @return Future which completes when entry is actually removed.
          */
-        private IgniteFuture<Object> deleteAsync(GridGgfsFileInfo info) {
+        private IgniteInternalFuture<Object> deleteAsync(GridGgfsFileInfo info) {
             GridFutureAdapter<Object> fut = new GridFutureAdapter<>(ggfsCtx.kernalContext());
 
             delReqs.offer(F.t(fut, info));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsEx.java
index 154a641..153d8f4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsEx.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.fs;
 
 import org.apache.ignite.*;
 import org.apache.ignite.fs.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.jetbrains.annotations.*;
 
@@ -100,7 +101,7 @@ public interface GridGgfsEx extends IgniteFs {
      * @return Future which will be completed when all entries existed in trash by the time of invocation are removed.
      * @throws IgniteCheckedException If failed.
      */
-    public IgniteFuture<?> awaitDeletesAsync() throws IgniteCheckedException;
+    public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException;
 
     /**
      * Gets client file system log directory.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java
index 84942ce..2e20e62 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java
@@ -1550,7 +1550,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
      *
      * @return Future.
      */
-    IgniteFuture<?> formatAsync() throws IgniteCheckedException {
+    IgniteInternalFuture<?> formatAsync() throws IgniteCheckedException {
         IgniteUuid id = meta.softDelete(null, null, ROOT_ID);
 
         if (id == null)
@@ -1576,7 +1576,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
         Collection<IgniteUuid> ids = meta.pendingDeletes();
 
         if (!ids.isEmpty()) {
@@ -1588,7 +1588,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
             for (IgniteUuid id : ids) {
                 GridFutureAdapter<Object> fut = new GridFutureAdapter<>(ggfsCtx.kernalContext());
 
-                IgniteFuture<Object> oldFut = delFuts.putIfAbsent(id, fut);
+                IgniteInternalFuture<Object> oldFut = delFuts.putIfAbsent(id, fut);
 
                 if (oldFut != null)
                     resFut.add(oldFut);
@@ -1708,7 +1708,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
      * @param arg Optional task argument.
      * @return Execution future.
      */
-    <T, R> IgniteFuture<R> executeAsync(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
+    <T, R> IgniteInternalFuture<R> executeAsync(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
         Collection<IgniteFsPath> paths, @Nullable T arg) {
         return executeAsync(task, rslvr, paths, true, cfg.getMaximumTaskRangeLength(), arg);
     }
@@ -1727,7 +1727,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
      * @param arg Optional task argument.
      * @return Execution future.
      */
-    <T, R> IgniteFuture<R> executeAsync(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
+    <T, R> IgniteInternalFuture<R> executeAsync(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
         Collection<IgniteFsPath> paths, boolean skipNonExistentFiles, long maxRangeLen, @Nullable T arg) {
         return ggfsCtx.kernalContext().task().execute(task, new IgniteFsTaskArgsImpl<>(cfg.getName(), paths, rslvr,
             skipNonExistentFiles, maxRangeLen, arg));
@@ -1742,7 +1742,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
      * @param arg Optional task argument.
      * @return Execution future.
      */
-    <T, R> IgniteFuture<R> executeAsync(Class<? extends IgniteFsTask<T, R>> taskCls,
+    <T, R> IgniteInternalFuture<R> executeAsync(Class<? extends IgniteFsTask<T, R>> taskCls,
         @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) {
         return executeAsync(taskCls, rslvr, paths, true, cfg.getMaximumTaskRangeLength(), arg);
     }
@@ -1760,7 +1760,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
      * @param arg Optional task argument.
      * @return Execution future.
      */
-    <T, R> IgniteFuture<R> executeAsync(Class<? extends IgniteFsTask<T, R>> taskCls,
+    <T, R> IgniteInternalFuture<R> executeAsync(Class<? extends IgniteFsTask<T, R>> taskCls,
         @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, boolean skipNonExistentFiles,
         long maxRangeLen, @Nullable T arg) {
         return ggfsCtx.kernalContext().task().execute((Class<IgniteFsTask<T, R>>)taskCls,
@@ -1835,7 +1835,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public <R> IgniteFuture<R> future() {
+    @Override public <R> IgniteInternalFuture<R> future() {
         throw new IllegalStateException("Asynchronous mode is not enabled.");
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsInputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsInputStreamImpl.java
index 19e899a..558d932 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsInputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsInputStreamImpl.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.fs;
 
 import org.apache.ignite.*;
 import org.apache.ignite.fs.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -59,13 +60,13 @@ public class GridGgfsInputStreamImpl extends GridGgfsInputStreamAdapter {
     private long pos;
 
     /** Local cache. */
-    private final Map<Long, IgniteFuture<byte[]>> locCache;
+    private final Map<Long, IgniteInternalFuture<byte[]>> locCache;
 
     /** Maximum local cache size. */
     private final int maxLocCacheSize;
 
     /** Pending data read futures which were evicted from the local cache before completion. */
-    private final Set<IgniteFuture<byte[]>> pendingFuts;
+    private final Set<IgniteInternalFuture<byte[]>> pendingFuts;
 
     /** Pending futures lock. */
     private final Lock pendingFutsLock = new ReentrantLock();
@@ -280,7 +281,7 @@ public class GridGgfsInputStreamImpl extends GridGgfsInputStreamAdapter {
                 secReader.close();
 
                 // Ensuring local cache futures completion.
-                for (IgniteFuture<byte[]> fut : locCache.values()) {
+                for (IgniteInternalFuture<byte[]> fut : locCache.values()) {
                     try {
                         fut.get();
                     }
@@ -427,7 +428,7 @@ public class GridGgfsInputStreamImpl extends GridGgfsInputStreamAdapter {
     private byte[] block(long blockIdx) throws IOException, IgniteCheckedException {
         assert blockIdx >= 0;
 
-        IgniteFuture<byte[]> bytesFut = locCache.get(blockIdx);
+        IgniteInternalFuture<byte[]> bytesFut = locCache.get(blockIdx);
 
         if (bytesFut == null) {
             if (closed)
@@ -482,18 +483,18 @@ public class GridGgfsInputStreamImpl extends GridGgfsInputStreamAdapter {
      * @param idx Block index.
      * @param fut Future.
      */
-    private void addLocalCacheFuture(long idx, IgniteFuture<byte[]> fut) {
+    private void addLocalCacheFuture(long idx, IgniteInternalFuture<byte[]> fut) {
         assert Thread.holdsLock(this);
 
         if (!locCache.containsKey(idx)) {
             if (locCache.size() == maxLocCacheSize) {
-                final IgniteFuture<byte[]> evictFut = locCache.remove(locCache.keySet().iterator().next());
+                final IgniteInternalFuture<byte[]> evictFut = locCache.remove(locCache.keySet().iterator().next());
 
                 if (!evictFut.isDone()) {
                     pendingFuts.add(evictFut);
 
-                    evictFut.listenAsync(new IgniteInClosure<IgniteFuture<byte[]>>() {
-                        @Override public void apply(IgniteFuture<byte[]> t) {
+                    evictFut.listenAsync(new IgniteInClosure<IgniteInternalFuture<byte[]>>() {
+                        @Override public void apply(IgniteInternalFuture<byte[]> t) {
                             pendingFuts.remove(evictFut);
 
                             pendingFutsLock.lock();
@@ -521,7 +522,7 @@ public class GridGgfsInputStreamImpl extends GridGgfsInputStreamAdapter {
      * @return Requested data block or {@code null} if nothing found.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable protected IgniteFuture<byte[]> dataBlock(GridGgfsFileInfo fileInfo, long blockIdx) throws IgniteCheckedException {
+    @Nullable protected IgniteInternalFuture<byte[]> dataBlock(GridGgfsFileInfo fileInfo, long blockIdx) throws IgniteCheckedException {
         return data.dataBlock(fileInfo, path, blockIdx, secReader);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsIpcHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsIpcHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsIpcHandler.java
index 323c1a0..dd9a13c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsIpcHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsIpcHandler.java
@@ -104,7 +104,7 @@ class GridGgfsIpcHandler implements GridGgfsServerHandler {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridGgfsMessage> handleAsync(final GridGgfsClientSession ses,
+    @Override public IgniteInternalFuture<GridGgfsMessage> handleAsync(final GridGgfsClientSession ses,
         final GridGgfsMessage msg, DataInput in) {
         if (!mgmt)
             GridLicenseUseRegistry.onUsage(HADOOP, getClass());
@@ -116,7 +116,7 @@ class GridGgfsIpcHandler implements GridGgfsServerHandler {
 
             final GridGgfsIpcCommand cmd = msg.command();
 
-            IgniteFuture<GridGgfsMessage> fut;
+            IgniteInternalFuture<GridGgfsMessage> fut;
 
             switch (cmd) {
                 // Execute not-blocking command synchronously in worker thread.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java
index feb86a0..d63f25a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsMetaManager.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.fs.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
@@ -55,7 +56,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
     private GridCache<Object, Object> metaCache;
 
     /** */
-    private IgniteFuture<?> metaCacheStartFut;
+    private IgniteInternalFuture<?> metaCacheStartFut;
 
     /** File ID to file info projection. */
     private GridCacheProjectionEx<IgniteUuid, GridGgfsFileInfo> id2InfoPrj;
@@ -1672,12 +1673,12 @@ public class GridGgfsMetaManager extends GridGgfsManager {
                                 id2InfoPrj.invoke(parentInfo.id(),
                                     new UpdateListing(path.name(), new GridGgfsListingEntry(newInfo), false));
 
-                                IgniteFuture<?> delFut = ggfsCtx.data().delete(oldInfo);
+                                IgniteInternalFuture<?> delFut = ggfsCtx.data().delete(oldInfo);
 
                                 // Record PURGE event if needed.
                                 if (evts.isRecordable(EVT_GGFS_FILE_PURGED)) {
-                                    delFut.listenAsync(new CI1<IgniteFuture<?>>() {
-                                        @Override public void apply(IgniteFuture<?> t) {
+                                    delFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                                        @Override public void apply(IgniteInternalFuture<?> t) {
                                             try {
                                                 t.get(); // Ensure delete succeeded.
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServer.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServer.java
index 73010c5..4ce2d47 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServer.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServer.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.fs;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.thread.*;
 import org.apache.ignite.internal.fs.common.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -256,7 +256,7 @@ public class GridGgfsServer {
 
                     GridGgfsMessage msg = marsh.unmarshall(cmd, hdr, dis);
 
-                    IgniteFuture<GridGgfsMessage> fut = hnd.handleAsync(ses, msg, dis);
+                    IgniteInternalFuture<GridGgfsMessage> fut = hnd.handleAsync(ses, msg, dis);
 
                     // If fut is null, no response is required.
                     if (fut != null) {
@@ -287,8 +287,8 @@ public class GridGgfsServer {
                             }
                         }
                         else {
-                            fut.listenAsync(new CIX1<IgniteFuture<GridGgfsMessage>>() {
-                                @Override public void applyx(IgniteFuture<GridGgfsMessage> fut) {
+                            fut.listenAsync(new CIX1<IgniteInternalFuture<GridGgfsMessage>>() {
+                                @Override public void applyx(IgniteInternalFuture<GridGgfsMessage> fut) {
                                     GridGgfsMessage res;
 
                                     try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServerHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServerHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServerHandler.java
index df5091a..470f56d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServerHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsServerHandler.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.fs;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.fs.common.*;
 import org.jetbrains.annotations.*;
 
@@ -38,7 +38,7 @@ public interface GridGgfsServerHandler {
      * @return Future that will be completed when response is ready or {@code null} if no
      *      response is required.
      */
-    @Nullable public IgniteFuture<GridGgfsMessage> handleAsync(GridGgfsClientSession ses,
+    @Nullable public IgniteInternalFuture<GridGgfsMessage> handleAsync(GridGgfsClientSession ses,
         GridGgfsMessage msg, DataInput in);
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/IgniteFsOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/IgniteFsOutputStreamImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/IgniteFsOutputStreamImpl.java
index f069eae..955a615 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/IgniteFsOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/IgniteFsOutputStreamImpl.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.fs;
 
 import org.apache.ignite.*;
 import org.apache.ignite.fs.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.task.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -68,7 +69,7 @@ class IgniteFsOutputStreamImpl extends IgniteFsOutputStreamAdapter {
     private int remainderDataLen;
 
     /** Write completion future. */
-    private final IgniteFuture<Boolean> writeCompletionFut;
+    private final IgniteInternalFuture<Boolean> writeCompletionFut;
 
     /** GGFS mode. */
     private final IgniteFsMode mode;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopNoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopNoopProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopNoopProcessor.java
index 03fa0a2..c861ea8 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopNoopProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopNoopProcessor.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.internal.util.future.*;
 
@@ -50,7 +49,7 @@ public class IgniteHadoopNoopProcessor extends IgniteHadoopProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
+    @Override public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
         return new GridFinishedFutureEx<>(new IgniteCheckedException("Hadoop is not available."));
     }
 
@@ -65,7 +64,7 @@ public class IgniteHadoopNoopProcessor extends IgniteHadoopProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
         return null;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessorAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessorAdapter.java
index fbbf647..05e0e35 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessorAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessorAdapter.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.hadoop;
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.hadoop.*;
 
 /**
@@ -56,7 +55,7 @@ public abstract class IgniteHadoopProcessorAdapter extends GridProcessorAdapter
      * @param jobInfo Job info to submit.
      * @return Execution future.
      */
-    public abstract IgniteFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo);
+    public abstract IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo);
 
     /**
      * Gets Hadoop job execution status.
@@ -83,7 +82,7 @@ public abstract class IgniteHadoopProcessorAdapter extends GridProcessorAdapter
      * @return Job finish future or {@code null}.
      * @throws IgniteCheckedException If failed.
      */
-    public abstract IgniteFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException;
+    public abstract IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException;
 
     /**
      * Kills job.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
index f91c7e0..60528fa 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryIndexing.java
@@ -54,7 +54,7 @@ public interface GridQueryIndexing {
      * @param qry Query.
      * @return Future.
      */
-    public IgniteFuture<GridCacheSqlResult> queryTwoStep(String space,  GridCacheTwoStepQuery qry);
+    public IgniteInternalFuture<GridCacheSqlResult> queryTwoStep(String space,  GridCacheTwoStepQuery qry);
 
     /**
      * @param space Space.
@@ -62,7 +62,7 @@ public interface GridQueryIndexing {
      * @param params Parameters.
      * @return Result.
      */
-    public IgniteFuture<GridCacheSqlResult> queryTwoStep(String space, String sqlQry, Object[] params);
+    public IgniteInternalFuture<GridCacheSqlResult> queryTwoStep(String space, String sqlQry, Object[] params);
 
     /**
      * Queries individual fields (generally used by JDBC drivers).

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
index e14aa10..b094250 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/query/GridQueryProcessor.java
@@ -168,7 +168,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param valTypeName Value type name.
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
-    public IgniteFuture<?> rebuildIndexes(@Nullable final String space, String valTypeName) {
+    public IgniteInternalFuture<?> rebuildIndexes(@Nullable final String space, String valTypeName) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to rebuild indexes (grid is stopping).");
 
@@ -185,7 +185,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param desc Type descriptor.
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
-    private IgniteFuture<?> rebuildIndexes(@Nullable final String space, @Nullable final TypeDescriptor desc) {
+    private IgniteInternalFuture<?> rebuildIndexes(@Nullable final String space, @Nullable final TypeDescriptor desc) {
         if (idx == null)
             return new GridFinishedFuture<>(ctx, new IgniteCheckedException("Indexing is disabled."));
 
@@ -225,7 +225,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @return Future that will be completed when rebuilding of all indexes is finished.
      */
     @SuppressWarnings("unchecked")
-    public IgniteFuture<?> rebuildAllIndexes() {
+    public IgniteInternalFuture<?> rebuildAllIndexes() {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to get space size (grid is stopping).");
 
@@ -233,7 +233,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
             GridCompoundFuture<?, ?> fut = new GridCompoundFuture<Object, Object>(ctx);
 
             for (Map.Entry<TypeId, TypeDescriptor> e : types.entrySet())
-                fut.add((IgniteFuture)rebuildIndexes(e.getKey().space, e.getValue()));
+                fut.add((IgniteInternalFuture)rebuildIndexes(e.getKey().space, e.getValue()));
 
             fut.markInitialized();
 
@@ -440,7 +440,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param qry Query.
      * @return Future.
      */
-    public IgniteFuture<GridCacheSqlResult> queryTwoStep(String space, GridCacheTwoStepQuery qry) {
+    public IgniteInternalFuture<GridCacheSqlResult> queryTwoStep(String space, GridCacheTwoStepQuery qry) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 
@@ -458,7 +458,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
      * @param params Parameters.
      * @return Result.
      */
-    public IgniteFuture<GridCacheSqlResult> queryTwoStep(String space, String sqlQry, Object[] params) {
+    public IgniteInternalFuture<GridCacheSqlResult> queryTwoStep(String space, String sqlQry, Object[] params) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to execute query (grid is stopping).");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
index 0735447..1b480e2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProcessor.java
@@ -79,7 +79,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
             return handleAsync(req).get();
         }
 
-        @Override public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req) {
+        @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
             return handleAsync0(req);
         }
     };
@@ -88,7 +88,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
      * @param req Request.
      * @return Future.
      */
-    private IgniteFuture<GridRestResponse> handleAsync0(final GridRestRequest req) {
+    private IgniteInternalFuture<GridRestResponse> handleAsync0(final GridRestRequest req) {
         if (!busyLock.tryReadLock())
             return new GridFinishedFuture<>(ctx,
                 new IgniteCheckedException("Failed to handle request (received request while stopping grid)."));
@@ -101,10 +101,10 @@ public class GridRestProcessor extends GridProcessorAdapter {
             GridWorker w = new GridWorker(ctx.gridName(), "rest-proc-worker", log) {
                 @Override protected void body() {
                     try {
-                        IgniteFuture<GridRestResponse> res = handleRequest(req);
+                        IgniteInternalFuture<GridRestResponse> res = handleRequest(req);
 
-                        res.listenAsync(new IgniteInClosure<IgniteFuture<GridRestResponse>>() {
-                            @Override public void apply(IgniteFuture<GridRestResponse> f) {
+                        res.listenAsync(new IgniteInClosure<IgniteInternalFuture<GridRestResponse>>() {
+                            @Override public void apply(IgniteInternalFuture<GridRestResponse> f) {
                                 try {
                                     fut.onDone(f.get());
                                 }
@@ -150,7 +150,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
      * @param req Request.
      * @return Future.
      */
-    private IgniteFuture<GridRestResponse> handleRequest(final GridRestRequest req) {
+    private IgniteInternalFuture<GridRestResponse> handleRequest(final GridRestRequest req) {
         if (startLatch.getCount() > 0) {
             try {
                 startLatch.await();
@@ -195,7 +195,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
         GridRestCommandHandler hnd = handlers.get(req.command());
 
-        IgniteFuture<GridRestResponse> res = hnd == null ? null : hnd.handleAsync(req);
+        IgniteInternalFuture<GridRestResponse> res = hnd == null ? null : hnd.handleAsync(req);
 
         if (res == null)
             return new GridFinishedFuture<>(ctx,
@@ -203,8 +203,8 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
         final GridSecurityContext subjCtx0 = subjCtx;
 
-        return res.chain(new C1<IgniteFuture<GridRestResponse>, GridRestResponse>() {
-            @Override public GridRestResponse apply(IgniteFuture<GridRestResponse> f) {
+        return res.chain(new C1<IgniteInternalFuture<GridRestResponse>, GridRestResponse>() {
+            @Override public GridRestResponse apply(IgniteInternalFuture<GridRestResponse> f) {
                 GridRestResponse res;
 
                 try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProtocolHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProtocolHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProtocolHandler.java
index 1fcba69..ed9fd81 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProtocolHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/GridRestProtocolHandler.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.rest;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.rest.request.*;
 
 /**
@@ -36,5 +36,5 @@ public interface GridRestProtocolHandler {
      * @param req Request.
      * @return Future.
      */
-    public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req);
+    public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandler.java
index 1f585ce..e185332 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/GridRestCommandHandler.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.rest.handlers;
 
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.rest.request.*;
 
@@ -36,5 +36,5 @@ public interface GridRestCommandHandler {
      * @param req Request.
      * @return Future.
      */
-    public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req);
+    public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index 8d67de4..9497936 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -127,7 +127,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridRestResponse> handleAsync(final GridRestRequest req) {
+    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(final GridRestRequest req) {
         assert req instanceof GridRestCacheRequest : "Invalid command for topology handler: " + req;
 
         assert SUPPORTED_COMMANDS.contains(req.command());
@@ -153,7 +153,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
             final Long ttl = req0.ttl();
 
-            IgniteFuture<GridRestResponse> fut;
+            IgniteInternalFuture<GridRestResponse> fut;
 
             switch (cmd) {
                 case CACHE_GET: {
@@ -337,7 +337,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @return Operation result in future.
      * @throws IgniteCheckedException If failed
      */
-    private IgniteFuture<GridRestResponse> executeCommand(
+    private IgniteInternalFuture<GridRestResponse> executeCommand(
         @Nullable UUID destId,
         UUID clientId,
         final String cacheName,
@@ -382,7 +382,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @return Operation result in future.
      * @throws IgniteCheckedException If failed
      */
-    private IgniteFuture<GridRestResponse> executeCommand(
+    private IgniteInternalFuture<GridRestResponse> executeCommand(
         @Nullable UUID destId,
         UUID clientId,
         final String cacheName,
@@ -417,7 +417,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @return Future of operation result.
      * @throws IgniteCheckedException In case of error.
      */
-    private static IgniteFuture<?> incrementOrDecrement(CacheProjection<Object, Object> cache, String key,
+    private static IgniteInternalFuture<?> incrementOrDecrement(CacheProjection<Object, Object> cache, String key,
         GridRestCacheRequest req, final boolean decr) throws IgniteCheckedException {
         assert cache != null;
         assert key != null;
@@ -451,7 +451,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @return Future of operation result.
      * @throws IgniteCheckedException In case of any exception.
      */
-    private static IgniteFuture<?> appendOrPrepend(
+    private static IgniteInternalFuture<?> appendOrPrepend(
         final GridKernalContext ctx,
         final CacheProjection<Object, Object> cache,
         final Object key, GridRestCacheRequest req, final boolean prepend) throws IgniteCheckedException {
@@ -555,10 +555,10 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @param key Affinity key for previous operation.
      * @return Rest response.
      */
-    private static IgniteClosure<IgniteFuture<?>, GridRestResponse> resultWrapper(
+    private static IgniteClosure<IgniteInternalFuture<?>, GridRestResponse> resultWrapper(
         final CacheProjection<Object, Object> c, @Nullable final Object key) {
-        return new CX1<IgniteFuture<?>, GridRestResponse>() {
-            @Override public GridRestResponse applyx(IgniteFuture<?> f) throws IgniteCheckedException {
+        return new CX1<IgniteInternalFuture<?>, GridRestResponse>() {
+            @Override public GridRestResponse applyx(IgniteInternalFuture<?> f) throws IgniteCheckedException {
                 GridCacheRestResponse resp = new GridCacheRestResponse();
 
                 resp.setResponse(f.get());
@@ -617,7 +617,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
     /**
      * Fixed result closure.
      */
-    private static final class FixedResult extends CX1<IgniteFuture<?>, Object> {
+    private static final class FixedResult extends CX1<IgniteInternalFuture<?>, Object> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -632,7 +632,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public Object applyx(IgniteFuture<?> f) throws IgniteCheckedException {
+        @Override public Object applyx(IgniteInternalFuture<?> f) throws IgniteCheckedException {
             f.get();
 
             return res;
@@ -643,7 +643,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * Type alias.
      */
     private abstract static class CacheCommand
-        extends IgniteClosure2X<CacheProjection<Object, Object>, GridKernalContext, IgniteFuture<?>> {
+        extends IgniteClosure2X<CacheProjection<Object, Object>, GridKernalContext, IgniteInternalFuture<?>> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -654,7 +654,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * Type alias.
      */
     private abstract static class CacheProjectionCommand
-        extends IgniteClosure2X<CacheProjection<Object, Object>, GridKernalContext, IgniteFuture<?>> {
+        extends IgniteClosure2X<CacheProjection<Object, Object>, GridKernalContext, IgniteInternalFuture<?>> {
         /** */
         private static final long serialVersionUID = 0L;
 
@@ -788,7 +788,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             return c.getAsync(key);
         }
     }
@@ -809,7 +809,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             return c.getAllAsync(keys);
         }
     }
@@ -830,7 +830,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             return c.putAllAsync(map).chain(new FixedResult(true));
         }
     }
@@ -851,7 +851,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             return c.removexAsync(key);
         }
     }
@@ -872,7 +872,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             return (F.isEmpty(keys) ? c.removeAllAsync() : c.removeAllAsync(keys))
                 .chain(new FixedResult(true));
         }
@@ -904,7 +904,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             return exp == null && val == null ? c.removexAsync(key) :
                 exp == null ? c.putxIfAbsentAsync(key, val) :
                     val == null ? c.removeAsync(key, exp) :
@@ -938,7 +938,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             if (ttl != null && ttl > 0) {
                 Duration duration = new Duration(MILLISECONDS, ttl);
 
@@ -975,7 +975,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             if (ttl != null && ttl > 0) {
                 Duration duration = new Duration(MILLISECONDS, ttl);
 
@@ -1012,7 +1012,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             if (ttl != null && ttl > 0) {
                 Duration duration = new Duration(MILLISECONDS, ttl);
 
@@ -1044,7 +1044,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx)
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx)
             throws IgniteCheckedException {
             return incrementOrDecrement(c, (String)key, req, false);
         }
@@ -1071,7 +1071,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) throws IgniteCheckedException {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) throws IgniteCheckedException {
             return incrementOrDecrement(c, (String)key, req, true);
         }
     }
@@ -1097,7 +1097,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx)
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx)
             throws IgniteCheckedException {
             return appendOrPrepend(ctx, c, key, req, false);
         }
@@ -1124,7 +1124,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx)
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx)
             throws IgniteCheckedException {
             return appendOrPrepend(ctx, c, key, req, true);
         }
@@ -1136,7 +1136,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         private static final long serialVersionUID = 0L;
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
+        @Override public IgniteInternalFuture<?> applyx(CacheProjection<Object, Object> c, GridKernalContext ctx) {
             CacheMetrics metrics = c.cache().metrics();
 
             assert metrics != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
index 72236a0..2ebea11 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
@@ -69,7 +69,7 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req) {
+    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
         assert req instanceof GridRestCacheQueryRequest;
         assert SUPPORTED_COMMANDS.contains(req.command());
 
@@ -114,7 +114,7 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
      * @param c Closure to execute.
      * @return Execution future.
      */
-    private IgniteFuture<GridRestResponse> execute(UUID destId, String cacheName, Callable<GridRestResponse> c) {
+    private IgniteInternalFuture<GridRestResponse> execute(UUID destId, String cacheName, Callable<GridRestResponse> c) {
         boolean locExec = destId == null || destId.equals(ctx.localNodeId()) || replicatedCacheAvailable(cacheName);
 
         if (locExec)
@@ -143,16 +143,16 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
      * @param c Closure to execute.
      * @return Execution future.
      */
-    private IgniteFuture<GridRestResponse> broadcast(String cacheName, Callable<Object> c) {
+    private IgniteInternalFuture<GridRestResponse> broadcast(String cacheName, Callable<Object> c) {
         IgniteCompute comp = ctx.grid().compute(ctx.grid().forCache(cacheName)).withNoFailover().withAsync();
 
         try {
             comp.broadcast(c);
 
-            IgniteFuture<Collection<Object>> fut = comp.future();
+            IgniteInternalFuture<Collection<Object>> fut = comp.future();
 
-            return fut.chain(new C1<IgniteFuture<Collection<Object>>, GridRestResponse>() {
-                @Override public GridRestResponse apply(IgniteFuture<Collection<Object>> fut) {
+            return fut.chain(new C1<IgniteInternalFuture<Collection<Object>>, GridRestResponse>() {
+                @Override public GridRestResponse apply(IgniteInternalFuture<Collection<Object>> fut) {
                     try {
                         fut.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/log/GridLogCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/log/GridLogCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/log/GridLogCommandHandler.java
index 3c65b6d..6a676fd 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/log/GridLogCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/log/GridLogCommandHandler.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.rest.handlers.log;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.rest.handlers.*;
 import org.apache.ignite.internal.processors.rest.request.*;
@@ -82,7 +81,7 @@ public class GridLogCommandHandler extends GridRestCommandHandlerAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req) {
+    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
         assert req instanceof GridRestLogRequest : "Invalid command for topology handler: " + req;
 
         assert SUPPORTED_COMMANDS.contains(req.command());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java
index b0a3f10..ea4a70b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.rest.handlers.metadata;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.portables.*;
 import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.rest.client.message.*;
@@ -55,7 +54,7 @@ public class GridPortableMetadataHandler extends GridRestCommandHandlerAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req) {
+    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
         assert SUPPORTED_COMMANDS.contains(req.command()) : req.command();
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
index 0012712..5178847 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
@@ -128,7 +128,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req) {
+    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
         try {
             return handleAsyncUnsafe(req);
         }
@@ -148,7 +148,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
      * @return Future.
      * @throws IgniteCheckedException On any handling exception.
      */
-    private IgniteFuture<GridRestResponse> handleAsyncUnsafe(final GridRestRequest req) throws IgniteCheckedException {
+    private IgniteInternalFuture<GridRestResponse> handleAsyncUnsafe(final GridRestRequest req) throws IgniteCheckedException {
         assert req instanceof GridRestTaskRequest : "Invalid command for topology handler: " + req;
 
         assert SUPPORTED_COMMANDS.contains(req.command());
@@ -226,8 +226,8 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                     fut.onDone(res);
                 }
 
-                taskFut.listenAsync(new IgniteInClosure<IgniteFuture<Object>>() {
-                    @Override public void apply(IgniteFuture<Object> f) {
+                taskFut.listenAsync(new IgniteInClosure<IgniteInternalFuture<Object>>() {
+                    @Override public void apply(IgniteInternalFuture<Object> f) {
                         try {
                             TaskDescriptor desc;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
index ba5e9da..7c05047 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/top/GridTopologyCommandHandler.java
@@ -22,7 +22,6 @@ import org.apache.ignite.cache.affinity.consistenthash.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.internal.processors.port.*;
 import org.apache.ignite.internal.processors.rest.*;
@@ -59,7 +58,7 @@ public class GridTopologyCommandHandler extends GridRestCommandHandlerAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req) {
+    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
         assert req instanceof GridRestTopologyRequest : "Invalid command for topology handler: " + req;
 
         assert SUPPORTED_COMMANDS.contains(req.command());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/version/GridVersionCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/version/GridVersionCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/version/GridVersionCommandHandler.java
index d1f71cb..77e9288 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/version/GridVersionCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/version/GridVersionCommandHandler.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal.processors.rest.handlers.version;
 
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.rest.handlers.*;
 import org.apache.ignite.internal.processors.rest.request.*;
@@ -50,7 +49,7 @@ public class GridVersionCommandHandler extends GridRestCommandHandlerAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridRestResponse> handleAsync(GridRestRequest req) {
+    @Override public IgniteInternalFuture<GridRestResponse> handleAsync(GridRestRequest req) {
         assert req != null;
 
         assert SUPPORTED_COMMANDS.contains(req.command());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
index 8fbe916..5bf5263 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.rest.protocols.tcp;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.apache.ignite.internal.processors.rest.*;
@@ -120,20 +119,20 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
             return;
         }
 
-        IgniteFuture<GridRestResponse> lastFut = ses.removeMeta(LAST_FUT.ordinal());
+        IgniteInternalFuture<GridRestResponse> lastFut = ses.removeMeta(LAST_FUT.ordinal());
 
         if (lastFut != null && lastFut.isDone())
             lastFut = null;
 
-        IgniteFuture<GridRestResponse> f;
+        IgniteInternalFuture<GridRestResponse> f;
 
         if (lastFut == null)
             f = handleRequest0(ses, req, cmd);
         else {
             f = new GridEmbeddedFuture<>(
                 lastFut,
-                new C2<GridRestResponse, Exception, IgniteFuture<GridRestResponse>>() {
-                    @Override public IgniteFuture<GridRestResponse> apply(GridRestResponse res, Exception e) {
+                new C2<GridRestResponse, Exception, IgniteInternalFuture<GridRestResponse>>() {
+                    @Override public IgniteInternalFuture<GridRestResponse> apply(GridRestResponse res, Exception e) {
                         return handleRequest0(ses, req, cmd);
                     }
                 },
@@ -150,7 +149,7 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
      * @param cmd Command.
      * @return Future or {@code null} if processed immediately.
      */
-    @Nullable private IgniteFuture<GridRestResponse> handleRequest0(
+    @Nullable private IgniteInternalFuture<GridRestResponse> handleRequest0(
         final GridNioSession ses,
         final GridMemcachedMessage req,
         final GridTuple3<GridRestCommand, Boolean, Boolean> cmd
@@ -165,10 +164,10 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
             return null;
         }
 
-        IgniteFuture<GridRestResponse> f = hnd.handleAsync(createRestRequest(req, cmd.get1()));
+        IgniteInternalFuture<GridRestResponse> f = hnd.handleAsync(createRestRequest(req, cmd.get1()));
 
-        f.listenAsync(new CIX1<IgniteFuture<GridRestResponse>>() {
-            @Override public void applyx(IgniteFuture<GridRestResponse> f) throws IgniteCheckedException {
+        f.listenAsync(new CIX1<IgniteInternalFuture<GridRestResponse>>() {
+            @Override public void applyx(IgniteInternalFuture<GridRestResponse> f) throws IgniteCheckedException {
                 GridRestResponse restRes = f.get();
 
                 // Handle 'Stat' command (special case because several packets are included in response).

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
index 64a3a68..9aaf3c7 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.rest.protocols.tcp;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.client.marshaller.*;
 import org.apache.ignite.internal.processors.rest.*;
 import org.apache.ignite.internal.processors.rest.client.message.*;
@@ -177,8 +176,8 @@ public class GridTcpRestNioListener extends GridNioServerListenerAdapter<GridCli
                 final GridRestRequest req = createRestRequest(ses, msg);
 
                 if (req != null)
-                    hnd.handleAsync(req).listenAsync(new CI1<IgniteFuture<GridRestResponse>>() {
-                        @Override public void apply(IgniteFuture<GridRestResponse> fut) {
+                    hnd.handleAsync(req).listenAsync(new CI1<IgniteInternalFuture<GridRestResponse>>() {
+                        @Override public void apply(IgniteInternalFuture<GridRestResponse> fut) {
                             GridClientResponse res = new GridClientResponse();
 
                             res.requestId(msg.requestId());


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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
index 6215dd5..5a3e536 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceTest.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.ignite.*;
 import org.apache.ignite.fs.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.counter.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
@@ -98,7 +98,7 @@ public class GridHadoopMapReduceTest extends GridHadoopAbstractWordCountTest {
 
             GridHadoopJobId jobId = new GridHadoopJobId(UUID.randomUUID(), 1);
 
-            IgniteFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
+            IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
 
             fut.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java
index 98367ee..b6e8f41 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopTaskExecutionSelfTest.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.mapreduce.lib.input.*;
 import org.apache.hadoop.mapreduce.lib.output.*;
 import org.apache.ignite.*;
 import org.apache.ignite.fs.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.fs.hadoop.v1.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.internal.util.lang.*;
@@ -144,7 +144,7 @@ public class GridHadoopTaskExecutionSelfTest extends GridHadoopAbstractSelfTest
 
         job.setJarByClass(getClass());
 
-        IgniteFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
+        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
                 createJobInfo(job.getConfiguration()));
 
         fut.get();
@@ -190,7 +190,7 @@ public class GridHadoopTaskExecutionSelfTest extends GridHadoopAbstractSelfTest
 
         GridHadoopJobId jobId = new GridHadoopJobId(UUID.randomUUID(), 2);
 
-        IgniteFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
+        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(job.getConfiguration()));
 
         fut.get();
 
@@ -227,7 +227,7 @@ public class GridHadoopTaskExecutionSelfTest extends GridHadoopAbstractSelfTest
 
         job.setJarByClass(getClass());
 
-        final IgniteFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 3),
+        final IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 3),
                 createJobInfo(job.getConfiguration()));
 
         GridTestUtils.assertThrows(log, new Callable<Object>() {
@@ -315,7 +315,7 @@ public class GridHadoopTaskExecutionSelfTest extends GridHadoopAbstractSelfTest
 
         GridHadoopJobId jobId = new GridHadoopJobId(UUID.randomUUID(), 1);
 
-        final IgniteFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(cfg));
+        final IgniteInternalFuture<?> fut = grid(0).hadoop().submit(jobId, createJobInfo(cfg));
 
         if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {
@@ -366,7 +366,7 @@ public class GridHadoopTaskExecutionSelfTest extends GridHadoopAbstractSelfTest
 
         assertFalse(killRes);
 
-        final IgniteFuture<?> fut = hadoop.submit(jobId, createJobInfo(cfg));
+        final IgniteInternalFuture<?> fut = hadoop.submit(jobId, createJobInfo(cfg));
 
         if (!GridTestUtils.waitForCondition(new GridAbsPredicate() {
             @Override public boolean apply() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorServiceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorServiceTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorServiceTest.java
index a9e2251..66def30 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorServiceTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/GridHadoopExecutorServiceTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.internal.processors.hadoop.taskexecutor;
 
-import org.apache.ignite.lang.IgniteFuture;
+import org.apache.ignite.internal.IgniteInternalFuture;
 import org.apache.ignite.internal.util.typedef.X;
 import org.apache.ignite.testframework.junits.common.GridCommonAbstractTest;
 import org.jdk8.backport.LongAdder;
@@ -83,7 +83,7 @@ public class GridHadoopExecutorServiceTest extends GridCommonAbstractTest {
 
             final AtomicBoolean finish = new AtomicBoolean();
 
-            IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     while (!finish.get()) {
                         exec.submit(new Callable<Void>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java
index 5641735..7a8f8bf 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutionSelfTest.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.*;
 import org.apache.ignite.*;
 import org.apache.ignite.fs.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -93,7 +93,7 @@ public class GridHadoopExternalTaskExecutionSelfTest extends GridHadoopAbstractS
 
         job.setJarByClass(getClass());
 
-        IgniteFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
+        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
             createJobInfo(job.getConfiguration()));
 
         fut.get();
@@ -129,7 +129,7 @@ public class GridHadoopExternalTaskExecutionSelfTest extends GridHadoopAbstractS
 
         job.setJarByClass(getClass());
 
-        IgniteFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
+        IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
             createJobInfo(job.getConfiguration()));
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/test/java/org/apache/ignite/loadtests/ggfs/GridGgfsPerformanceBenchmark.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/loadtests/ggfs/GridGgfsPerformanceBenchmark.java b/modules/hadoop/src/test/java/org/apache/ignite/loadtests/ggfs/GridGgfsPerformanceBenchmark.java
index b85df20..d6f3130 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/loadtests/ggfs/GridGgfsPerformanceBenchmark.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/loadtests/ggfs/GridGgfsPerformanceBenchmark.java
@@ -19,7 +19,7 @@ package org.apache.ignite.loadtests.ggfs;
 
 import org.apache.hadoop.conf.*;
 import org.apache.hadoop.fs.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
 import org.jetbrains.annotations.*;
@@ -86,7 +86,7 @@ public class GridGgfsPerformanceBenchmark {
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
+        IgniteInternalFuture<Long> fut = GridTestUtils.runMultiThreadedAsync(new Runnable() {
             @Override public void run() {
                 String fileIdx = op == OP_READ ? String.valueOf(idx.getAndIncrement()) : UUID.randomUUID().toString();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
index 2d105d2..c7ca6dc 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/IgniteH2Indexing.java
@@ -753,12 +753,12 @@ public class IgniteH2Indexing implements GridQueryIndexing {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheSqlResult> queryTwoStep(String space, GridCacheTwoStepQuery qry) {
+    @Override public IgniteInternalFuture<GridCacheSqlResult> queryTwoStep(String space, GridCacheTwoStepQuery qry) {
         return rdcQryExec.query(space, qry);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheSqlResult> queryTwoStep(String space, String sqlQry, Object[] params) {
+    @Override public IgniteInternalFuture<GridCacheSqlResult> queryTwoStep(String space, String sqlQry, Object[] params) {
         Connection c;
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
index 9b9463d..7278eb8 100644
--- a/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
+++ b/modules/indexing/src/main/java/org/apache/ignite/internal/processors/query/h2/twostep/GridReduceQueryExecutor.java
@@ -110,7 +110,7 @@ public class GridReduceQueryExecutor {
      * @param qry Query.
      * @return Future.
      */
-    public IgniteFuture<GridCacheSqlResult> query(String space, GridCacheTwoStepQuery qry) {
+    public IgniteInternalFuture<GridCacheSqlResult> query(String space, GridCacheTwoStepQuery qry) {
         long qryReqId = reqIdGen.incrementAndGet();
 
         QueryRun r = new QueryRun();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
index 7c79018..d490b79 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
@@ -512,7 +512,7 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
     public void testIteratorsCleanup() throws Exception {
         final GridCache<Long, Long> cache = populate();
 
-        IgniteFuture<?> offHeapFut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> offHeapFut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     Iterator<Map.Entry<Long, Long>> ohIt = cache.offHeapIterator();
@@ -535,7 +535,7 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
             }
         }, 20);
 
-        IgniteFuture<?> swapFut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> swapFut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     Iterator<Map.Entry<Long, Long>> ohIt = cache.swapIterator();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
index 096bb2e..6f28208 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
@@ -259,7 +259,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> fut = multithreadedAsync(new CAX() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 Random rnd = new Random();
 
@@ -339,7 +339,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> fut = multithreadedAsync(new CAX() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 Random rnd = new Random();
 
@@ -423,7 +423,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> fut = multithreadedAsync(new CAX() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 Random rnd = new Random();
 
@@ -504,7 +504,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> fut = multithreadedAsync(new CAX() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 Random rnd = new Random();
 
@@ -584,7 +584,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
         final CacheQuery<Map.Entry<Integer, Integer>> qry = c.queries().createSqlQuery(Integer.class, "_val >= 0");
 
-        IgniteFuture<?> fut = multithreadedAsync(
+        IgniteInternalFuture<?> fut = multithreadedAsync(
             new CAX() {
                 @Override public void applyx() throws IgniteCheckedException {
                     int iter = 0;
@@ -648,7 +648,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> fut = multithreadedAsync(new CAX() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 int iter = 0;
 
@@ -730,7 +730,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
         final AtomicBoolean stop = new AtomicBoolean();
 
-        IgniteFuture<?> fut = multithreadedAsync(new CAX() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 while (!stop.get()) {
                     Collection<Integer> rmtVals = rdcQry.execute(rmtRdc).get();
@@ -799,7 +799,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
         final CacheQuery<Map.Entry<Integer, Integer>> qry = c.queries().createScanQuery(null);
 
-        IgniteFuture<?> fut = multithreadedAsync(
+        IgniteInternalFuture<?> fut = multithreadedAsync(
             new CAX() {
                 @Override public void applyx() throws IgniteCheckedException {
                     int iter = 0;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java
index 07e0d32..6fcd412 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheReduceQueryMultithreadedSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 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.marshaller.optimized.*;
 import org.apache.ignite.internal.util.typedef.*;
 
@@ -88,7 +88,7 @@ public class GridCacheReduceQueryMultithreadedSelfTest extends GridCacheAbstract
 
         final CountDownLatch startLatch = new CountDownLatch(1);
 
-        IgniteFuture<?> fut1 = multithreadedAsync(new Callable() {
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(new Callable() {
             @Override public Object call() throws Exception {
                 for (int i = 1; i < keyCnt; i++) {
                     assertTrue(c.putx(String.valueOf(i), i));
@@ -126,7 +126,7 @@ public class GridCacheReduceQueryMultithreadedSelfTest extends GridCacheAbstract
 
         startLatch.await();
 
-        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/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java
index 33a5001..eb6f1c9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/near/GridCacheQueryNodeRestartSelfTest.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -115,7 +116,7 @@ public class GridCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTest
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> fut1 = multithreadedAsync(new CAX() {
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 while (!done.get()) {
                     CacheQuery<Map.Entry<Integer, Integer>> qry =
@@ -141,7 +142,7 @@ public class GridCacheQueryNodeRestartSelfTest extends GridCacheAbstractSelfTest
         for (int i = 0; i < GRID_CNT; i++)
             grid(i).events().localListen(lsnr, IgniteEventType.EVT_CACHE_PRELOAD_STOPPED);
 
-        IgniteFuture<?> fut2 = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut2 = multithreadedAsync(new Callable<Object>() {
             @SuppressWarnings({"BusyWait"})
             @Override public Object call() throws Exception {
                 while (!done.get()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
index 4feb06a..6eda993 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
@@ -458,7 +458,7 @@ public class GridCacheReplicatedQuerySelfTest extends GridCacheAbstractQuerySelf
             assertEquals(1, futs.size());
 
             GridCloseableIterator<IgniteBiTuple<Integer, Integer>> iter =
-                (GridCloseableIterator<IgniteBiTuple<Integer, Integer>>)((IgniteFuture)F.first(futs.values()).get()).get();
+                (GridCloseableIterator<IgniteBiTuple<Integer, Integer>>)((IgniteInternalFuture)F.first(futs.values()).get()).get();
 
             ResultSet rs = U.field(iter, "data");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexRebuildTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexRebuildTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexRebuildTest.java
index a56a44f..e305482 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexRebuildTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexRebuildTest.java
@@ -21,6 +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.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.query.*;
@@ -207,13 +208,13 @@ public class GridH2IndexRebuildTest extends GridCacheAbstractSelfTest {
 
         spi.sleepInRebuild = false;
 
-        final IgniteFuture<?> fut1 = grid(0).cache(null).queries().rebuildIndexes(TestValue1.class);
+        final IgniteInternalFuture<?> fut1 = grid(0).cache(null).queries().rebuildIndexes(TestValue1.class);
 
         assertFalse(fut1.isCancelled());
 
         fut1.get();
 
-        final IgniteFuture<?> fut2 = grid(0).cache(null).queries().rebuildAllIndexes();
+        final IgniteInternalFuture<?> fut2 = grid(0).cache(null).queries().rebuildAllIndexes();
 
         assertFalse(fut2.isCancelled());
 
@@ -223,7 +224,7 @@ public class GridH2IndexRebuildTest extends GridCacheAbstractSelfTest {
     /**
      * @throws Exception if failed.
      */
-    private void checkCancel(final IgniteFuture<?> fut) throws Exception {
+    private void checkCancel(final IgniteInternalFuture<?> fut) throws Exception {
         assertTrue(fut.cancel());
 
         GridTestUtils.assertThrows(log, new Callable<Void>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java
index a878ffa..71705d9 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/GridH2IndexingGeoSelfTest.java
@@ -21,8 +21,8 @@ import com.vividsolutions.jts.geom.*;
 import com.vividsolutions.jts.io.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
@@ -137,7 +137,7 @@ public class GridH2IndexingGeoSelfTest extends GridCacheAbstractSelfTest {
         final AtomicBoolean stop = new AtomicBoolean();
         final AtomicReference<Exception> err = new AtomicReference<>();
 
-        IgniteFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+        IgniteInternalFuture<?> putFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 WKTReader r = new WKTReader();
 
@@ -161,7 +161,7 @@ public class GridH2IndexingGeoSelfTest extends GridCacheAbstractSelfTest {
             }
         }, Runtime.getRuntime().availableProcessors(), "put-thread");
 
-        IgniteFuture<?> qryFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
+        IgniteInternalFuture<?> qryFut = GridTestUtils.runMultiThreadedAsync(new Callable<Void>() {
             @Override public Void call() throws Exception {
                 WKTReader r = new WKTReader();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
index f76ae6f..47f5b70 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
@@ -125,7 +125,7 @@ public class GridOrderedMessageCancelSelfTest extends GridCommonAbstractTest {
      * @param fut Future to cancel.
      * @throws Exception If failed.
      */
-    private void testMessageSet(IgniteFuture<?> fut) throws Exception {
+    private void testMessageSet(IgniteInternalFuture<?> fut) throws Exception {
         cancelLatch.await();
 
         assertTrue(fut.cancel());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarProjectionPimp.scala
----------------------------------------------------------------------
diff --git a/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarProjectionPimp.scala b/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarProjectionPimp.scala
index 5634ab3..e9e9f1d 100644
--- a/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarProjectionPimp.scala
+++ b/modules/scalar/src/main/scala/org/apache/ignite/scalar/pimps/ScalarProjectionPimp.scala
@@ -19,7 +19,8 @@ package org.apache.ignite.scalar.pimps
 
 import org.apache.ignite._
 import org.apache.ignite.cluster.{ClusterGroup, ClusterGroupEmptyException, ClusterNode}
-import org.apache.ignite.lang.{IgniteFuture, IgniteFutureCancelledException, IgnitePredicate}
+import org.apache.ignite.internal.IgniteInternalFuture
+import org.apache.ignite.lang.{IgniteFutureCancelledException, IgnitePredicate}
 import org.jetbrains.annotations._
 
 /**
@@ -375,7 +376,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      * @see `org.apache.ignite.cluster.ClusterGroup.call(...)`
      */
     def callAsync$[R](@Nullable s: Seq[Call[R]], @Nullable p: NF):
-        IgniteFuture[java.util.Collection[R]] = {
+        IgniteInternalFuture[java.util.Collection[R]] = {
         val comp = value.ignite().compute(forPredicate(p)).withAsync()
 
         comp.call[R](toJavaCollection(s, (f: Call[R]) => toCallable(f)))
@@ -393,7 +394,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      *      closures were executed or `null` (see above).
      * @see `org.apache.ignite.cluster.ClusterGroup.call(...)`
      */
-    def #?[R](@Nullable s: Seq[Call[R]], @Nullable p: NF): IgniteFuture[java.util.Collection[R]] = {
+    def #?[R](@Nullable s: Seq[Call[R]], @Nullable p: NF): IgniteInternalFuture[java.util.Collection[R]] = {
         callAsync$(s, p)
     }
 
@@ -408,7 +409,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      *      closures were executed or `null` (see above).
      * @see `org.apache.ignite.cluster.ClusterGroup.call(...)`
      */
-    def callAsync$[R](@Nullable s: Call[R], @Nullable p: NF): IgniteFuture[java.util.Collection[R]] = {
+    def callAsync$[R](@Nullable s: Call[R], @Nullable p: NF): IgniteInternalFuture[java.util.Collection[R]] = {
         callAsync$(Seq(s), p)
     }
 
@@ -422,7 +423,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      *      closures were executed or `null` (see above).
      * @see `org.apache.ignite.cluster.ClusterGroup.call(...)`
      */
-    def #?[R](@Nullable s: Call[R], @Nullable p: NF): IgniteFuture[java.util.Collection[R]] = {
+    def #?[R](@Nullable s: Call[R], @Nullable p: NF): IgniteInternalFuture[java.util.Collection[R]] = {
         callAsync$(s, p)
     }
 
@@ -435,7 +436,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      * @param p Optional node filter predicate. If `null` provided - all nodes in projection will be used.
      * @see `org.apache.ignite.cluster.ClusterGroup.call(...)`
      */
-    def runAsync$(@Nullable s: Seq[Run], @Nullable p: NF): IgniteFuture[_] = {
+    def runAsync$(@Nullable s: Seq[Run], @Nullable p: NF): IgniteInternalFuture[_] = {
         val comp = value.ignite().compute(forPredicate(p)).withAsync()
 
         comp.run(toJavaCollection(s, (f: Run) => toRunnable(f)))
@@ -451,7 +452,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      * @param p Optional node filter predicate. If `null` provided - all nodes in projection will be used.
      * @see `org.apache.ignite.cluster.ClusterGroup.call(...)`
      */
-    def *?(@Nullable s: Seq[Run], @Nullable p: NF): IgniteFuture[_] = {
+    def *?(@Nullable s: Seq[Run], @Nullable p: NF): IgniteInternalFuture[_] = {
         runAsync$(s, p)
     }
 
@@ -464,7 +465,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      * @param p Optional node filter predicate. If `null` provided - all nodes in projection will be used.
      * @see `org.apache.ignite.cluster.ClusterGroup.run(...)`
      */
-    def runAsync$(@Nullable s: Run, @Nullable p: NF): IgniteFuture[_] = {
+    def runAsync$(@Nullable s: Run, @Nullable p: NF): IgniteInternalFuture[_] = {
         runAsync$(Seq(s), p)
     }
 
@@ -476,7 +477,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      * @param p Optional node filter predicate. If `null` provided - all nodes in projection will be used.
      * @see `org.apache.ignite.cluster.ClusterGroup.run(...)`
      */
-    def *?(@Nullable s: Run, @Nullable p: NF): IgniteFuture[_] = {
+    def *?(@Nullable s: Run, @Nullable p: NF): IgniteInternalFuture[_] = {
         runAsync$(s, p)
     }
 
@@ -492,7 +493,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      * @return Future over the reduced result or `null` (see above).
      * @see `org.apache.ignite.cluster.ClusterGroup.reduce(...)`
      */
-    def reduceAsync$[R1, R2](s: Seq[Call[R1]], r: Seq[R1] => R2, @Nullable p: NF): IgniteFuture[R2] = {
+    def reduceAsync$[R1, R2](s: Seq[Call[R1]], r: Seq[R1] => R2, @Nullable p: NF): IgniteInternalFuture[R2] = {
         assert(s != null && r != null)
 
         val comp = value.ignite().compute(forPredicate(p)).withAsync()
@@ -513,7 +514,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      * @return Future over the reduced result or `null` (see above).
      * @see `org.apache.ignite.cluster.ClusterGroup.reduce(...)`
      */
-    def @?[R1, R2](s: Seq[Call[R1]], r: Seq[R1] => R2, @Nullable p: NF): IgniteFuture[R2] = {
+    def @?[R1, R2](s: Seq[Call[R1]], r: Seq[R1] => R2, @Nullable p: NF): IgniteInternalFuture[R2] = {
         reduceAsync$(s, r, p)
     }
 
@@ -648,7 +649,7 @@ class ScalarProjectionPimp[A <: ClusterGroup] extends PimpedType[A] with Iterabl
      * @throws IgniteFutureCancelledException Subclass of `IgniteCheckedException` thrown if computation was cancelled.
      */
     def affinityRunAsync$(cacheName: String, @Nullable affKey: Any, @Nullable r: Run,
-        @Nullable p: NF): IgniteFuture[_] = {
+        @Nullable p: NF): IgniteInternalFuture[_] = {
         val comp = value.ignite().compute(forPredicate(p)).withAsync()
 
         comp.affinityRun(cacheName, affKey, toRunnable(r))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/schedule/src/main/java/org/apache/ignite/internal/processors/schedule/ScheduleFutureImpl.java
----------------------------------------------------------------------
diff --git a/modules/schedule/src/main/java/org/apache/ignite/internal/processors/schedule/ScheduleFutureImpl.java b/modules/schedule/src/main/java/org/apache/ignite/internal/processors/schedule/ScheduleFutureImpl.java
index 076ec1a..a4c91ca 100644
--- a/modules/schedule/src/main/java/org/apache/ignite/internal/processors/schedule/ScheduleFutureImpl.java
+++ b/modules/schedule/src/main/java/org/apache/ignite/internal/processors/schedule/ScheduleFutureImpl.java
@@ -76,7 +76,7 @@ class ScheduleFutureImpl<R> implements SchedulerFuture<R>, Externalizable {
     private final AtomicBoolean descheduled = new AtomicBoolean(false);
 
     /** Listeners. */
-    private Collection<IgniteInClosure<? super IgniteFuture<R>>> lsnrs =
+    private Collection<IgniteInClosure<? super IgniteInternalFuture<R>>> lsnrs =
         new ArrayList<>(1);
 
     /** Statistics. */
@@ -575,7 +575,7 @@ class ScheduleFutureImpl<R> implements SchedulerFuture<R>, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public void listenAsync(@Nullable IgniteInClosure<? super IgniteFuture<R>> lsnr) {
+    @Override public void listenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<R>> lsnr) {
         if (lsnr != null) {
             Throwable err;
             R res;
@@ -605,7 +605,7 @@ class ScheduleFutureImpl<R> implements SchedulerFuture<R>, Externalizable {
     }
 
     /** {@inheritDoc} */
-    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteFuture<R>>... lsnr) {
+    @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<R>>... lsnr) {
         if (!F.isEmpty(lsnr))
             synchronized (mux) {
                 lsnrs.removeAll(F.asList(lsnr));
@@ -614,7 +614,7 @@ class ScheduleFutureImpl<R> implements SchedulerFuture<R>, Externalizable {
 
     /** {@inheritDoc} */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
-    @Override public <T> IgniteFuture<T> chain(final IgniteClosure<? super IgniteFuture<R>, T> doneCb) {
+    @Override public <T> IgniteInternalFuture<T> chain(final IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb) {
         final GridFutureAdapter<T> fut = new GridFutureAdapter<T>(ctx, syncNotify) {
             @Override public String toString() {
                 return "ChainFuture[orig=" + ScheduleFutureImpl.this + ", doneCb=" + doneCb + ']';
@@ -632,7 +632,7 @@ class ScheduleFutureImpl<R> implements SchedulerFuture<R>, Externalizable {
      * @param err Last execution error.
      * @param syncNotify Synchronous notification flag.
      */
-    private void notifyListener(final IgniteInClosure<? super IgniteFuture<R>> lsnr, R res, Throwable err,
+    private void notifyListener(final IgniteInClosure<? super IgniteInternalFuture<R>> lsnr, R res, Throwable err,
         boolean syncNotify) {
         assert lsnr != null;
         assert !Thread.holdsLock(mux);
@@ -661,7 +661,7 @@ class ScheduleFutureImpl<R> implements SchedulerFuture<R>, Externalizable {
      * @param err Last execution error.
      */
     private void notifyListeners(R res, Throwable err) {
-        final Collection<IgniteInClosure<? super IgniteFuture<R>>> tmp;
+        final Collection<IgniteInClosure<? super IgniteInternalFuture<R>>> tmp;
 
         synchronized (mux) {
             tmp = new ArrayList<>(lsnrs);
@@ -670,7 +670,7 @@ class ScheduleFutureImpl<R> implements SchedulerFuture<R>, Externalizable {
         final SchedulerFuture<R> snapshot = snapshot(res, err);
 
         if (concurNotify) {
-            for (final IgniteInClosure<? super IgniteFuture<R>> lsnr : tmp)
+            for (final IgniteInClosure<? super IgniteInternalFuture<R>> lsnr : tmp)
                 ctx.closure().runLocalSafe(new GPR() {
                     @Override public void run() {
                         lsnr.apply(snapshot);
@@ -680,7 +680,7 @@ class ScheduleFutureImpl<R> implements SchedulerFuture<R>, Externalizable {
         else {
             ctx.closure().runLocalSafe(new GPR() {
                 @Override public void run() {
-                    for (IgniteInClosure<? super IgniteFuture<R>> lsnr : tmp)
+                    for (IgniteInClosure<? super IgniteInternalFuture<R>> lsnr : tmp)
                         lsnr.apply(snapshot);
                 }
             }, true);
@@ -934,17 +934,17 @@ class ScheduleFutureImpl<R> implements SchedulerFuture<R>, Externalizable {
         }
 
         /** {@inheritDoc} */
-        @Override public void listenAsync(@Nullable IgniteInClosure<? super IgniteFuture<R>> lsnr) {
+        @Override public void listenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<R>> lsnr) {
             ref.listenAsync(lsnr);
         }
 
         /** {@inheritDoc} */
-        @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteFuture<R>>... lsnr) {
+        @Override public void stopListenAsync(@Nullable IgniteInClosure<? super IgniteInternalFuture<R>>... lsnr) {
             ref.stopListenAsync(lsnr);
         }
 
         /** {@inheritDoc} */
-        @Override public <T> IgniteFuture<T> chain(IgniteClosure<? super IgniteFuture<R>, T> doneCb) {
+        @Override public <T> IgniteInternalFuture<T> chain(IgniteClosure<? super IgniteInternalFuture<R>, T> doneCb) {
             return ref.chain(doneCb);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/schedule/src/test/java/org/apache/ignite/internal/processors/schedule/GridScheduleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/schedule/src/test/java/org/apache/ignite/internal/processors/schedule/GridScheduleSelfTest.java b/modules/schedule/src/test/java/org/apache/ignite/internal/processors/schedule/GridScheduleSelfTest.java
index a7121b2..c9f45a8 100644
--- a/modules/schedule/src/test/java/org/apache/ignite/internal/processors/schedule/GridScheduleSelfTest.java
+++ b/modules/schedule/src/test/java/org/apache/ignite/internal/processors/schedule/GridScheduleSelfTest.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.schedule;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.scheduler.*;
@@ -62,7 +63,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
      */
     public void testRunLocal() throws Exception {
         for (int i = 0; i < NODES_CNT; i++) {
-            IgniteFuture<?> fut = grid(i).scheduler().runLocal(new TestRunnable());
+            IgniteInternalFuture<?> fut = grid(i).scheduler().runLocal(new TestRunnable());
 
             assert fut.get() == null;
 
@@ -75,7 +76,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
      */
     public void testCallLocal() throws Exception {
         for (int i = 0; i < NODES_CNT; i++) {
-            IgniteFuture<?> fut = grid(i).scheduler().callLocal(new TestCallable());
+            IgniteInternalFuture<?> fut = grid(i).scheduler().callLocal(new TestCallable());
 
             assertEquals(1, fut.get());
 
@@ -112,8 +113,8 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
             final AtomicInteger notifyCnt = new AtomicInteger();
 
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> e) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> e) {
                     notifyCnt.incrementAndGet();
                 }
             });
@@ -170,8 +171,8 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
             final AtomicInteger notifyCnt = new AtomicInteger();
 
-            fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                @Override public void apply(IgniteFuture<?> e) {
+            fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                @Override public void apply(IgniteInternalFuture<?> e) {
                     notifyCnt.incrementAndGet();
                 }
             });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/urideploy/src/test/java/org/apache/ignite/internal/GridTaskUriDeploymentDeadlockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/urideploy/src/test/java/org/apache/ignite/internal/GridTaskUriDeploymentDeadlockSelfTest.java b/modules/urideploy/src/test/java/org/apache/ignite/internal/GridTaskUriDeploymentDeadlockSelfTest.java
index 35bd477..dce45ae 100644
--- a/modules/urideploy/src/test/java/org/apache/ignite/internal/GridTaskUriDeploymentDeadlockSelfTest.java
+++ b/modules/urideploy/src/test/java/org/apache/ignite/internal/GridTaskUriDeploymentDeadlockSelfTest.java
@@ -79,7 +79,7 @@ public class GridTaskUriDeploymentDeadlockSelfTest extends GridCommonAbstractTes
                 }
             }, EVT_NODE_JOINED);
 
-            IgniteFuture<?> f = multithreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<?> f = multithreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     startGrid(2);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
index 7b99774..acf62a5 100644
--- a/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
+++ b/modules/web/src/test/java/org/apache/ignite/internal/websession/WebSessionSelfTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.websession;
 
 import org.apache.ignite.cache.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.eclipse.jetty.server.*;
 import org.eclipse.jetty.servlet.*;
 import org.eclipse.jetty.webapp.*;
@@ -118,7 +118,7 @@ public class WebSessionSelfTest extends GridCommonAbstractTest {
 
         final AtomicBoolean stop = new AtomicBoolean();
 
-        IgniteFuture<?> restarterFut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> restarterFut = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
             @SuppressWarnings("BusyWait")
             @Override public Object call() throws Exception {
                 Random rnd = new Random();


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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFilterEvaluationEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFilterEvaluationEntry.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFilterEvaluationEntry.java
index be2af95..d5ea41f 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFilterEvaluationEntry.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFilterEvaluationEntry.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.internal.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
@@ -117,7 +118,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getAsync() {
+    @Override public IgniteInternalFuture<V> getAsync() {
         throw new UnsupportedOperationException("getAsync");
     }
 
@@ -132,7 +133,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> reloadAsync() {
+    @Override public IgniteInternalFuture<V> reloadAsync() {
         throw new UnsupportedOperationException("reloadAsync");
     }
 
@@ -209,7 +210,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> setAsync(V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<V> setAsync(V val, @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         throw new UnsupportedOperationException("setAsync");
     }
 
@@ -219,7 +220,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> setIfAbsentAsync(V val) {
+    @Override public IgniteInternalFuture<V> setIfAbsentAsync(V val) {
         throw new UnsupportedOperationException("setIfAbsentAsync");
     }
 
@@ -230,7 +231,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> setxAsync(V val,
+    @Override public IgniteInternalFuture<Boolean> setxAsync(V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         throw new UnsupportedOperationException("setxAsync");
     }
@@ -241,7 +242,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> setxIfAbsentAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> setxIfAbsentAsync(V val) {
         throw new UnsupportedOperationException("setxIfAbsentAsync");
     }
 
@@ -251,7 +252,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(V val) {
         throw new UnsupportedOperationException("replaceAsync");
     }
 
@@ -261,7 +262,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(V val) {
         throw new UnsupportedOperationException("replacexAsync");
     }
 
@@ -271,7 +272,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(V oldVal, V newVal) {
         throw new UnsupportedOperationException("replaceAsync");
     }
 
@@ -282,7 +283,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<V> removeAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         throw new UnsupportedOperationException("removeAsync");
     }
 
@@ -292,7 +293,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<Boolean> removexAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         throw new UnsupportedOperationException("removexAsync");
     }
 
@@ -302,7 +303,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(V val) {
         throw new UnsupportedOperationException("removeAsync");
     }
 
@@ -329,7 +330,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements CacheEntry<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAsync(long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAsync(long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         throw new UnsupportedOperationException("lockAsync");
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFuture.java
index df62cba..e836e99 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheFuture.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.lang.*;
 
@@ -26,7 +27,7 @@ import java.util.*;
 /**
  * This interface should be implemented by all distributed futures.
  */
-public interface GridCacheFuture<R> extends IgniteFuture<R> {
+public interface GridCacheFuture<R> extends IgniteInternalFuture<R> {
     /**
      * @return Unique identifier for this future.
      */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
index 837b162..3b7e2a0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheIoManager.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.managers.communication.*;
@@ -112,13 +113,13 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
                     log.debug("Received message has higher topology version [msg=" + msg +
                         ", locTopVer=" + locTopVer + ", rmtTopVer=" + rmtTopVer + ']');
 
-                IgniteFuture<Long> topFut = cctx.discovery().topologyFuture(rmtTopVer);
+                IgniteInternalFuture<Long> topFut = cctx.discovery().topologyFuture(rmtTopVer);
 
                 if (!topFut.isDone()) {
                     final IgniteBiInClosure<UUID, GridCacheMessage<K, V>> c0 = c;
 
-                    topFut.listenAsync(new CI1<IgniteFuture<Long>>() {
-                        @Override public void apply(IgniteFuture<Long> t) {
+                    topFut.listenAsync(new CI1<IgniteInternalFuture<Long>>() {
+                        @Override public void apply(IgniteInternalFuture<Long> t) {
                             onMessage0(nodeId, cacheMsg, c0);
                         }
                     });
@@ -203,7 +204,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
             if (cacheMsg.allowForStartup())
                 processMessage(nodeId, cacheMsg, c);
             else {
-                IgniteFuture<?> startFut = startFuture(cacheMsg);
+                IgniteInternalFuture<?> startFut = startFuture(cacheMsg);
 
                 if (startFut.isDone())
                     processMessage(nodeId, cacheMsg, c);
@@ -213,8 +214,8 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
                             ", locId=" + cctx.localNodeId() + ", msg=" + cacheMsg + ']');
 
                     // Don't hold this thread waiting for preloading to complete.
-                    startFut.listenAsync(new CI1<IgniteFuture<?>>() {
-                        @Override public void apply(IgniteFuture<?> f) {
+                    startFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> f) {
                             rw.readLock();
 
                             try {
@@ -269,7 +270,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
      * @param cacheMsg Cache message to get start future.
      * @return Preloader start future.
      */
-    private IgniteFuture<Object> startFuture(GridCacheMessage<K, V> cacheMsg) {
+    private IgniteInternalFuture<Object> startFuture(GridCacheMessage<K, V> cacheMsg) {
         int cacheId = cacheMsg.cacheId();
 
         return cacheId != 0 ? cctx.cacheContext(cacheId).preloader().startFuture() : cctx.preloadersStartFuture();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
index 0bfcf90..aa593f5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMultiTxFuture.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.transactions.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -103,8 +103,8 @@ public final class GridCacheMultiTxFuture<K, V> extends GridFutureAdapter<Boolea
 
             for (final IgniteTxEx<K, V> tx : txs) {
                 if (!tx.done()) {
-                    tx.finishFuture().listenAsync(new CI1<IgniteFuture<IgniteTx>>() {
-                        @Override public void apply(IgniteFuture<IgniteTx> t) {
+                    tx.finishFuture().listenAsync(new CI1<IgniteInternalFuture<IgniteTx>>() {
+                        @Override public void apply(IgniteInternalFuture<IgniteTx> t) {
                             remainingTxs.remove(tx);
 
                             checkRemaining();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
index 2e8704d..9d69d95 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManager.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -200,7 +201,7 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
                 }
             }
 
-            for (IgniteFuture<?> fut : atomicFuts.values()) {
+            for (IgniteInternalFuture<?> fut : atomicFuts.values()) {
                 if (fut instanceof GridCacheFuture) {
                     GridCacheFuture cacheFut = (GridCacheFuture)fut;
 
@@ -298,7 +299,7 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
      * @param fut Future.
      */
     public void addAtomicFuture(GridCacheVersion futVer, GridCacheAtomicFuture<K, ?> fut) {
-        IgniteFuture<?> old = atomicFuts.put(futVer, fut);
+        IgniteInternalFuture<?> old = atomicFuts.put(futVer, fut);
 
         assert old == null;
     }
@@ -316,7 +317,7 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
      * @param futVer Future ID.
      * @return Future.
      */
-    @Nullable public IgniteFuture<?> atomicFuture(GridCacheVersion futVer) {
+    @Nullable public IgniteInternalFuture<?> atomicFuture(GridCacheVersion futVer) {
         return atomicFuts.get(futVer);
     }
 
@@ -324,7 +325,7 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
      * @param futVer Future ID.
      * @return Removed future.
      */
-    @Nullable public IgniteFuture<?> removeAtomicFuture(GridCacheVersion futVer) {
+    @Nullable public IgniteInternalFuture<?> removeAtomicFuture(GridCacheVersion futVer) {
         return atomicFuts.remove(futVer);
     }
 
@@ -496,10 +497,10 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
      * @param ver Version.
      * @return All futures for given lock version.
      */
-    public <T> Collection<? extends IgniteFuture<T>> futures(GridCacheVersion ver) {
+    public <T> Collection<? extends IgniteInternalFuture<T>> futures(GridCacheVersion ver) {
         Collection c = futs.get(ver);
 
-        return c == null ? Collections.<IgniteFuture<T>>emptyList() : (Collection<IgniteFuture<T>>)c;
+        return c == null ? Collections.<IgniteInternalFuture<T>>emptyList() : (Collection<IgniteInternalFuture<T>>)c;
     }
 
     /**
@@ -902,7 +903,7 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
      * @return Future that signals when all locks for given partitions are released.
      */
     @SuppressWarnings({"unchecked"})
-    public IgniteFuture<?> finishLocks(long topVer) {
+    public IgniteInternalFuture<?> finishLocks(long topVer) {
         assert topVer > 0;
         return finishLocks(null, topVer);
     }
@@ -914,7 +915,7 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
      * @param topVer Topology version to wait for.
      * @return Explicit locks release future.
      */
-    public IgniteFuture<?> finishExplicitLocks(long topVer) {
+    public IgniteInternalFuture<?> finishExplicitLocks(long topVer) {
         GridCompoundFuture<Object, Object> res = new GridCompoundFuture<>(cctx.kernalContext());
 
         for (GridCacheExplicitLockSpan<K> span : pendingExplicit.values()) {
@@ -934,14 +935,14 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
      *
      * @return Finish update future.
      */
-    public IgniteFuture<?> finishAtomicUpdates(long topVer) {
+    public IgniteInternalFuture<?> finishAtomicUpdates(long topVer) {
         GridCompoundFuture<Object, Object> res = new GridCompoundFuture<>(cctx.kernalContext());
 
         res.ignoreChildFailures(ClusterTopologyException.class, CachePartialUpdateCheckedException.class);
 
         for (GridCacheAtomicFuture<K, ?> fut : atomicFuts.values()) {
             if (fut.waitForPartitionExchange() && fut.topologyVersion() < topVer)
-                res.add((IgniteFuture<Object>)fut);
+                res.add((IgniteInternalFuture<Object>)fut);
         }
 
         res.markInitialized();
@@ -955,7 +956,7 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
      * @return Future that signals when all locks for given keys are released.
      */
     @SuppressWarnings("unchecked")
-    public IgniteFuture<?> finishKeys(Collection<K> keys, long topVer) {
+    public IgniteInternalFuture<?> finishKeys(Collection<K> keys, long topVer) {
         if (!(keys instanceof Set))
             keys = new HashSet<>(keys);
 
@@ -973,7 +974,7 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
      * @param topVer Topology version.
      * @return Future that signals when all locks for given partitions will be released.
      */
-    private IgniteFuture<?> finishLocks(@Nullable final IgnitePredicate<K> keyFilter, long topVer) {
+    private IgniteInternalFuture<?> finishLocks(@Nullable final IgnitePredicate<K> keyFilter, long topVer) {
         assert topVer != 0;
 
         if (topVer < 0)
@@ -993,8 +994,8 @@ public class GridCacheMvccManager<K, V> extends GridCacheSharedManagerAdapter<K,
 
         finishFuts.add(finishFut);
 
-        finishFut.listenAsync(new CI1<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> e) {
+        finishFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> e) {
                 finishFuts.remove(finishFut);
 
                 // This call is required to make sure that the concurrent queue

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
index c1ab90b..4cb2d10 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePartitionExchangeManager.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.thread.*;
@@ -130,8 +131,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
                 if (log.isDebugEnabled())
                     log.debug("Discovery event (will start exchange): " + exchId);
 
-                locExchFut.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> t) {
+                locExchFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> t) {
                         if (!enterBusy())
                             return;
 
@@ -338,8 +339,8 @@ public class GridCachePartitionExchangeManager<K, V> extends GridCacheSharedMana
      * @return Exchange futures.
      */
     @SuppressWarnings( {"unchecked", "RedundantCast"})
-    public List<IgniteFuture<?>> exchangeFutures() {
-        return (List<IgniteFuture<?>>)(List)exchFuts.values();
+    public List<IgniteInternalFuture<?>> exchangeFutures() {
+        return (List<IgniteInternalFuture<?>>)(List)exchFuts.values();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
index a29f681..eee1267 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloader.java
@@ -18,6 +18,7 @@
 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.processors.cache.distributed.dht.preloader.*;
 import org.jetbrains.annotations.*;
@@ -100,12 +101,12 @@ public interface GridCachePreloader<K, V> {
     /**
      * @return Future which will complete when preloader is safe to use.
      */
-    public IgniteFuture<Object> startFuture();
+    public IgniteInternalFuture<Object> startFuture();
 
     /**
      * @return Future which will complete when preloading is finished.
      */
-    public IgniteFuture<?> syncFuture();
+    public IgniteInternalFuture<?> syncFuture();
 
     /**
      * Requests that preloader sends the request for the key.
@@ -114,7 +115,7 @@ public interface GridCachePreloader<K, V> {
      * @param topVer Topology version, {@code -1} if not required.
      * @return Future to complete when all keys are preloaded.
      */
-    public IgniteFuture<Object> request(Collection<? extends K> keys, long topVer);
+    public IgniteInternalFuture<Object> request(Collection<? extends K> keys, long topVer);
 
     /**
      * Force preload process.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
index 8128b3b..64efa5a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCachePreloaderAdapter.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.affinity.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.preloader.*;
 import org.apache.ignite.internal.util.future.*;
@@ -40,7 +41,7 @@ public class GridCachePreloaderAdapter<K, V> implements GridCachePreloader<K, V>
     protected final CacheAffinityFunction aff;
 
     /** Start future (always completed by default). */
-    private final IgniteFuture finFut;
+    private final IgniteInternalFuture finFut;
 
     /** Preload predicate. */
     protected IgnitePredicate<GridCacheEntryInfo<K, V>> preloadPred;
@@ -95,12 +96,12 @@ public class GridCachePreloaderAdapter<K, V> implements GridCachePreloader<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Object> startFuture() {
+    @Override public IgniteInternalFuture<Object> startFuture() {
         return finFut;
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> syncFuture() {
+    @Override public IgniteInternalFuture<?> syncFuture() {
         return finFut;
     }
 
@@ -110,7 +111,7 @@ public class GridCachePreloaderAdapter<K, V> implements GridCachePreloader<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Object> request(Collection<? extends K> keys, long topVer) {
+    @Override public IgniteInternalFuture<Object> request(Collection<? extends K> keys, long topVer) {
         return new GridFinishedFuture<>(cctx.kernalContext());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
index e38f39a..b25c8b5 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProcessor.java
@@ -31,7 +31,6 @@ import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.lifecycle.LifecycleAware;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.internal.processors.cache.datastructures.*;
@@ -89,7 +88,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
     private final Map<String, GridCache<?, ?>> publicProxies;
 
     /** Map of preload finish futures grouped by preload order. */
-    private final NavigableMap<Integer, IgniteFuture<?>> preloadFuts;
+    private final NavigableMap<Integer, IgniteInternalFuture<?>> preloadFuts;
 
     /** Maximum detected preload order. */
     private int maxPreloadOrder;
@@ -1336,7 +1335,7 @@ public class GridCacheProcessor extends GridProcessorAdapter {
             }
         }
 
-        for (IgniteFuture<?> fut : preloadFuts.values())
+        for (IgniteInternalFuture<?> fut : preloadFuts.values())
             ((GridCompoundFuture<Object, Object>)fut).markInitialized();
 
         for (GridCacheSharedManager<?, ?> mgr : sharedCtx.managers())
@@ -1475,8 +1474,8 @@ public class GridCacheProcessor extends GridProcessorAdapter {
      * @param order Cache order.
      * @return Compound preload future or {@code null} if order is minimal order found.
      */
-    @Nullable public IgniteFuture<?> orderedPreloadFuture(int order) {
-        Map.Entry<Integer, IgniteFuture<?>> entry = preloadFuts.lowerEntry(order);
+    @Nullable public IgniteInternalFuture<?> orderedPreloadFuture(int order) {
+        Map.Entry<Integer, IgniteInternalFuture<?>> entry = preloadFuts.lowerEntry(order);
 
         return entry == null ? null : entry.getValue();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionEx.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionEx.java
index 0ebb255..25e3cbb 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionEx.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionEx.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
@@ -74,7 +75,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @param filter Optional filter.
      * @return Put operation future.
      */
-    public IgniteFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
+    public IgniteInternalFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
@@ -101,7 +102,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @param filter Optional filter.
      * @return Putx operation future.
      */
-    public IgniteFuture<Boolean> putxAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
+    public IgniteInternalFuture<Boolean> putxAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
@@ -121,7 +122,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @throws IgniteCheckedException If put operation failed.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap) throws IgniteCheckedException;
+    public IgniteInternalFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap) throws IgniteCheckedException;
 
     /**
      * Internal method that is called from {@link GridCacheEntryImpl}.
@@ -143,7 +144,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @param filter Optional filter.
      * @return Put operation future.
      */
-    public IgniteFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
+    public IgniteInternalFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
@@ -163,7 +164,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @throws IgniteCheckedException If remove failed.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException;
+    public IgniteInternalFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException;
 
     /**
      * Internal method that is called from {@link GridCacheEntryImpl}.
@@ -185,7 +186,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @param filter Optional filter.
      * @return Putx operation future.
      */
-    public IgniteFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
+    public IgniteInternalFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter);
 
     /**
@@ -211,7 +212,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @throws NullPointerException If either key or value are {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal);
+    public IgniteInternalFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal);
 
     /**
      * Stores given key-value pair in cache only if only if the previous value is equal to the
@@ -281,7 +282,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @throws NullPointerException if the key or value is {@code null}.
      * @throws CacheFlagException If projection flags validation failed.
      */
-    public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val);
+    public IgniteInternalFuture<GridCacheReturn<V>> removexAsync(K key, V val);
 
     /**
      * @param key Key to retrieve the value for.
@@ -309,7 +310,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @param key Key to get value for.
      * @return Future with result.
      */
-    public IgniteFuture<V> getForcePrimaryAsync(K key);
+    public IgniteInternalFuture<V> getForcePrimaryAsync(K key);
 
     /**
      * Gets values from cache. Will bypass started transaction, if any, i.e. will not enlist entries
@@ -328,7 +329,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @param keys Keys to get values for.
      * @return Future with result.
      */
-    public IgniteFuture<Map<K, V>> getAllOutTxAsync(List<K> keys);
+    public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(List<K> keys);
 
     /**
      * Checks whether this cache is GGFS data cache.
@@ -409,7 +410,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @param args Arguments.
      * @return Future.
      */
-    public <T> IgniteFuture<EntryProcessorResult<T>> invokeAsync(K key,
+    public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args);
 
@@ -430,7 +431,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @param args Arguments.
      * @return Future.
      */
-    public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
+    public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args);
 
@@ -449,7 +450,7 @@ public interface GridCacheProjectionEx<K, V> extends CacheProjection<K, V> {
      * @param args Arguments.
      * @return Future.
      */
-    public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args);
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
index 212c5bf..72b2505 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProjectionImpl.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
@@ -614,7 +615,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> containsKeyAsync(K key) {
+    @Override public IgniteInternalFuture<Boolean> containsKeyAsync(K key) {
         return cache.containsKeyAsync(key, entryFilter(false));
     }
 
@@ -639,7 +640,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> reloadAsync(K key) {
+    @Override public IgniteInternalFuture<V> reloadAsync(K key) {
         return cache.reloadAsync(key, entryFilter(false));
     }
 
@@ -649,7 +650,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> reloadAllAsync() {
+    @Override public IgniteInternalFuture<?> reloadAllAsync() {
         return cache.reloadAllAsync(entryFilter(false));
     }
 
@@ -659,7 +660,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys) {
+    @Override public IgniteInternalFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys) {
         return cache.reloadAllAsync(keys, entryFilter(false));
     }
 
@@ -675,7 +676,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getAsync(K key) {
+    @Override public IgniteInternalFuture<V> getAsync(K key) {
         return cache.getAsync(key, deserializePortables(), entryFilter(false));
     }
 
@@ -685,7 +686,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getForcePrimaryAsync(K key) {
+    @Override public IgniteInternalFuture<V> getForcePrimaryAsync(K key) {
         return cache.getForcePrimaryAsync(key);
     }
 
@@ -695,7 +696,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllOutTxAsync(List<K> keys) {
+    @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(List<K> keys) {
         return cache.getAllOutTxAsync(keys);
     }
 
@@ -730,7 +731,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys) {
+    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys) {
         return cache.getAllAsync(keys, deserializePortables(), entryFilter(false));
     }
 
@@ -747,13 +748,13 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putAsync(K key, V val,
+    @Override public IgniteInternalFuture<V> putAsync(K key, V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return putAsync(key, val, null, -1, filter);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
+    @Override public IgniteInternalFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         A.notNull(key, "key", val, "val");
 
@@ -782,7 +783,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap)
+    @Override public IgniteInternalFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap)
         throws IgniteCheckedException {
         return cache.putAllDrAsync(drMap);
     }
@@ -801,14 +802,14 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<EntryProcessorResult<T>> invokeAsync(K key,
+    @Override public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
         return cache.invokeAsync(key, entryProcessor, args);
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(Set<? extends K> keys,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
         return cache.invokeAllAsync(keys, entryProcessor, args);
@@ -822,20 +823,20 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args) {
         return cache.invokeAllAsync(map, args);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxAsync(K key, V val,
+    @Override public IgniteInternalFuture<Boolean> putxAsync(K key, V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return putxAsync(key, val, null, -1, filter);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<Boolean> putxAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry,
         long ttl, @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         A.notNull(key, "key", val, "val");
 
@@ -852,7 +853,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> putIfAbsentAsync(K key, V val) {
         return putAsync(key, val, cctx.noPeekArray());
     }
 
@@ -862,7 +863,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> putxIfAbsentAsync(K key, V val) {
         return putxAsync(key, val, cctx.noPeekArray());
     }
 
@@ -872,7 +873,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(K key, V val) {
         return putAsync(key, val, cctx.hasPeekArray());
     }
 
@@ -882,7 +883,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(K key, V val) {
         return putxAsync(key, val, cctx.hasPeekArray());
     }
 
@@ -892,7 +893,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
         IgnitePredicate<CacheEntry<K, V>> fltr = and(F.<K, V>cacheContainsPeek(oldVal), false);
 
         return cache.putxAsync(key, newVal, fltr);
@@ -905,7 +906,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
+    @Override public IgniteInternalFuture<?> putAllAsync(Map<? extends K, ? extends V> m,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         m = isAll(m, true);
 
@@ -1072,12 +1073,12 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(K key, IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<V> removeAsync(K key, IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return removeAsync(key, null, filter);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         return cache.removeAsync(key, entry, and(filter, false));
     }
@@ -1094,7 +1095,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException {
         return cache.removeAllDrAsync(drMap);
     }
 
@@ -1105,19 +1106,19 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(K key,
+    @Override public IgniteInternalFuture<Boolean> removexAsync(K key,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return removexAsync(key, null, filter);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         return cache.removexAsync(key, entry, and(filter, false));
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
         A.notNull(key, "key", oldVal, "oldVal", newVal, "newVal");
 
         // Check k-v predicate first.
@@ -1138,7 +1139,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
         return !isAll(key, val, true) ? new GridFinishedFuture<>(cctx.kernalContext(),
             new GridCacheReturn<V>(false)) : cache.removexAsync(key, val);
     }
@@ -1149,7 +1150,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
         return !isAll(key, val, true) ? new GridFinishedFuture<>(cctx.kernalContext(), false) :
             cache.removeAsync(key, val);
     }
@@ -1161,7 +1162,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(@Nullable Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<?> removeAllAsync(@Nullable Collection<? extends K> keys,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return cache.removeAllAsync(keys, and(filter, false));
     }
@@ -1173,7 +1174,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
+    @Override public IgniteInternalFuture<?> removeAllAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         return cache.removeAllAsync(and(filter, false));
     }
 
@@ -1184,7 +1185,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAsync(K key, long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAsync(K key, long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return cache.lockAsync(key, timeout, and(filter, false));
     }
@@ -1196,7 +1197,7 @@ public class GridCacheProjectionImpl<K, V> implements GridCacheProjectionEx<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys, long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys, long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         return cache.lockAllAsync(keys, timeout, and(filter, false));
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
index 5d78aac..a5b73dc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheProxyImpl.java
@@ -23,6 +23,7 @@ import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.datastructures.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.mxbean.*;
@@ -245,7 +246,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> loadCacheAsync(IgniteBiPredicate<K, V> p, long ttl, @Nullable Object[] args) {
+    @Override public IgniteInternalFuture<?> loadCacheAsync(IgniteBiPredicate<K, V> p, long ttl, @Nullable Object[] args) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -354,7 +355,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> containsKeyAsync(K key) {
+    @Override public IgniteInternalFuture<Boolean> containsKeyAsync(K key) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -415,7 +416,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> reloadAsync(K key) {
+    @Override public IgniteInternalFuture<V> reloadAsync(K key) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -439,7 +440,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> reloadAllAsync() {
+    @Override public IgniteInternalFuture<?> reloadAllAsync() {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -463,7 +464,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys) {
+    @Override public IgniteInternalFuture<?> reloadAllAsync(@Nullable Collection<? extends K> keys) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -500,7 +501,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getAsync(K key) {
+    @Override public IgniteInternalFuture<V> getAsync(K key) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -524,7 +525,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> getForcePrimaryAsync(K key) {
+    @Override public IgniteInternalFuture<V> getForcePrimaryAsync(K key) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -548,7 +549,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllOutTxAsync(List<K> keys) {
+    @Override public IgniteInternalFuture<Map<K, V>> getAllOutTxAsync(List<K> keys) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -632,7 +633,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys) {
+    @Override public IgniteInternalFuture<Map<K, V>> getAllAsync(@Nullable Collection<? extends K> keys) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -670,7 +671,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putAsync(K key, V val,
+    @Override public IgniteInternalFuture<V> putAsync(K key, V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -683,7 +684,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
+    @Override public IgniteInternalFuture<V> putAsync(K key, V val, @Nullable GridCacheEntryEx<K, V> entry, long ttl,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -734,7 +735,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap)
+    @Override public IgniteInternalFuture<?> putAllDrAsync(Map<? extends K, GridCacheDrInfo<V>> drMap)
         throws IgniteCheckedException {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -761,7 +762,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<EntryProcessorResult<T>> invokeAsync(K key,
+    @Override public <T> IgniteInternalFuture<EntryProcessorResult<T>> invokeAsync(K key,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
@@ -789,7 +790,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         Set<? extends K> keys,
         EntryProcessor<K, V, T> entryProcessor,
         Object... args) {
@@ -818,7 +819,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public <T> IgniteFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
+    @Override public <T> IgniteInternalFuture<Map<K, EntryProcessorResult<T>>> invokeAllAsync(
         Map<? extends K, ? extends EntryProcessor<K, V, T>> map,
         Object... args) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
@@ -832,7 +833,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxAsync(K key, V val,
+    @Override public IgniteInternalFuture<Boolean> putxAsync(K key, V val,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -845,7 +846,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxAsync(K key,
+    @Override public IgniteInternalFuture<Boolean> putxAsync(K key,
         V val,
         @Nullable GridCacheEntryEx<K, V> entry,
         long ttl,
@@ -873,7 +874,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> putIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> putIfAbsentAsync(K key, V val) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -897,7 +898,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> putxIfAbsentAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> putxIfAbsentAsync(K key, V val) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -921,7 +922,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> replaceAsync(K key, V val) {
+    @Override public IgniteInternalFuture<V> replaceAsync(K key, V val) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -945,7 +946,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replacexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> replacexAsync(K key, V val) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -969,7 +970,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<Boolean> replaceAsync(K key, V oldVal, V newVal) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -994,7 +995,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> putAllAsync(@Nullable Map<? extends K, ? extends V> m,
+    @Override public IgniteInternalFuture<?> putAllAsync(@Nullable Map<? extends K, ? extends V> m,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -1362,7 +1363,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(K key, IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<V> removeAsync(K key, IgnitePredicate<CacheEntry<K, V>>[] filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -1374,7 +1375,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<V> removeAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -1412,7 +1413,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> removeAllDrAsync(Map<? extends K, GridCacheVersion> drMap) throws IgniteCheckedException {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -1437,7 +1438,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(K key,
+    @Override public IgniteInternalFuture<Boolean> removexAsync(K key,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -1450,7 +1451,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
+    @Override public IgniteInternalFuture<Boolean> removexAsync(K key, @Nullable GridCacheEntryEx<K, V> entry,
         @Nullable IgnitePredicate<CacheEntry<K, V>>... filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -1463,7 +1464,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> replacexAsync(K key, V oldVal, V newVal) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -1499,7 +1500,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
+    @Override public IgniteInternalFuture<GridCacheReturn<V>> removexAsync(K key, V val) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -1523,7 +1524,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> removeAsync(K key, V val) {
+    @Override public IgniteInternalFuture<Boolean> removeAsync(K key, V val) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -1548,7 +1549,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(@Nullable Collection<? extends K> keys,
+    @Override public IgniteInternalFuture<?> removeAllAsync(@Nullable Collection<? extends K> keys,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -1574,7 +1575,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeAllAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
+    @Override public IgniteInternalFuture<?> removeAllAsync(@Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
@@ -1599,7 +1600,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAsync(K key, long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAsync(K key, long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -1625,7 +1626,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys, long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAllAsync(@Nullable Collection<? extends K> keys, long timeout,
         @Nullable IgnitePredicate<CacheEntry<K, V>>[] filter) {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
@@ -1855,7 +1856,7 @@ public class GridCacheProxyImpl<K, V> implements GridCacheProxy<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> forceRepartition() {
+    @Override public IgniteInternalFuture<?> forceRepartition() {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
index 34d8ee0..956c503 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheSharedContext.java
@@ -22,7 +22,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.version.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.managers.communication.*;
@@ -74,7 +73,7 @@ public class GridCacheSharedContext<K, V> {
     private volatile IgniteTxMetricsAdapter txMetrics;
 
     /** Preloaders start future. */
-    private IgniteFuture<Object> preloadersStartFut;
+    private IgniteInternalFuture<Object> preloadersStartFut;
 
     /**
      * @param txMgr Transaction manager.
@@ -196,12 +195,12 @@ public class GridCacheSharedContext<K, V> {
     /**
      * @return Compound preloaders start future.
      */
-    public IgniteFuture<Object> preloadersStartFuture() {
+    public IgniteInternalFuture<Object> preloadersStartFuture() {
         if (preloadersStartFut == null) {
             GridCompoundFuture<Object, Object> compound = null;
 
             for (GridCacheContext<K, V> cacheCtx : cacheContexts()) {
-                IgniteFuture<Object> startFut = cacheCtx.preloader().startFuture();
+                IgniteInternalFuture<Object> startFut = cacheCtx.preloader().startFuture();
 
                 if (!startFut.isDone()) {
                     if (compound == null)
@@ -382,7 +381,7 @@ public class GridCacheSharedContext<K, V> {
      * @return {@code true} if waiting was successful.
      */
     @SuppressWarnings({"unchecked"})
-    public IgniteFuture<?> partitionReleaseFuture(long topVer) {
+    public IgniteInternalFuture<?> partitionReleaseFuture(long topVer) {
         GridCompoundFuture f = new GridCompoundFuture(kernalCtx);
 
         f.add(mvcc().finishExplicitLocks(topVer));
@@ -462,7 +461,7 @@ public class GridCacheSharedContext<K, V> {
      * @param tx Transaction to commit.
      * @return Commit future.
      */
-    public IgniteFuture<IgniteTx> commitTxAsync(IgniteTxEx<K, V> tx) {
+    public IgniteInternalFuture<IgniteTx> commitTxAsync(IgniteTxEx<K, V> tx) {
         Collection<Integer> cacheIds = tx.activeCacheIds();
 
         if (cacheIds.isEmpty())
@@ -484,7 +483,7 @@ public class GridCacheSharedContext<K, V> {
      * @param tx Transaction to rollback.
      * @throws IgniteCheckedException If failed.
      */
-    public IgniteFuture rollbackTxAsync(IgniteTxEx<K, V> tx) throws IgniteCheckedException {
+    public IgniteInternalFuture rollbackTxAsync(IgniteTxEx<K, V> tx) throws IgniteCheckedException {
         Collection<Integer> cacheIds = tx.activeCacheIds();
 
         if (!cacheIds.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
index 28cc285..7cbcd85 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheUtils.java
@@ -22,6 +22,7 @@ import org.apache.ignite.cache.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.fs.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.distributed.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -1060,10 +1061,10 @@ public class GridCacheUtils {
      * @param excl Excludes.
      * @return Future listener that logs errors.
      */
-    public static IgniteInClosure<IgniteFuture<?>> errorLogger(final IgniteLogger log,
+    public static IgniteInClosure<IgniteInternalFuture<?>> errorLogger(final IgniteLogger log,
         final Class<? extends Exception>... excl) {
-        return new CI1<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> f) {
+        return new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> f) {
                 try {
                     f.get();
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
index cca29b7..743e5b9 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/IgniteCacheProxy.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.mxbean.*;
 import org.apache.ignite.internal.util.tostring.*;
@@ -458,11 +459,11 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter<IgniteCach
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {
-            IgniteFuture<?>  fut = ctx.cache().loadAll(keys, replaceExisting);
+            IgniteInternalFuture<?> fut = ctx.cache().loadAll(keys, replaceExisting);
 
             if (completionLsnr != null) {
-                fut.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> fut) {
+                fut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> fut) {
                         try {
                             fut.get();
 
@@ -786,10 +787,10 @@ public class IgniteCacheProxy<K, V> extends IgniteAsyncSupportAdapter<IgniteCach
 
             try {
                 if (isAsync()) {
-                    IgniteFuture<EntryProcessorResult<T>> fut = delegate.invokeAsync(key, entryProcessor, args);
+                    IgniteInternalFuture<EntryProcessorResult<T>> fut = delegate.invokeAsync(key, entryProcessor, args);
 
-                    IgniteFuture<T> fut0 = fut.chain(new CX1<IgniteFuture<EntryProcessorResult<T>>, T>() {
-                        @Override public T applyx(IgniteFuture<EntryProcessorResult<T>> fut)
+                    IgniteInternalFuture<T> fut0 = fut.chain(new CX1<IgniteInternalFuture<EntryProcessorResult<T>>, T>() {
+                        @Override public T applyx(IgniteInternalFuture<EntryProcessorResult<T>> fut)
                             throws IgniteCheckedException {
                             EntryProcessorResult<T> res = fut.get();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java
index 67499a7..628d696 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAtomicSequenceImpl.java
@@ -20,6 +20,7 @@ package org.apache.ignite.internal.processors.cache.datastructures;
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.datastructures.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
@@ -263,7 +264,7 @@ public final class GridCacheAtomicSequenceImpl implements GridCacheAtomicSequenc
      * @throws IgniteCheckedException If update failed.
      */
     @SuppressWarnings("SignalWithoutCorrespondingAwait")
-    private IgniteFuture<Long> internalUpdateAsync(long l, @Nullable Callable<Long> updateCall, boolean updated)
+    private IgniteInternalFuture<Long> internalUpdateAsync(long l, @Nullable Callable<Long> updateCall, boolean updated)
         throws IgniteCheckedException {
         checkRemoved();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java
index 19629ae..dbfbb2c 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheOptimisticCheckPreparedTxFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -219,7 +220,7 @@ public class GridCacheOptimisticCheckPreparedTxFuture<K, V> extends GridCompound
      */
     public void onResult(UUID nodeId, GridCacheOptimisticCheckPreparedTxResponse<K, V> res) {
         if (!isDone()) {
-            for (IgniteFuture<Boolean> fut : pending()) {
+            for (IgniteInternalFuture<Boolean> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -252,7 +253,7 @@ public class GridCacheOptimisticCheckPreparedTxFuture<K, V> extends GridCompound
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        for (IgniteFuture<?> fut : futures())
+        for (IgniteInternalFuture<?> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -302,7 +303,7 @@ public class GridCacheOptimisticCheckPreparedTxFuture<K, V> extends GridCompound
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java
index 818477d..2afa7a4 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCachePessimisticCheckCommittedTxFuture.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.internal.util.*;
@@ -169,7 +170,7 @@ public class GridCachePessimisticCheckCommittedTxFuture<K, V> extends GridCompou
      */
     public void onResult(UUID nodeId, GridCachePessimisticCheckCommittedTxResponse<K, V> res) {
         if (!isDone()) {
-            for (IgniteFuture<GridCacheCommittedTxInfo<K, V>> fut : pending()) {
+            for (IgniteInternalFuture<GridCacheCommittedTxInfo<K, V>> fut : pending()) {
                 if (isMini(fut)) {
                     MiniFuture f = (MiniFuture)fut;
 
@@ -202,7 +203,7 @@ public class GridCachePessimisticCheckCommittedTxFuture<K, V> extends GridCompou
 
     /** {@inheritDoc} */
     @Override public boolean onNodeLeft(UUID nodeId) {
-        for (IgniteFuture<?> fut : futures())
+        for (IgniteInternalFuture<?> fut : futures())
             if (isMini(fut)) {
                 MiniFuture f = (MiniFuture)fut;
 
@@ -258,7 +259,7 @@ public class GridCachePessimisticCheckCommittedTxFuture<K, V> extends GridCompou
      * @param f Future.
      * @return {@code True} if mini-future.
      */
-    private boolean isMini(IgniteFuture<?> f) {
+    private boolean isMini(IgniteInternalFuture<?> f) {
         return f.getClass().equals(MiniFuture.class);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java
index 9023c5d..3441d69 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridCacheTxFinishSync.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.future.*;
 import org.jdk8.backport.*;
@@ -71,7 +71,7 @@ public class GridCacheTxFinishSync<K, V> {
      * @param threadId Thread ID to wait ack.
      * @return {@code null} if ack was received or future that will be completed when ack is received.
      */
-    public IgniteFuture<?> awaitAckAsync(UUID nodeId, long threadId) {
+    public IgniteInternalFuture<?> awaitAckAsync(UUID nodeId, long threadId) {
         ThreadFinishSync threadSync = threadMap.get(threadId);
 
         if (threadSync == null)
@@ -150,7 +150,7 @@ public class GridCacheTxFinishSync<K, V> {
          * @param nodeId Node ID to wait ack from.
          * @return {@code null} if ack has been received or future that will be completed when ack is received.
          */
-        public IgniteFuture<?> awaitAckAsync(UUID nodeId) {
+        public IgniteInternalFuture<?> awaitAckAsync(UUID nodeId) {
             TxFinishSync sync = nodeMap.get(nodeId);
 
             if (sync == null)
@@ -233,7 +233,7 @@ public class GridCacheTxFinishSync<K, V> {
          *
          * @return {@code null} if ack has been received, or future that will be completed when ack is received.
          */
-        @Nullable public IgniteFuture<?> awaitAckAsync() {
+        @Nullable public IgniteInternalFuture<?> awaitAckAsync() {
             synchronized (this) {
                 if (cnt == 0)
                     return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
index 0a33a4c..52eea01 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedCacheAdapter.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.lang.*;
@@ -60,7 +61,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> txLockAsync(
+    @Override public IgniteInternalFuture<Boolean> txLockAsync(
         Collection<? extends K> keys,
         long timeout,
         IgniteTxLocalEx<K, V> tx,
@@ -77,7 +78,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
+    @Override public IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys, long timeout,
         IgnitePredicate<CacheEntry<K, V>>... filter) {
         IgniteTxLocalEx<K, V> tx = ctx.tm().userTxx();
 
@@ -97,7 +98,7 @@ public abstract class GridDistributedCacheAdapter<K, V> extends GridCacheAdapter
      * @param filter Optional filter.
      * @return Future for locks.
      */
-    protected abstract IgniteFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
+    protected abstract IgniteInternalFuture<Boolean> lockAllAsync(Collection<? extends K> keys,
         long timeout,
         @Nullable IgniteTxLocalEx<K, V> tx,
         boolean isInvalidate,

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
index f7376cf..c417e9d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/distributed/GridDistributedTxRemoteAdapter.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.cache.distributed;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.processors.cache.version.*;
 import org.apache.ignite.lang.*;
@@ -359,7 +360,7 @@ public class GridDistributedTxRemoteAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<IgniteTxEx<K, V>> prepareAsync() {
+    @Override public IgniteInternalFuture<IgniteTxEx<K, V>> prepareAsync() {
         assert false;
         return null;
     }
@@ -717,7 +718,7 @@ public class GridDistributedTxRemoteAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<IgniteTx> commitAsync() {
+    @Override public IgniteInternalFuture<IgniteTx> commitAsync() {
         try {
             commit();
 
@@ -748,7 +749,7 @@ public class GridDistributedTxRemoteAdapter<K, V> extends IgniteTxAdapter<K, V>
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<IgniteTx> rollbackAsync() {
+    @Override public IgniteInternalFuture<IgniteTx> rollbackAsync() {
         rollback();
 
         return new GridFinishedFutureEx<IgniteTx>(this);


[03/50] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'origin/sprint-1' into sprint-1

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/sprint-1' into sprint-1


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

Branch: refs/heads/ignite-16
Commit: 6ce04499b37b0eaf83967f26c00df013ee1e1d8b
Parents: 8e8a0c3 79ba5af
Author: vozerov-gridgain <vo...@gridgain.com>
Authored: Wed Jan 28 18:05:07 2015 +0300
Committer: vozerov-gridgain <vo...@gridgain.com>
Committed: Wed Jan 28 18:05:07 2015 +0300

----------------------------------------------------------------------
 .../src/main/java/org/apache/ignite/internal/util/GridUtils.java   | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[34/50] [abbrv] incubator-ignite git commit: GridEx -> IgniteEx GridKernal -> IgniteKernal

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
index fda2751..0e83e66 100644
--- a/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
+++ b/modules/core/src/test/java/org/apache/ignite/testsuites/IgniteComputeGridTestSuite.java
@@ -71,10 +71,10 @@ public class IgniteComputeGridTestSuite {
         suite.addTestSuite(GridDeploymentSelfTest.class);
         suite.addTestSuite(GridDeploymentMultiThreadedSelfTest.class);
         suite.addTestSuite(GridMultipleVersionsDeploymentSelfTest.class);
-        suite.addTestSuite(GridExplicitImplicitDeploymentSelfTest.class);
+        suite.addTestSuite(IgniteExplicitImplicitDeploymentSelfTest.class);
         suite.addTestSuite(GridEventStorageCheckAllEventsSelfTest.class);
         suite.addTestSuite(GridCommunicationManagerListenersSelfTest.class);
-        suite.addTestSuite(GridExecutorServiceTest.class);
+        suite.addTestSuite(IgniteExecutorServiceTest.class);
         suite.addTestSuite(GridTaskInstantiationSelfTest.class);
         suite.addTestSuite(GridManagementJobSelfTest.class);
         suite.addTestSuite(GridMultipleJobsSelfTest.class);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlanner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlanner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlanner.java
index 651ed02..d98d8f4 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlanner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/planner/GridHadoopDefaultMapReducePlanner.java
@@ -158,7 +158,7 @@ public class GridHadoopDefaultMapReducePlanner implements GridHadoopMapReducePla
                 GridGgfsEx ggfs = null;
 
                 if (F.eq(ignite.name(), endpoint.grid()))
-                    ggfs = (GridGgfsEx)((GridEx)ignite).ggfsx(endpoint.ggfs());
+                    ggfs = (GridGgfsEx)((IgniteEx)ignite).ggfsx(endpoint.ggfs());
 
                 if (ggfs != null && !ggfs.isProxy(split0.file())) {
                     Collection<IgniteFsBlockLocation> blocks = ggfs.affinity(new IgniteFsPath(split0.file()),

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/hadoop/src/test/java/org/apache/ignite/fs/GridGgfsHadoopFileSystemClientSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/fs/GridGgfsHadoopFileSystemClientSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/fs/GridGgfsHadoopFileSystemClientSelfTest.java
index 13618fc..8f66dae 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/fs/GridGgfsHadoopFileSystemClientSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/fs/GridGgfsHadoopFileSystemClientSelfTest.java
@@ -164,7 +164,7 @@ public class GridGgfsHadoopFileSystemClientSelfTest extends GridGgfsCommonAbstra
      */
     @SuppressWarnings("ConstantConditions")
     private void switchHandlerErrorFlag(boolean flag) throws Exception {
-        IgniteFsProcessorAdapter ggfsProc = ((GridKernal)grid(0)).context().ggfs();
+        IgniteFsProcessorAdapter ggfsProc = ((IgniteKernal)grid(0)).context().ggfs();
 
         Map<String, GridGgfsContext> ggfsMap = getField(ggfsProc, "ggfsCache");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
index 5688500..65b8ba2 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
@@ -70,7 +70,7 @@ public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstrac
     private static final String INVALID_HOST_3 = "invalid_host3";
 
     /** Mocked Grid. */
-    private static final MockGrid GRID = new MockGrid();
+    private static final MockIgnite GRID = new MockIgnite();
 
     /** Mocked GGFS. */
     private static final IgniteFs GGFS = new MockGgfs();
@@ -930,7 +930,7 @@ public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstrac
      * Mocked Grid.
      */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
-    private static class MockGrid extends IgniteSpringBean implements GridEx {
+    private static class MockIgnite extends IgniteSpringBean implements IgniteEx {
         /** {@inheritDoc} */
         @Override public IgniteFs ggfsx(String name) {
             assert F.eq("ggfs", name);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
index 85f55d8..b735263 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
@@ -216,7 +216,7 @@ public class GridHadoopJobTrackerSelfTest extends GridHadoopAbstractSelfTest {
      */
     private void checkStatus(GridHadoopJobId jobId, boolean complete) throws Exception {
         for (int i = 0; i < gridCount(); i++) {
-            GridKernal kernal = (GridKernal)grid(i);
+            IgniteKernal kernal = (IgniteKernal)grid(i);
 
             GridHadoop hadoop = kernal.hadoop();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractQuerySelfTest.java
index 53b9607..61d0237 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractQuerySelfTest.java
@@ -647,8 +647,8 @@ public abstract class GridCacheAbstractQuerySelfTest extends GridCommonAbstractT
         cache.putx(key, val);
         cache1.putx(key, val);
 
-        GridCacheQueryManager<Object, Object> qryMgr = ((GridKernal) ignite).internalCache().context().queries();
-        GridCacheQueryManager<Object, Object> qryMgr1 = ((GridKernal) ignite).internalCache("c1").context().queries();
+        GridCacheQueryManager<Object, Object> qryMgr = ((IgniteKernal) ignite).internalCache().context().queries();
+        GridCacheQueryManager<Object, Object> qryMgr1 = ((IgniteKernal) ignite).internalCache("c1").context().queries();
 
         assert hasIndexTable(ObjectValue.class, qryMgr);
         assert hasIndexTable(ObjectValue.class, qryMgr1);
@@ -883,7 +883,7 @@ public abstract class GridCacheAbstractQuerySelfTest extends GridCommonAbstractT
 
         for (int i = 0; i < gridCount(); i++) {
             GridCacheQueryManager<Object, Object> qryMgr =
-                ((GridKernal)grid(i)).internalCache().context().queries();
+                ((IgniteKernal)grid(i)).internalCache().context().queries();
 
             assert !hasIndexTable(EmptyObject.class, qryMgr);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
index d490b79..b529279 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOffHeapAndSwapSelfTest.java
@@ -285,7 +285,7 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
     public void testPartitionIterators() throws Exception {
         populate();
 
-        GridCacheAdapter<Long, Object> cacheAdapter = ((GridKernal)grid(0)).internalCache();
+        GridCacheAdapter<Long, Object> cacheAdapter = ((IgniteKernal)grid(0)).internalCache();
         GridNearCacheAdapter<Long, Object> cache = (GridNearCacheAdapter<Long, Object>)cacheAdapter;
 
         Map<Integer, Collection<Long>> grouped = new HashMap<>();
@@ -566,6 +566,6 @@ public class GridCacheOffHeapAndSwapSelfTest extends GridCommonAbstractTest {
         // Runs iterator queue cleanup in GridCacheSwapManager.read method.
         cache.get(1L + ENTRY_CNT);
 
-        assertEquals(0, ((GridKernal)grid(0)).internalCache().context().swap().iteratorSetSize());
+        assertEquals(0, ((IgniteKernal)grid(0)).internalCache().context().swap().iteratorSetSize());
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryLoadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryLoadSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryLoadSelfTest.java
index 7c93844..1e50a92 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryLoadSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryLoadSelfTest.java
@@ -101,7 +101,7 @@ public class GridCacheQueryLoadSelfTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private long size(Class<?> cls) throws IgniteCheckedException {
-        GridCacheQueryManager<Object, Object> qryMgr = ((GridKernal)grid()).internalCache().context().queries();
+        GridCacheQueryManager<Object, Object> qryMgr = ((IgniteKernal)grid()).internalCache().context().queries();
 
         assert qryMgr != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
index 6f28208..b7bace1 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryMultiThreadedSelfTest.java
@@ -603,7 +603,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
                         if (cnt.incrementAndGet() % logMod == 0) {
                             GridCacheQueryManager<Object, Object> qryMgr =
-                                ((GridKernal)g).internalCache().context().queries();
+                                ((IgniteKernal)g).internalCache().context().queries();
 
                             assert qryMgr != null;
 
@@ -668,7 +668,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
                     if (cnt.incrementAndGet() % logMod == 0) {
                         GridCacheQueryManager<Object, Object> qryMgr =
-                            ((GridKernal)g).internalCache().context().queries();
+                            ((IgniteKernal)g).internalCache().context().queries();
 
                         assert qryMgr != null;
 
@@ -757,7 +757,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
                     if (cnt.incrementAndGet() % logMod == 0) {
                         GridCacheQueryManager<Object, Object> qryMgr =
-                            ((GridKernal)g).internalCache().context().queries();
+                            ((IgniteKernal)g).internalCache().context().queries();
 
                         assert qryMgr != null;
 
@@ -817,7 +817,7 @@ public class GridCacheQueryMultiThreadedSelfTest extends GridCommonAbstractTest
 
                         if (cnt.incrementAndGet() % logMod == 0) {
                             GridCacheQueryManager<Object, Object> qryMgr =
-                                ((GridKernal)g).internalCache().context().queries();
+                                ((IgniteKernal)g).internalCache().context().queries();
 
                             assert qryMgr != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java
index ab02b26..b21dee0 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedFieldsQuerySelfTest.java
@@ -59,7 +59,7 @@ public class GridCacheReplicatedFieldsQuerySelfTest extends GridCacheAbstractFie
 
         try {
             final Map<UUID, Map<Long, GridFutureAdapter<GridQueryFieldsResult>>> map =
-                U.field(((GridKernal)grid(0)).internalCache().context().queries(), "fieldsQryRes");
+                U.field(((IgniteKernal)grid(0)).internalCache().context().queries(), "fieldsQryRes");
 
             // Ensure that iterators map empty.
             map.clear();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
index 6eda993..4d480b3 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/cache/distributed/replicated/GridCacheReplicatedQuerySelfTest.java
@@ -285,7 +285,7 @@ public class GridCacheReplicatedQuerySelfTest extends GridCacheAbstractQuerySelf
     private ConcurrentMap<UUID,
         Map<Long, GridFutureAdapter<GridCloseableIterator<IgniteBiTuple<CacheKey, CacheValue>>>>>
         distributedQueryManagerQueryItersMap(Ignite g) {
-        GridCacheContext ctx = ((GridKernal)g).internalCache().context();
+        GridCacheContext ctx = ((IgniteKernal)g).internalCache().context();
 
         Field qryItersField = ReflectionUtils.findField(ctx.queries().getClass(), "qryIters");
 
@@ -442,7 +442,7 @@ public class GridCacheReplicatedQuerySelfTest extends GridCacheAbstractQuerySelf
 
             final ConcurrentMap<UUID, Map<Long, GridFutureAdapter<GridCloseableIterator<
                 IgniteBiTuple<Integer, Integer>>>>> map =
-                U.field(((GridKernal)grid(0)).internalCache().context().queries(), "qryIters");
+                U.field(((IgniteKernal)grid(0)).internalCache().context().queries(), "qryIters");
 
             // fut.nextX() does not guarantee the request has completed on remote node
             // (we could receive page from local one), so we need to wait.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
index 96d46cb..4bac005 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/internal/processors/query/h2/sql/GridQueryParsingTest.java
@@ -194,7 +194,7 @@ public class GridQueryParsingTest extends GridCommonAbstractTest {
      *
      */
     private JdbcConnection connection() throws Exception {
-        GridKernalContext ctx = ((GridKernal)ignite).context();
+        GridKernalContext ctx = ((IgniteKernal)ignite).context();
 
         GridQueryProcessor qryProcessor = ctx.query();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java b/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
index 47f5b70..7f95f34 100644
--- a/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
+++ b/modules/indexing/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
@@ -134,7 +134,7 @@ public class GridOrderedMessageCancelSelfTest extends GridCommonAbstractTest {
 
         assertTrue(U.await(finishLatch, 5000, MILLISECONDS));
 
-        Map map = U.field(((GridKernal)grid(0)).context().io(), "msgSetMap");
+        Map map = U.field(((IgniteKernal)grid(0)).context().io(), "msgSetMap");
 
         info("Map: " + map);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
index e7d9d60..30cdd61 100644
--- a/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
+++ b/modules/jta/src/test/java/org/apache/ignite/internal/processors/cache/GridTmLookupLifecycleAwareSelfTest.java
@@ -84,7 +84,7 @@ public class GridTmLookupLifecycleAwareSelfTest extends GridAbstractLifecycleAwa
     /** {@inheritDoc} */
     @Override protected void afterGridStart(Ignite ignite) {
         TestTxLookup tmLookup =
-            (TestTxLookup)((GridKernal) ignite).context().cache().internalCache(CACHE_NAME).context().jta().tmLookup();
+            (TestTxLookup)((IgniteKernal) ignite).context().cache().internalCache(CACHE_NAME).context().jta().tmLookup();
 
         assertNotNull(tmLookup);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringDynamicCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringDynamicCacheManager.java b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringDynamicCacheManager.java
index c37fe8f..ba088e3 100644
--- a/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringDynamicCacheManager.java
+++ b/modules/spring/src/main/java/org/apache/ignite/cache/spring/SpringDynamicCacheManager.java
@@ -103,7 +103,7 @@ public class SpringDynamicCacheManager extends SpringCacheManager {
     @Override public void afterPropertiesSet() throws Exception {
         super.afterPropertiesSet();
 
-        metaCache = ((GridEx)grid).utilityCache(MetaKey.class, org.springframework.cache.Cache.class);
+        metaCache = ((IgniteEx)grid).utilityCache(MetaKey.class, org.springframework.cache.Cache.class);
         dataCache = grid.cache(dataCacheName);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
----------------------------------------------------------------------
diff --git a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
index d56ab3b..cb7ce1b 100644
--- a/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
+++ b/modules/visor-console/src/main/scala/org/apache/ignite/visor/visor.scala
@@ -37,7 +37,7 @@ import org.apache.ignite.internal.util.{GridConfigurationFinder, GridUtils}
 import org.apache.ignite.internal.visor.VisorTaskArgument
 import org.apache.ignite.internal.visor.node.VisorNodeEventsCollectorTask
 import org.apache.ignite.internal.visor.node.VisorNodeEventsCollectorTask.VisorNodeEventsCollectorTaskArg
-import org.apache.ignite.internal.{GridEx, GridProductImpl}
+import org.apache.ignite.internal.{IgniteEx, GridProductImpl}
 import org.apache.ignite.lang.{IgniteNotPeerDeployable, IgnitePredicate}
 import org.apache.ignite.lifecycle.IgniteListener
 import org.apache.ignite.spi.communication.tcp.TcpCommunicationSpi
@@ -228,7 +228,7 @@ object visor extends VisorTag {
     @volatile var cfgPath: String = null
 
     /** */
-    @volatile var grid: GridEx = null
+    @volatile var grid: IgniteEx = null
 
     /**
      * Get grid node for specified ID.
@@ -1603,7 +1603,7 @@ object visor extends VisorTag {
 
         grid =
             try
-                Ignition.ignite(startedGridName).asInstanceOf[GridEx]
+                Ignition.ignite(startedGridName).asInstanceOf[IgniteEx]
             catch {
                 case _: IllegalStateException =>
                     this.cfgPath = null


[49/50] [abbrv] incubator-ignite git commit: Merge remote-tracking branch 'origin/sprint-1' into sprint-1

Posted by vo...@apache.org.
Merge remote-tracking branch 'origin/sprint-1' into sprint-1


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

Branch: refs/heads/ignite-16
Commit: 9ea96070ac972ca23ccf1e34c980303a4c45e823
Parents: 535ca6f 4535322
Author: Yakov Zhdanov <yz...@gridgain.com>
Authored: Thu Jan 29 13:33:03 2015 +0300
Committer: Yakov Zhdanov <yz...@gridgain.com>
Committed: Thu Jan 29 13:33:03 2015 +0300

----------------------------------------------------------------------
 assembly/release-base-fabric.xml | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[08/50] [abbrv] incubator-ignite git commit: #Tests: fix GridHibernateL2CacheExampleSelfTest

Posted by vo...@apache.org.
#Tests: fix GridHibernateL2CacheExampleSelfTest


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

Branch: refs/heads/ignite-16
Commit: 8643e944b332c526ca05a09aaf3c611199c1f674
Parents: 79ba5af
Author: ivasilinets <iv...@gridgain.com>
Authored: Wed Jan 28 19:19:46 2015 +0300
Committer: ivasilinets <iv...@gridgain.com>
Committed: Wed Jan 28 19:19:46 2015 +0300

----------------------------------------------------------------------
 .../config/hibernate/example-hibernate-L2-cache.xml   | 14 +++++++-------
 .../cache/hibernate/GridHibernateRegionFactory.java   | 10 +++++-----
 2 files changed, 12 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8643e944/examples/config/hibernate/example-hibernate-L2-cache.xml
----------------------------------------------------------------------
diff --git a/examples/config/hibernate/example-hibernate-L2-cache.xml b/examples/config/hibernate/example-hibernate-L2-cache.xml
index 86b224a..779dd36 100644
--- a/examples/config/hibernate/example-hibernate-L2-cache.xml
+++ b/examples/config/hibernate/example-hibernate-L2-cache.xml
@@ -45,21 +45,21 @@
         <property name="cache.region.factory_class">org.apache.ignite.cache.hibernate.GridHibernateRegionFactory</property>
 
         <!-- Specify the name of the grid, that will be used for second level caching. -->
-        <property name="org.gridgain.hibernate.grid_name">hibernate-grid</property>
+        <property name="org.apache.ignite.hibernate.grid_name">hibernate-grid</property>
 
         <!-- Specify connection release mode. -->
         <property name="connection.release_mode">on_close</property>
 
         <!-- Set default L2 cache access type. -->
-        <property name="org.gridgain.hibernate.default_access_type">READ_ONLY</property>
+        <property name="org.apache.ignite.hibernate.default_access_type">READ_ONLY</property>
 
         <!-- Specify the entity classes for mapping. -->
-        <mapping class="org.gridgain.examples.datagrid.hibernate.User"/>
-        <mapping class="org.gridgain.examples.datagrid.hibernate.Post"/>
+        <mapping class="org.apache.ignite.examples.datagrid.hibernate.User"/>
+        <mapping class="org.apache.ignite.examples.datagrid.hibernate.Post"/>
 
         <!-- Per-class L2 cache settings. -->
-        <class-cache class="org.gridgain.examples.datagrid.hibernate.User" usage="read-only"/>
-        <class-cache class="org.gridgain.examples.datagrid.hibernate.Post" usage="read-only"/>
-        <collection-cache collection="org.gridgain.examples.datagrid.hibernate.User.posts" usage="read-only"/>
+        <class-cache class="org.apache.ignite.examples.datagrid.hibernate.User" usage="read-only"/>
+        <class-cache class="org.apache.ignite.examples.datagrid.hibernate.Post" usage="read-only"/>
+        <collection-cache collection="org.apache.ignite.examples.datagrid.hibernate.User.posts" usage="read-only"/>
     </session-factory>
 </hibernate-configuration>

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8643e944/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/GridHibernateRegionFactory.java
----------------------------------------------------------------------
diff --git a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/GridHibernateRegionFactory.java b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/GridHibernateRegionFactory.java
index 751785d..d9f84d2 100644
--- a/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/GridHibernateRegionFactory.java
+++ b/modules/hibernate/src/main/java/org/apache/ignite/cache/hibernate/GridHibernateRegionFactory.java
@@ -60,19 +60,19 @@ public class GridHibernateRegionFactory implements RegionFactory {
     private static final long serialVersionUID = 0L;
 
     /** Hibernate L2 cache grid name property name. */
-    public static final String GRID_NAME_PROPERTY = "org.gridgain.hibernate.grid_name";
+    public static final String GRID_NAME_PROPERTY = "org.apache.ignite.hibernate.grid_name";
 
     /** Default cache property name. */
-    public static final String DFLT_CACHE_NAME_PROPERTY = "org.gridgain.hibernate.default_cache";
+    public static final String DFLT_CACHE_NAME_PROPERTY = "org.apache.ignite.hibernate.default_cache";
 
     /** Property prefix used to specify region name to cache name mapping. */
-    public static final String REGION_CACHE_PROPERTY = "org.gridgain.hibernate.region_cache.";
+    public static final String REGION_CACHE_PROPERTY = "org.apache.ignite.hibernate.region_cache.";
 
     /** */
-    public static final String DFLT_ACCESS_TYPE_PROPERTY = "org.gridgain.hibernate.default_access_type";
+    public static final String DFLT_ACCESS_TYPE_PROPERTY = "org.apache.ignite.hibernate.default_access_type";
 
     /** */
-    public static final String GRID_CONFIG_PROPERTY = "org.gridgain.hibernate.grid_config";
+    public static final String GRID_CONFIG_PROPERTY = "org.apache.ignite.hibernate.grid_config";
 
     /** Grid providing caches. */
     private Ignite ignite;


[05/50] [abbrv] incubator-ignite git commit: #Fixed ClusterMetricsSnapshotSerializeSelfTest test.

Posted by vo...@apache.org.
#Fixed ClusterMetricsSnapshotSerializeSelfTest test.


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

Branch: refs/heads/ignite-16
Commit: be6686d8985386df2bb891337e41bfa42d646261
Parents: 6ce0449
Author: nikolay_tikhonov <nt...@gridgain.com>
Authored: Wed Jan 28 18:26:06 2015 +0300
Committer: nikolay_tikhonov <nt...@gridgain.com>
Committed: Wed Jan 28 18:27:14 2015 +0300

----------------------------------------------------------------------
 ...ClusterMetricsSnapshotSerializeSelfTest.java | 52 ++++++++++++++++++--
 1 file changed, 48 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/be6686d8/modules/core/src/test/java/org/apache/ignite/spi/discovery/ClusterMetricsSnapshotSerializeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/ClusterMetricsSnapshotSerializeSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/ClusterMetricsSnapshotSerializeSelfTest.java
index a737a91..5eca33f 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/ClusterMetricsSnapshotSerializeSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/ClusterMetricsSnapshotSerializeSelfTest.java
@@ -21,9 +21,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.internal.*;
 import org.apache.ignite.testframework.junits.common.*;
 
-import java.io.*;
-import java.util.*;
-
 /**
  * Grid discovery metrics test.
  */
@@ -65,7 +62,7 @@ public class ClusterMetricsSnapshotSerializeSelfTest extends GridCommonAbstractT
 
         assert metrics2 != null;
 
-        assert metrics1.equals(metrics2);
+        assert isMetricsEquals(metrics1, metrics2);
     }
 
     /**
@@ -123,4 +120,51 @@ public class ClusterMetricsSnapshotSerializeSelfTest extends GridCommonAbstractT
 
         return metrics;
     }
+
+    private boolean isMetricsEquals(ClusterMetrics obj, ClusterMetrics obj1) {
+        return
+            obj.getAverageActiveJobs() == obj1.getAverageActiveJobs() &&
+            obj.getAverageCancelledJobs() == obj1.getAverageCancelledJobs() &&
+            obj.getAverageJobExecuteTime() == obj1.getAverageJobExecuteTime() &&
+            obj.getAverageJobWaitTime() == obj1.getAverageJobWaitTime() &&
+            obj.getAverageRejectedJobs() == obj1.getAverageRejectedJobs() &&
+            obj.getAverageWaitingJobs() == obj1.getAverageWaitingJobs() &&
+            obj.getCurrentActiveJobs() == obj1.getCurrentActiveJobs() &&
+            obj.getCurrentCancelledJobs() == obj1.getCurrentCancelledJobs() &&
+            obj.getCurrentIdleTime() == obj1.getCurrentIdleTime() &&
+            obj.getCurrentJobExecuteTime() == obj1.getCurrentJobExecuteTime() &&
+            obj.getCurrentJobWaitTime() == obj1.getCurrentJobWaitTime() &&
+            obj.getCurrentRejectedJobs() == obj1.getCurrentRejectedJobs() &&
+            obj.getCurrentWaitingJobs() == obj1.getCurrentWaitingJobs() &&
+            obj.getCurrentDaemonThreadCount() == obj1.getCurrentDaemonThreadCount() &&
+            obj.getHeapMemoryCommitted() == obj1.getHeapMemoryCommitted() &&
+            obj.getHeapMemoryInitialized() == obj1.getHeapMemoryInitialized() &&
+            obj.getHeapMemoryMaximum() == obj1.getHeapMemoryMaximum() &&
+            obj.getHeapMemoryUsed() == obj1.getHeapMemoryUsed() &&
+            obj.getMaximumActiveJobs() == obj1.getMaximumActiveJobs() &&
+            obj.getMaximumCancelledJobs() == obj1.getMaximumCancelledJobs() &&
+            obj.getMaximumJobExecuteTime() == obj1.getMaximumJobExecuteTime() &&
+            obj.getMaximumJobWaitTime() == obj1.getMaximumJobWaitTime() &&
+            obj.getMaximumRejectedJobs() == obj1.getMaximumRejectedJobs() &&
+            obj.getMaximumWaitingJobs() == obj1.getMaximumWaitingJobs() &&
+            obj.getNonHeapMemoryCommitted() == obj1.getNonHeapMemoryCommitted() &&
+            obj.getNonHeapMemoryInitialized() == obj1.getNonHeapMemoryInitialized() &&
+            obj.getNonHeapMemoryMaximum() == obj1.getNonHeapMemoryMaximum() &&
+            obj.getNonHeapMemoryUsed() == obj1.getNonHeapMemoryUsed() &&
+            obj.getMaximumThreadCount() == obj1.getMaximumThreadCount() &&
+            obj.getStartTime() == obj1.getStartTime() &&
+            obj.getCurrentCpuLoad() == obj1.getCurrentCpuLoad() &&
+            obj.getCurrentThreadCount() == obj1.getCurrentThreadCount() &&
+            obj.getTotalCancelledJobs() == obj1.getTotalCancelledJobs() &&
+            obj.getTotalExecutedJobs() == obj1.getTotalExecutedJobs() &&
+            obj.getTotalIdleTime() == obj1.getTotalIdleTime() &&
+            obj.getTotalRejectedJobs() == obj1.getTotalRejectedJobs() &&
+            obj.getTotalStartedThreadCount() == obj1.getTotalStartedThreadCount() &&
+            obj.getUpTime() == obj1.getUpTime() &&
+            obj.getSentMessagesCount() == obj1.getSentMessagesCount() &&
+            obj.getSentBytesCount() == obj1.getSentBytesCount() &&
+            obj.getReceivedMessagesCount() == obj1.getReceivedMessagesCount() &&
+            obj.getReceivedBytesCount() == obj1.getReceivedBytesCount() &&
+            obj.getOutboundMessagesQueueSize() == obj1.getOutboundMessagesQueueSize();
+    }
 }


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

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
index f985d45..7442d25 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/communication/GridIoManagerBenchmark0.java
@@ -159,7 +159,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
 
         final AtomicBoolean finish = new AtomicBoolean();
 
-        IgniteFuture<?> f = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> f = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 try {
                     IgniteUuid msgId = IgniteUuid.randomUuid();
@@ -251,7 +251,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
 
         final AtomicBoolean finish = new AtomicBoolean();
 
-        IgniteFuture<?> f = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> f = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 try {
                     IgniteUuid msgId = IgniteUuid.randomUuid();
@@ -341,7 +341,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
         final AtomicBoolean finish = new AtomicBoolean();
         final AtomicReference<CountDownLatch> latchRef = new AtomicReference<>();
 
-        IgniteFuture<?> f = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> f = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!finish.get()) {
                     CountDownLatch latch = latchRef.get();
@@ -360,7 +360,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
             }
         }, THREADS, "send-thread");
 
-        IgniteFuture<?> f1 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> f1 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
             private long ts = System.currentTimeMillis();
 
             @Override public Object call() throws Exception {
@@ -413,7 +413,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
             }
         }, 1, "load-dispatcher");
 
-        IgniteFuture<?> f2 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> f2 = GridTestUtils.runMultiThreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while (!finish.get()) {
                     U.sleep(1000);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/continuous/GridContinuousOperationsLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/continuous/GridContinuousOperationsLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/continuous/GridContinuousOperationsLoadTest.java
index 7b86298..b0699f2 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/continuous/GridContinuousOperationsLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/continuous/GridContinuousOperationsLoadTest.java
@@ -21,8 +21,8 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.processors.cache.query.continuous.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -181,7 +181,7 @@ public class GridContinuousOperationsLoadTest {
             X.println("Starting " + threadsCnt + " generator thread(s).");
 
             // Start generator threads.
-            IgniteFuture<Long> genFut = runMultiThreadedAsync(new Callable<Object>() {
+            IgniteInternalFuture<Long> genFut = runMultiThreadedAsync(new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     byte[] val = new byte[valSize];
                     ThreadLocalRandom8 rnd = ThreadLocalRandom8.current();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java b/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java
index a8d25a7..b988dae 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java
@@ -20,6 +20,7 @@ package org.apache.ignite.loadtests.job;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -57,8 +58,8 @@ public class GridJobExecutionLoadTestClientSemaphore implements Callable<Object>
     /** {@inheritDoc} */
     @SuppressWarnings("InfiniteLoopStatement")
     @Nullable @Override public Object call() throws Exception {
-        final IgniteInClosure<IgniteFuture<?>> lsnr = new CI1<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> t) {
+        final IgniteInClosure<IgniteInternalFuture<?>> lsnr = new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> t) {
                 tasksSem.release();
             }
         };

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionSingleNodeSemaphoreLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionSingleNodeSemaphoreLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionSingleNodeSemaphoreLoadTest.java
index 90267c8..83603ab 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionSingleNodeSemaphoreLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/job/GridJobExecutionSingleNodeSemaphoreLoadTest.java
@@ -20,6 +20,7 @@ package org.apache.ignite.loadtests.job;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -93,7 +94,7 @@ public class GridJobExecutionSingleNodeSemaphoreLoadTest {
 
                 X.println("Running main test.");
 
-                IgniteFuture<Void> collectorFut = GridTestUtils.runAsync(new Callable<Void>() {
+                IgniteInternalFuture<Void> collectorFut = GridTestUtils.runAsync(new Callable<Void>() {
                     @Override public Void call() throws Exception {
                         GridCumulativeAverage avgTasksPerSec = new GridCumulativeAverage();
 
@@ -163,8 +164,8 @@ public class GridJobExecutionSingleNodeSemaphoreLoadTest {
         final LongAdder iterCntr) {
         final Semaphore sem = new Semaphore(taskCnt);
 
-        final IgniteInClosure<IgniteFuture> lsnr = new CI1<IgniteFuture>() {
-            @Override public void apply(IgniteFuture t) {
+        final IgniteInClosure<IgniteInternalFuture> lsnr = new CI1<IgniteInternalFuture>() {
+            @Override public void apply(IgniteInternalFuture t) {
                 sem.release();
             }
         };

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/mergesort/GridMergeSortLoadTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/mergesort/GridMergeSortLoadTask.java b/modules/core/src/test/java/org/apache/ignite/loadtests/mergesort/GridMergeSortLoadTask.java
index 45dd14f..116dba4 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/mergesort/GridMergeSortLoadTask.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/mergesort/GridMergeSortLoadTask.java
@@ -19,7 +19,7 @@ package org.apache.ignite.loadtests.mergesort;
 
 import org.apache.ignite.*;
 import org.apache.ignite.compute.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.internal.util.typedef.*;
 
@@ -77,8 +77,8 @@ public class GridMergeSortLoadTask extends ComputeTaskSplitAdapter<int[], int[]>
 
                         // Add a listener to the future, that will resume the
                         // parent task once the child one is completed.
-                        fut.listenAsync(new CI1<IgniteFuture<int[]>>() {
-                            @Override public void apply(IgniteFuture<int[]> fut) {
+                        fut.listenAsync(new CI1<IgniteInternalFuture<int[]>>() {
+                            @Override public void apply(IgniteInternalFuture<int[]> fut) {
                                 // CONTINUATION:
                                 // =============
                                 // Resume suspended job execution.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
index 8a1e825..4ae94b5 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/offheap/GridOffHeapPartitionedMapPerformanceAbstractTest.java
@@ -19,7 +19,7 @@ package org.apache.ignite.loadtests.offheap;
 
 import org.apache.ignite.cache.affinity.*;
 import org.apache.ignite.cache.affinity.consistenthash.*;
-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.offheap.*;
@@ -168,7 +168,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 
@@ -223,7 +223,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 
@@ -294,7 +294,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 
@@ -349,7 +349,7 @@ public abstract class GridOffHeapPartitionedMapPerformanceAbstractTest extends G
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/loadtests/streamer/GridStreamerIndexLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/loadtests/streamer/GridStreamerIndexLoadTest.java b/modules/core/src/test/java/org/apache/ignite/loadtests/streamer/GridStreamerIndexLoadTest.java
index 7ae1add..d6db0a1 100644
--- a/modules/core/src/test/java/org/apache/ignite/loadtests/streamer/GridStreamerIndexLoadTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/loadtests/streamer/GridStreamerIndexLoadTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.loadtests.streamer;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.streamer.index.*;
 import org.apache.ignite.streamer.index.hash.*;
 import org.apache.ignite.streamer.index.tree.*;
@@ -114,7 +114,7 @@ public class GridStreamerIndexLoadTest {
 
         final AtomicLong enqueueCntr = new AtomicLong();
 
-        IgniteFuture<Long> enqueueFut = runMultiThreadedAsync(new CAX() {
+        IgniteInternalFuture<Long> enqueueFut = runMultiThreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 Random rnd = new Random();
 
@@ -128,7 +128,7 @@ public class GridStreamerIndexLoadTest {
 
         final AtomicLong evictCntr = new AtomicLong();
 
-        IgniteFuture<Long> evictFut = runMultiThreadedAsync(new CAX() {
+        IgniteInternalFuture<Long> evictFut = runMultiThreadedAsync(new CAX() {
             @Override public void applyx() throws IgniteCheckedException {
                 while (!Thread.currentThread().isInterrupted()) {
                     win.pollEvicted();
@@ -138,7 +138,7 @@ public class GridStreamerIndexLoadTest {
             }
         }, thrCnt, "evictor");
 
-        IgniteFuture<Long> collFut = runMultiThreadedAsync(new CAX() {
+        IgniteInternalFuture<Long> collFut = runMultiThreadedAsync(new CAX() {
             @Override public void applyx() {
                 int nSec = 0;
                 long prevEnqueue = enqueueCntr.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
index 0409da5..bd6a75c 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/GridMarshallerAbstractTest.java
@@ -120,11 +120,14 @@ public abstract class GridMarshallerAbstractTest extends GridCommonAbstractTest
      */
     private static StreamerConfiguration streamerConfiguration() {
         Collection<StreamerStage> stages = F.<StreamerStage>asList(new StreamerStage() {
-            @Override public String name() {
+            @Override
+            public String name() {
                 return "name";
             }
 
-            @Nullable @Override public Map<String, Collection<?>> run(StreamerContext ctx, Collection evts) {
+            @Nullable
+            @Override
+            public Map<String, Collection<?>> run(StreamerContext ctx, Collection evts) {
                 return null;
             }
         });
@@ -132,7 +135,7 @@ public abstract class GridMarshallerAbstractTest extends GridCommonAbstractTest
         StreamerConfiguration cfg = new StreamerConfiguration();
 
         cfg.setAtLeastOnce(true);
-        cfg.setWindows(F.asList((StreamerWindow)new StreamerUnboundedWindow()));
+        cfg.setWindows(F.asList((StreamerWindow) new StreamerUnboundedWindow()));
         cfg.setStages(stages);
 
         return cfg;
@@ -475,7 +478,8 @@ public abstract class GridMarshallerAbstractTest extends GridCommonAbstractTest
         final Ignite ignite = grid();
 
         GridMarshallerTestBean inBean = newTestBean(ignite.cluster().forPredicate(new IgnitePredicate<ClusterNode>() {
-            @Override public boolean apply(ClusterNode n) {
+            @Override
+            public boolean apply(ClusterNode n) {
                 return n.id().equals(ignite.cluster().localNode().id());
             }
         }));
@@ -618,7 +622,7 @@ public abstract class GridMarshallerAbstractTest extends GridCommonAbstractTest
     public void testScheduler() throws Exception {
         IgniteScheduler scheduler = grid().scheduler();
 
-        IgniteFuture<?> fut = scheduler.runLocal(new Runnable() {
+        IgniteInternalFuture<?> fut = scheduler.runLocal(new Runnable() {
             @Override public void run() {
                 // No-op.
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
index 528577e..125a438 100644
--- a/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/messaging/GridMessagingSelfTest.java
@@ -20,7 +20,7 @@ package org.apache.ignite.messaging;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 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.resources.*;
 import org.apache.ignite.spi.discovery.tcp.*;
@@ -1037,7 +1037,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         Assert.assertNull(id);
 
-        IgniteFuture<UUID> fut = msg.future();
+        IgniteInternalFuture<UUID> fut = msg.future();
 
         Assert.assertNotNull(fut);
 
@@ -1065,7 +1065,7 @@ public class GridMessagingSelfTest extends GridCommonAbstractTest {
 
         msg.stopRemoteListen(id);
 
-        IgniteFuture<?> stopFut = msg.future();
+        IgniteInternalFuture<?> stopFut = msg.future();
 
         Assert.assertNotNull(stopFut);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultiThreadedSelfTest.java
index a202085..407ba8c 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultiThreadedSelfTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.spi.checkpoint.sharedfs;
 
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.*;
 import org.apache.ignite.testframework.junits.spi.*;
@@ -66,7 +66,7 @@ public class GridSharedFsCheckpointSpiMultiThreadedSelfTest extends
 
         final AtomicBoolean fail = new AtomicBoolean();
 
-        IgniteFuture fut1 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture fut1 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     try {
@@ -86,7 +86,7 @@ public class GridSharedFsCheckpointSpiMultiThreadedSelfTest extends
             "writer-1"
         );
 
-        IgniteFuture fut2 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture fut2 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     try{
@@ -106,7 +106,7 @@ public class GridSharedFsCheckpointSpiMultiThreadedSelfTest extends
             "writer-2"
         );
 
-        IgniteFuture fut3 = GridTestUtils.runMultiThreadedAsync(
+        IgniteInternalFuture fut3 = GridTestUtils.runMultiThreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     while (writeFinished.get() < THREAD_CNT * 2) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/spi/collision/GridTestCollisionTaskSession.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/collision/GridTestCollisionTaskSession.java b/modules/core/src/test/java/org/apache/ignite/spi/collision/GridTestCollisionTaskSession.java
index 9e97fae..a70b867 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/collision/GridTestCollisionTaskSession.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/collision/GridTestCollisionTaskSession.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.collision;
 
 import org.apache.ignite.*;
 import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 
 import java.util.*;
@@ -197,7 +198,7 @@ public class GridTestCollisionTaskSession implements ComputeTaskSession {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> mapFuture() {
+    @Override public IgniteInternalFuture<?> mapFuture() {
         assert false : "Not implemented";
 
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
index a4362af..e35da3e 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiConcurrentConnectSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.communication.tcp;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.communication.*;
 import org.eclipse.jetty.util.*;
@@ -174,7 +175,7 @@ public class GridTcpCommunicationSpiConcurrentConnectSelfTest<T extends Communic
 
         final AtomicBoolean stop = new AtomicBoolean();
 
-        IgniteFuture<?> loadFut = null;
+        IgniteInternalFuture<?> loadFut = null;
 
         if (load) {
             loadFut = GridTestUtils.runMultiThreadedAsync(new Callable<Long>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java
index 763a14b..bd3bec5 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.communication.tcp;
 
 import mx4j.tools.adaptor.http.*;
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.communication.*;
 import org.apache.ignite.internal.util.direct.*;
@@ -165,7 +166,7 @@ public class GridTcpCommunicationSpiLanTest extends GridSpiAbstractTest<TcpCommu
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     while (cntr.getAndIncrement() < msgCnt) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index dc9ee56..ac49e45 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.communication.tcp;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.communication.*;
 import org.apache.ignite.internal.util.direct.*;
@@ -172,7 +173,7 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             /** Randomizer. */
             private Random rnd = new Random();
 
@@ -269,7 +270,7 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
 
         final int interval = 50;
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             /** {@inheritDoc} */
             @Override public void run() {
                 try {
@@ -308,7 +309,7 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
 
         final AtomicBoolean run = new AtomicBoolean(true);
 
-        IgniteFuture<?> fut2 = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut2 = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     while (run.get() && !Thread.currentThread().isInterrupted()) {
@@ -368,7 +369,7 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
 
         long start = System.currentTimeMillis();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Runnable() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Runnable() {
             @Override public void run() {
                 try {
                     ClusterNode from = nodes.get(0);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
index 6daf7a6..b90242f 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiRecoverySelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.spi.communication.tcp;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.communication.*;
@@ -281,7 +282,7 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     ses1.pauseReads().get();
 
-                    IgniteFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
+                    IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
                         @Override public Void call() throws Exception {
                             for (int i = 0; i < 5000; i++) {
                                 spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
@@ -391,7 +392,7 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     ses1.pauseReads().get();
 
-                    IgniteFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
+                    IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
                         @Override public Void call() throws Exception {
                             for (int i = 0; i < 5000; i++) {
                                 spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));
@@ -508,7 +509,7 @@ public class GridTcpCommunicationSpiRecoverySelfTest<T extends CommunicationSpi>
 
                     ses1.pauseReads().get();
 
-                    IgniteFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
+                    IgniteInternalFuture<?> sndFut = GridTestUtils.runAsync(new Callable<Void>() {
                         @Override public Void call() throws Exception {
                             for (int i = 0; i < 5000; i++) {
                                 spi0.sendMessage(node1, new GridTestMessage(node0.id(), msgId.incrementAndGet(), 0));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java
index f4dd773..e81fcba 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/discovery/tcp/GridTcpDiscoveryMultiThreadedTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.spi.discovery.tcp;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -154,7 +153,7 @@ public class GridTcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
 
         final AtomicInteger clientIdx = new AtomicInteger(GRID_CNT);
 
-        IgniteFuture<?> fut1 = multithreadedAsync(
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(
             new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     clientFlagPerThread.set(true);
@@ -177,7 +176,7 @@ public class GridTcpDiscoveryMultiThreadedTest extends GridCommonAbstractTest {
         for (int i = 0; i < GRID_CNT; i++)
             srvIdx.add(i);
 
-        IgniteFuture<?> fut2 = multithreadedAsync(
+        IgniteInternalFuture<?> fut2 = multithreadedAsync(
             new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     clientFlagPerThread.set(false);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
index 16cf0ce..378eee4 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/swapspace/file/GridFileSwapSpaceSpiSelfTest.java
@@ -17,6 +17,7 @@
 
 package org.apache.ignite.spi.swapspace.file;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
 import org.apache.ignite.spi.swapspace.*;
@@ -57,7 +58,7 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
 
         final AtomicBoolean done = new AtomicBoolean();
 
-        IgniteFuture<?> wFut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> wFut = multithreadedAsync(new Callable<Object>() {
             @Nullable @Override public Object call() throws Exception {
                 while (!done.get()) {
                     long val = valCntr.incrementAndGet();
@@ -74,7 +75,7 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
 
         wLatch.await();
 
-        IgniteFuture<?> rFut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> rFut = multithreadedAsync(new Callable<Object>() {
             @Nullable @Override public Object call() throws Exception {
                 while (valCntr.get() < 1000) {
                     byte[] val = spi.read(null, key, context());
@@ -139,7 +140,7 @@ public class GridFileSwapSpaceSpiSelfTest extends GridSwapSpaceSpiAbstractSelfTe
 
         final AtomicBoolean fin = new AtomicBoolean();
 
-        final IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        final IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesTest.java b/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesTest.java
index f70ea1b..f8c7fc6 100644
--- a/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/storevalbytes/GridCacheStoreValueBytesTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.storevalbytes;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -201,8 +202,8 @@ public class GridCacheStoreValueBytesTest {
 
         final Semaphore sem = new Semaphore(concurrentGetNum);
 
-        final IgniteInClosure<IgniteFuture> lsnr = new CI1<IgniteFuture>() {
-            @Override public void apply(IgniteFuture t) {
+        final IgniteInClosure<IgniteInternalFuture> lsnr = new CI1<IgniteInternalFuture>() {
+            @Override public void apply(IgniteInternalFuture t) {
                 sem.release();
             }
         };
@@ -231,7 +232,7 @@ public class GridCacheStoreValueBytesTest {
                             if (keys.size() == getKeyNum) {
                                 sem.acquire();
 
-                                IgniteFuture<Map<Integer, String>> f = cache.getAllAsync(keys);
+                                IgniteInternalFuture<Map<Integer, String>> f = cache.getAllAsync(keys);
 
                                 f.listenAsync(lsnr);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java b/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java
index e7692f0..aed1e33 100644
--- a/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.streamer.index;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.streamer.index.hash.*;
 import org.apache.ignite.streamer.index.tree.*;
 import org.apache.ignite.streamer.window.*;
@@ -296,7 +296,7 @@ public class GridStreamerIndexSelfTest extends GridCommonAbstractTest {
 
         win.start();
 
-        IgniteFuture<Long> pollFut = null;
+        IgniteInternalFuture<Long> pollFut = null;
 
         if (pollEvicted) {
             // These threads poll evicted events from the window if it doesn't break
@@ -331,7 +331,7 @@ public class GridStreamerIndexSelfTest extends GridCommonAbstractTest {
             // if it is still present in the window. In the tested index events are
             // sorted by value and the value is a number of repeated events, so, this
             // should be invariant.
-            IgniteFuture<Long> fut1 = runMultiThreadedAsync(new CAX() {
+            IgniteInternalFuture<Long> fut1 = runMultiThreadedAsync(new CAX() {
                 @Override public void applyx() throws IgniteCheckedException {
                     final String evt = Thread.currentThread().getName();
                     int cntr = 1;
@@ -358,7 +358,7 @@ public class GridStreamerIndexSelfTest extends GridCommonAbstractTest {
             }, threadCnt / 2, "test-multi");
 
             // This thread generates a set of single non-repeating events from 0 to iters.
-            IgniteFuture<Long> fut2 = runMultiThreadedAsync(new CAX() {
+            IgniteInternalFuture<Long> fut2 = runMultiThreadedAsync(new CAX() {
                 @Override public void applyx() throws IgniteCheckedException {
                     for (int i = 0; i < iters && !Thread.currentThread().isInterrupted(); i++)
                         win.enqueue(String.valueOf(i));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/streamer/window/GridStreamerWindowSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/streamer/window/GridStreamerWindowSelfTest.java b/modules/core/src/test/java/org/apache/ignite/streamer/window/GridStreamerWindowSelfTest.java
index cd164bb..2aca81c 100644
--- a/modules/core/src/test/java/org/apache/ignite/streamer/window/GridStreamerWindowSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/streamer/window/GridStreamerWindowSelfTest.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.streamer.window;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.streamer.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -792,7 +792,7 @@ public class GridStreamerWindowSelfTest extends GridCommonAbstractTest {
 
         final AtomicInteger added = new GridAtomicInteger();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
index 2bce759..433034d 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/GridTestUtils.java
@@ -455,7 +455,7 @@ public final class GridTestUtils {
      * @param threadName Thread name.
      * @return Future for the run. Future returns execution time in milliseconds.
      */
-    public static IgniteFuture<Long> runMultiThreadedAsync(Runnable run, int threadNum, String threadName) {
+    public static IgniteInternalFuture<Long> runMultiThreadedAsync(Runnable run, int threadNum, String threadName) {
         return runMultiThreadedAsync(makeCallable(run, null), threadNum, threadName);
     }
 
@@ -483,7 +483,7 @@ public final class GridTestUtils {
      * @return Future for the run. Future returns execution time in milliseconds.
      */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
-    public static IgniteFuture<Long> runMultiThreadedAsync(Callable<?> call, int threadNum, final String threadName) {
+    public static IgniteInternalFuture<Long> runMultiThreadedAsync(Callable<?> call, int threadNum, final String threadName) {
         final List<Callable<?>> calls = Collections.<Callable<?>>nCopies(threadNum, call);
         final GridTestSafeThreadFactory threadFactory = new GridTestSafeThreadFactory(threadName);
 
@@ -503,7 +503,7 @@ public final class GridTestUtils {
         };
 
         // Async execution future (doesn't support cancel()).
-        IgniteFuture<Long> runFut = runAsync(new Callable<Long>() {
+        IgniteInternalFuture<Long> runFut = runAsync(new Callable<Long>() {
             @Override public Long call() throws Exception {
                 return runMultiThreaded(calls, threadFactory);
             }
@@ -581,7 +581,7 @@ public final class GridTestUtils {
      * @return Future with task result.
      */
     @SuppressWarnings("ExternalizableWithoutPublicNoArgConstructor")
-    public static <T> IgniteFuture<T> runAsync(final Callable<T> task) {
+    public static <T> IgniteInternalFuture<T> runAsync(final Callable<T> task) {
         if (!busyLock.enterBusy())
             throw new IllegalStateException("Failed to start new threads (test is being stopped).");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
index 4153c61..4a4ca95 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/GridAbstractTest.java
@@ -297,7 +297,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      * @return Future.
      */
-    protected IgniteFuture<?> multithreadedAsync(Runnable r, int threadNum) throws Exception {
+    protected IgniteInternalFuture<?> multithreadedAsync(Runnable r, int threadNum) throws Exception {
         return multithreadedAsync(r, threadNum, getTestGridName());
     }
 
@@ -312,7 +312,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      * @return Future.
      */
-    protected IgniteFuture<?> multithreadedAsync(Runnable r, int threadNum, String threadName) throws Exception {
+    protected IgniteInternalFuture<?> multithreadedAsync(Runnable r, int threadNum, String threadName) throws Exception {
         return GridTestUtils.runMultiThreadedAsync(r, threadNum, threadName);
     }
 
@@ -350,7 +350,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      * @return Future.
      */
-    protected IgniteFuture<?> multithreadedAsync(Callable<?> c, int threadNum) throws Exception {
+    protected IgniteInternalFuture<?> multithreadedAsync(Callable<?> c, int threadNum) throws Exception {
         return multithreadedAsync(c, threadNum, getTestGridName());
     }
 
@@ -364,7 +364,7 @@ public abstract class GridAbstractTest extends TestCase {
      * @throws Exception If failed.
      * @return Future.
      */
-    protected IgniteFuture<?> multithreadedAsync(Callable<?> c, int threadNum, String threadName) throws Exception {
+    protected IgniteInternalFuture<?> multithreadedAsync(Callable<?> c, int threadNum, String threadName) throws Exception {
         return GridTestUtils.runMultiThreadedAsync(c, threadNum, threadName);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/GridAbstractCacheStoreSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/GridAbstractCacheStoreSelfTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/GridAbstractCacheStoreSelfTest.java
index bb9e502..b891646 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/GridAbstractCacheStoreSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/cache/GridAbstractCacheStoreSelfTest.java
@@ -19,6 +19,7 @@ package org.apache.ignite.testframework.junits.cache;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cache.store.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.transactions.*;
@@ -551,7 +552,7 @@ public abstract class GridAbstractCacheStoreSelfTest<T extends CacheStore<Object
         }
 
         /** {@inheritDoc} */
-        @Override public <R> IgniteFuture<R> future() {
+        @Override public <R> IgniteInternalFuture<R> future() {
             return null;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
index c8f3d78..b5d7c94 100644
--- a/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/testframework/junits/common/GridCommonAbstractTest.java
@@ -611,7 +611,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
      * @return Future.
      * @throws IgniteCheckedException If failed.
      */
-    protected <T extends IgniteEvent> IgniteFuture<T> waitForLocalEvent(IgniteEvents evts,
+    protected <T extends IgniteEvent> IgniteInternalFuture<T> waitForLocalEvent(IgniteEvents evts,
         @Nullable IgnitePredicate<T> filter, @Nullable int... types) throws IgniteCheckedException {
         evts = evts.withAsync();
 
@@ -619,7 +619,7 @@ public abstract class GridCommonAbstractTest extends GridAbstractTest {
 
         assertNull(evts.waitForLocal(filter, types));
 
-        IgniteFuture<T> fut = evts.future();
+        IgniteInternalFuture<T> fut = evts.future();
 
         assertNotNull(fut);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedDequeMultiThreadedTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedDequeMultiThreadedTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedDequeMultiThreadedTest.java
index ee32891..5422445 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedDequeMultiThreadedTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedDequeMultiThreadedTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.util;
 
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.jdk8.backport.*;
 import org.jdk8.backport.ConcurrentLinkedDeque8.*;
 import org.apache.ignite.testframework.junits.common.*;
@@ -44,7 +44,7 @@ public class GridConcurrentLinkedDequeMultiThreadedTest extends GridCommonAbstra
         final ConcurrentLinkedDeque8<Byte> queue = new ConcurrentLinkedDeque8<>();
 
         // Poll thread.
-        IgniteFuture<?> pollFut = multithreadedAsync(
+        IgniteInternalFuture<?> pollFut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     info("Thread started.");
@@ -69,7 +69,7 @@ public class GridConcurrentLinkedDequeMultiThreadedTest extends GridCommonAbstra
         );
 
         // Producer thread.
-        IgniteFuture<?> prodFut = multithreadedAsync(
+        IgniteInternalFuture<?> prodFut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     info("Thread started.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java
index fb896ae..7d68b07 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridConcurrentLinkedHashMapMultiThreadedSelfTest.java
@@ -17,7 +17,7 @@
 
 package org.apache.ignite.util;
 
-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.*;
 import org.jdk8.backport.*;
@@ -269,7 +269,7 @@ public class GridConcurrentLinkedHashMapMultiThreadedSelfTest extends GridCommon
         final ConcurrentHashMap<String, LinkedList<Integer>> res = new ConcurrentHashMap<>();
 
         // Producer thread.
-        IgniteFuture<?> fut = multithreadedAsync(
+        IgniteInternalFuture<?> fut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     String thNm = Thread.currentThread().getName();
@@ -331,7 +331,7 @@ public class GridConcurrentLinkedHashMapMultiThreadedSelfTest extends GridCommon
         long start = System.currentTimeMillis();
 
         // Updater threads.
-        IgniteFuture<?> fut = multithreadedAsync(
+        IgniteInternalFuture<?> fut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     Random rnd = new Random();
@@ -416,7 +416,7 @@ public class GridConcurrentLinkedHashMapMultiThreadedSelfTest extends GridCommon
         long start = System.currentTimeMillis();
 
         // Updater threads.
-        IgniteFuture<?> fut = multithreadedAsync(
+        IgniteInternalFuture<?> fut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     Random rnd = new Random();
@@ -490,7 +490,7 @@ public class GridConcurrentLinkedHashMapMultiThreadedSelfTest extends GridCommon
         long start = System.currentTimeMillis();
 
         // Updater threads.
-        IgniteFuture<?> fut = multithreadedAsync(
+        IgniteInternalFuture<?> fut = multithreadedAsync(
             new Callable<Object>() {
                 @Nullable @Override public Object call() throws Exception {
                     Random rnd = new Random();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/util/GridIndexFillTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridIndexFillTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridIndexFillTest.java
index e5e0a44..bd00326 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridIndexFillTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridIndexFillTest.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.util;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.snaptree.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -116,7 +116,7 @@ public class GridIndexFillTest extends GridCommonAbstractTest {
     public void testSnaptreeParallelBuild() throws Exception {
         final AtomicBoolean stop = new AtomicBoolean();
 
-        IgniteFuture<?> fut = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 ThreadLocalRandom rnd = ThreadLocalRandom.current();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/util/GridSnapshotLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridSnapshotLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridSnapshotLockSelfTest.java
index cb4ac26..dd9a17a 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridSnapshotLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridSnapshotLockSelfTest.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.util;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -54,7 +54,7 @@ public class GridSnapshotLockSelfTest extends GridCommonAbstractTest {
             }
         };
 
-        IgniteFuture<?> fut1 = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut1 = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 Random rnd = new Random();
 
@@ -82,7 +82,7 @@ public class GridSnapshotLockSelfTest extends GridCommonAbstractTest {
             }
         }, 15, "update");
 
-        IgniteFuture<?> fut2 = multithreadedAsync(new Callable<Object>() {
+        IgniteInternalFuture<?> fut2 = multithreadedAsync(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 while(!stop.get()) {
                     T3<Long, Long, Long> t;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/core/src/test/java/org/apache/ignite/util/GridSpinReadWriteLockSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/util/GridSpinReadWriteLockSelfTest.java b/modules/core/src/test/java/org/apache/ignite/util/GridSpinReadWriteLockSelfTest.java
index 6c16ddd..18df558 100644
--- a/modules/core/src/test/java/org/apache/ignite/util/GridSpinReadWriteLockSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/util/GridSpinReadWriteLockSelfTest.java
@@ -17,8 +17,8 @@
 
 package org.apache.ignite.util;
 
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 import org.apache.ignite.testframework.junits.common.*;
 
@@ -58,7 +58,7 @@ public class GridSpinReadWriteLockSelfTest extends GridCommonAbstractTest {
 
         final CountDownLatch latch = new CountDownLatch(1);
 
-        IgniteFuture<?> f = multithreadedAsync(
+        IgniteInternalFuture<?> f = multithreadedAsync(
             new Callable<Object>() {
                 @Override public Object call() throws Exception {
                     assert !lock.tryWriteLock();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/email/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessor.java
----------------------------------------------------------------------
diff --git a/modules/email/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessor.java b/modules/email/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessor.java
index 0fa56cc..57acecb 100644
--- a/modules/email/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessor.java
+++ b/modules/email/src/main/java/org/apache/ignite/internal/processors/email/IgniteEmailProcessor.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.email;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.thread.*;
 import org.apache.ignite.internal.util.future.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
@@ -165,7 +164,7 @@ public class IgniteEmailProcessor extends IgniteEmailProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<Boolean> schedule(String subj, String body, boolean html) {
+    @Override public IgniteInternalFuture<Boolean> schedule(String subj, String body, boolean html) {
         String[] addrs = ctx.config().getAdminEmails();
 
         return addrs == null || addrs.length == 0 ? new GridFinishedFuture<>(ctx, false) :
@@ -174,7 +173,7 @@ public class IgniteEmailProcessor extends IgniteEmailProcessorAdapter {
 
     /** {@inheritDoc} */
     @SuppressWarnings({"SynchronizeOnNonFinalField"})
-    @Override public IgniteFuture<Boolean> schedule(String subj, String body, boolean html, Collection<String> addrs) {
+    @Override public IgniteInternalFuture<Boolean> schedule(String subj, String body, boolean html, Collection<String> addrs) {
         assert subj != null;
         assert body != null;
         assert addrs != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/main/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolProvider.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolProvider.java b/modules/hadoop/src/main/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolProvider.java
index 5be8d5f..29967c7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolProvider.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/client/hadoop/GridHadoopClientProtocolProvider.java
@@ -21,7 +21,7 @@ import org.apache.hadoop.conf.*;
 import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.protocol.*;
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.client.*;
 import org.apache.ignite.client.marshaller.optimized.*;
 import org.apache.ignite.internal.util.future.*;
@@ -41,7 +41,7 @@ import static org.apache.ignite.client.hadoop.GridHadoopClientProtocol.*;
  */
 public class GridHadoopClientProtocolProvider extends ClientProtocolProvider {
     /** Clients. */
-    private static final ConcurrentHashMap<String, IgniteFuture<GridClient>> cliMap = new ConcurrentHashMap<>();
+    private static final ConcurrentHashMap<String, IgniteInternalFuture<GridClient>> cliMap = new ConcurrentHashMap<>();
 
     /** {@inheritDoc} */
     @Override public ClientProtocol create(Configuration conf) throws IOException {
@@ -96,12 +96,12 @@ public class GridHadoopClientProtocolProvider extends ClientProtocolProvider {
      */
     private static GridClient client(String addr) throws IOException {
         try {
-            IgniteFuture<GridClient> fut = cliMap.get(addr);
+            IgniteInternalFuture<GridClient> fut = cliMap.get(addr);
 
             if (fut == null) {
                 GridFutureAdapter<GridClient> fut0 = new GridFutureAdapter<>();
 
-                IgniteFuture<GridClient> oldFut = cliMap.putIfAbsent(addr, fut0);
+                IgniteInternalFuture<GridClient> oldFut = cliMap.putIfAbsent(addr, fut0);
 
                 if (oldFut != null)
                     return oldFut.get();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopImpl.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopImpl.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopImpl.java
index 8c48f5e..79f099d 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopImpl.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/GridHadoopImpl.java
@@ -18,8 +18,8 @@
 package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.util.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.hadoop.*;
 import org.jetbrains.annotations.*;
 
@@ -62,7 +62,7 @@ public class GridHadoopImpl implements GridHadoop {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
+    @Override public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
         if (busyLock.enterBusy()) {
             try {
                 return proc.submit(jobId, jobInfo);
@@ -104,7 +104,7 @@ public class GridHadoopImpl implements GridHadoop {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
+    @Nullable @Override public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
         if (busyLock.enterBusy()) {
             try {
                 return proc.finishFuture(jobId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
index cd5027e..c0c8e23 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/IgniteHadoopProcessor.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal.processors.hadoop;
 
 import org.apache.ignite.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.jobtracker.*;
 import org.apache.ignite.internal.processors.hadoop.planner.*;
@@ -179,7 +178,7 @@ public class IgniteHadoopProcessor extends IgniteHadoopProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
+    @Override public IgniteInternalFuture<?> submit(GridHadoopJobId jobId, GridHadoopJobInfo jobInfo) {
         return hctx.jobTracker().submit(jobId, jobInfo);
     }
 
@@ -194,7 +193,7 @@ public class IgniteHadoopProcessor extends IgniteHadoopProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
+    @Override public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
         return hctx.jobTracker().finishFuture(jobId);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobTracker.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobTracker.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobTracker.java
index 0711ebd..6e35f47 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobTracker.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/jobtracker/GridHadoopJobTracker.java
@@ -21,6 +21,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cache.*;
 import org.apache.ignite.cache.query.*;
 import org.apache.ignite.events.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.internal.util.*;
 import org.apache.ignite.lang.*;
@@ -82,8 +83,8 @@ public class GridHadoopJobTracker extends GridHadoopComponent {
     private GridSpinReadWriteLock busyLock;
 
     /** Closure to check result of async transform of system cache. */
-    private final IgniteInClosure<IgniteFuture<?>> failsLog = new CI1<IgniteFuture<?>>() {
-        @Override public void apply(IgniteFuture<?> gridFut) {
+    private final IgniteInClosure<IgniteInternalFuture<?>> failsLog = new CI1<IgniteInternalFuture<?>>() {
+        @Override public void apply(IgniteInternalFuture<?> gridFut) {
             try {
                 gridFut.get();
             }
@@ -237,7 +238,7 @@ public class GridHadoopJobTracker extends GridHadoopComponent {
      * @return Job completion future.
      */
     @SuppressWarnings("unchecked")
-    public IgniteFuture<GridHadoopJobId> submit(GridHadoopJobId jobId, GridHadoopJobInfo info) {
+    public IgniteInternalFuture<GridHadoopJobId> submit(GridHadoopJobId jobId, GridHadoopJobInfo info) {
         if (!busyLock.tryReadLock()) {
             return new GridFinishedFutureEx<>(new IgniteCheckedException("Failed to execute map-reduce job " +
                 "(grid is stopping): " + info));
@@ -344,7 +345,7 @@ public class GridHadoopJobTracker extends GridHadoopComponent {
      * @return Finish future or {@code null}.
      * @throws IgniteCheckedException If failed.
      */
-    @Nullable public IgniteFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
+    @Nullable public IgniteInternalFuture<?> finishFuture(GridHadoopJobId jobId) throws IgniteCheckedException {
         if (!busyLock.tryReadLock())
             return null; // Grid is stopping.
 
@@ -1038,7 +1039,7 @@ public class GridHadoopJobTracker extends GridHadoopComponent {
             busyLock.readUnlock();
         }
 
-        IgniteFuture<?> fut = finishFuture(jobId);
+        IgniteInternalFuture<?> fut = finishFuture(jobId);
 
         if (fut != null) {
             try {
@@ -1187,8 +1188,8 @@ public class GridHadoopJobTracker extends GridHadoopComponent {
                 return;
             }
 
-            IgniteInClosure<IgniteFuture<?>> cacheUpdater = new CIX1<IgniteFuture<?>>() {
-                @Override public void applyx(IgniteFuture<?> f) {
+            IgniteInClosure<IgniteInternalFuture<?>> cacheUpdater = new CIX1<IgniteInternalFuture<?>>() {
+                @Override public void applyx(IgniteInternalFuture<?> f) {
                     Throwable err = null;
 
                     if (f != null) {
@@ -1237,8 +1238,8 @@ public class GridHadoopJobTracker extends GridHadoopComponent {
                 // Fail the whole job.
                 transform(jobId, new RemoveMappersProcessor(prev, currMappers, status.failCause()));
             else {
-                ctx.shuffle().flush(jobId).listenAsync(new CIX1<IgniteFuture<?>>() {
-                    @Override public void applyx(IgniteFuture<?> f) {
+                ctx.shuffle().flush(jobId).listenAsync(new CIX1<IgniteInternalFuture<?>>() {
+                    @Override public void applyx(IgniteInternalFuture<?> f) {
                         Throwable err = null;
 
                         if (f != null) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobStatusTask.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobStatusTask.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobStatusTask.java
index f4f4da8..19adf3b 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobStatusTask.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/proto/GridHadoopProtocolJobStatusTask.java
@@ -19,6 +19,7 @@ package org.apache.ignite.internal.processors.hadoop.proto;
 
 import org.apache.ignite.*;
 import org.apache.ignite.compute.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.internal.util.typedef.*;
@@ -54,14 +55,14 @@ public class GridHadoopProtocolJobStatusTask extends GridHadoopProtocolTaskAdapt
             pollDelay = DFLT_POLL_DELAY;
 
         if (pollDelay > 0) {
-            IgniteFuture<?> fut = hadoop.finishFuture(jobId);
+            IgniteInternalFuture<?> fut = hadoop.finishFuture(jobId);
 
             if (fut != null) {
                 if (fut.isDone() || F.eq(jobCtx.getAttribute(ATTR_HELD), true))
                     return hadoop.status(jobId);
                 else {
-                    fut.listenAsync(new IgniteInClosure<IgniteFuture<?>>() {
-                        @Override public void apply(IgniteFuture<?> fut0) {
+                    fut.listenAsync(new IgniteInClosure<IgniteInternalFuture<?>>() {
+                        @Override public void apply(IgniteInternalFuture<?> fut0) {
                             jobCtx.callcc();
                         }
                     });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffle.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffle.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffle.java
index 087ae88..c186953 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffle.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffle.java
@@ -234,7 +234,7 @@ public class GridHadoopShuffle extends GridHadoopComponent {
      * @param jobId Job ID.
      * @return Future.
      */
-    public IgniteFuture<?> flush(GridHadoopJobId jobId) {
+    public IgniteInternalFuture<?> flush(GridHadoopJobId jobId) {
         GridHadoopShuffleJob job = jobs.get(jobId);
 
         if (job == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleJob.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleJob.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleJob.java
index 2055941..54ef6c7 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleJob.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/shuffle/GridHadoopShuffleJob.java
@@ -18,6 +18,7 @@
 package org.apache.ignite.internal.processors.hadoop.shuffle;
 
 import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.thread.*;
 import org.apache.ignite.hadoop.*;
@@ -382,8 +383,8 @@ public class GridHadoopShuffleJob<T> implements AutoCloseable {
             fut.onDone(U.unwrap(e));
         }
 
-        fut.listenAsync(new IgniteInClosure<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> f) {
+        fut.listenAsync(new IgniteInClosure<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> f) {
                 try {
                     f.get();
 
@@ -433,7 +434,7 @@ public class GridHadoopShuffleJob<T> implements AutoCloseable {
      * @return Future.
      */
     @SuppressWarnings("unchecked")
-    public IgniteFuture<?> flush() throws IgniteCheckedException {
+    public IgniteInternalFuture<?> flush() throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Flushing job " + job.id() + " on address " + locReduceAddr);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutor.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutor.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutor.java
index f7035ad..d1c4bf0 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutor.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/GridHadoopExternalTaskExecutor.java
@@ -157,8 +157,8 @@ public class GridHadoopExternalTaskExecutor extends GridHadoopTaskExecutorAdapte
                         "[jobId=" + meta.jobId() + ", meta=" + meta + ']');
             }
             else {
-                proc.initFut.listenAsync(new CI1<IgniteFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
-                    @Override public void apply(IgniteFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+                    @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
                         try {
                             f.get();
 
@@ -224,9 +224,9 @@ public class GridHadoopExternalTaskExecutor extends GridHadoopTaskExecutorAdapte
 
             final HadoopProcess proc0 = proc;
 
-            proc.initFut.listenAsync(new CI1<IgniteFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+            proc.initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
                 @Override public void apply(
-                    IgniteFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                    IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
                     if (!busyLock.tryReadLock())
                         return;
 
@@ -405,8 +405,8 @@ public class GridHadoopExternalTaskExecutor extends GridHadoopTaskExecutorAdapte
             }
         }, true);
 
-        fut.listenAsync(new CI1<IgniteFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
-            @Override public void apply(IgniteFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+        fut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+            @Override public void apply(IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
                 try {
                     // Make sure there were no exceptions.
                     f.get();
@@ -790,9 +790,9 @@ public class GridHadoopExternalTaskExecutor extends GridHadoopTaskExecutorAdapte
                 terminated = true;
 
                 if (!initFut.isDone())
-                    initFut.listenAsync(new CI1<IgniteFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
+                    initFut.listenAsync(new CI1<IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>>>() {
                         @Override public void apply(
-                            IgniteFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
+                            IgniteInternalFuture<IgniteBiTuple<Process, GridHadoopProcessDescriptor>> f) {
                             proc.destroy();
                         }
                     });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
index 2c2f59d..4408b9a 100644
--- a/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
+++ b/modules/hadoop/src/main/java/org/apache/ignite/internal/processors/hadoop/taskexecutor/external/child/GridHadoopChildProcessRunner.java
@@ -18,7 +18,7 @@
 package org.apache.ignite.internal.processors.hadoop.taskexecutor.external.child;
 
 import org.apache.ignite.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.message.*;
 import org.apache.ignite.internal.processors.hadoop.shuffle.*;
@@ -147,8 +147,8 @@ public class GridHadoopChildProcessRunner {
         if (!initFut.isDone() && log.isDebugEnabled())
             log.debug("Will wait for process initialization future completion: " + req);
 
-        initFut.listenAsync(new CI1<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> f) {
+        initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> f) {
                 try {
                     // Make sure init was successful.
                     f.get();
@@ -219,8 +219,8 @@ public class GridHadoopChildProcessRunner {
      * @param req Update request.
      */
     private void updateTasks(final GridHadoopJobInfoUpdateRequest req) {
-        initFut.listenAsync(new CI1<IgniteFuture<?>>() {
-            @Override public void apply(IgniteFuture<?> gridFut) {
+        initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+            @Override public void apply(IgniteInternalFuture<?> gridFut) {
                 assert initGuard.get();
 
                 assert req.jobId().equals(job.id());
@@ -316,8 +316,8 @@ public class GridHadoopChildProcessRunner {
             final long start = U.currentTimeMillis();
 
             try {
-                shuffleJob.flush().listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> f) {
+                shuffleJob.flush().listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> f) {
                         long end = U.currentTimeMillis();
 
                         if (log.isDebugEnabled())
@@ -396,8 +396,8 @@ public class GridHadoopChildProcessRunner {
                 if (log.isTraceEnabled())
                     log.trace("Received shuffle message [desc=" + desc + ", msg=" + msg + ']');
 
-                initFut.listenAsync(new CI1<IgniteFuture<?>>() {
-                    @Override public void apply(IgniteFuture<?> f) {
+                initFut.listenAsync(new CI1<IgniteInternalFuture<?>>() {
+                    @Override public void apply(IgniteInternalFuture<?> f) {
                         try {
                             GridHadoopShuffleMessage m = (GridHadoopShuffleMessage)msg;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
index 390e630..5688500 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopDefaultMapReducePlannerSelfTest.java
@@ -734,7 +734,7 @@ public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
+        @Override public IgniteInternalFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
             return null;
         }
 
@@ -921,7 +921,7 @@ public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public <R> IgniteFuture<R> future() {
+        @Override public <R> IgniteInternalFuture<R> future() {
             return null;
         }
     }
@@ -1001,7 +1001,7 @@ public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<Boolean> sendAdminEmailAsync(String subj, String body, boolean html) {
+        @Override public IgniteInternalFuture<Boolean> sendAdminEmailAsync(String subj, String body, boolean html) {
             return null;
         }
 
@@ -1215,7 +1215,7 @@ public class GridHadoopDefaultMapReducePlannerSelfTest extends GridHadoopAbstrac
         }
 
         /** {@inheritDoc} */
-        @Override public <R> IgniteFuture<R> future() {
+        @Override public <R> IgniteInternalFuture<R> future() {
             return null;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
index 1c44282..85f55d8 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopJobTrackerSelfTest.java
@@ -22,7 +22,6 @@ import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.FileSplit;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.lang.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.internal.util.typedef.internal.*;
 
@@ -225,7 +224,7 @@ public class GridHadoopJobTrackerSelfTest extends GridHadoopAbstractSelfTest {
 
             assert stat != null;
 
-            IgniteFuture<?> fut = hadoop.finishFuture(jobId);
+            IgniteInternalFuture<?> fut = hadoop.finishFuture(jobId);
 
             if (!complete)
                 assertFalse(fut.isDone());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/d5bef132/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java
index 7dc1cfe..4d8d610 100644
--- a/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java
+++ b/modules/hadoop/src/test/java/org/apache/ignite/internal/processors/hadoop/GridHadoopMapReduceEmbeddedSelfTest.java
@@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.*;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
 import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
 import org.apache.ignite.fs.*;
-import org.apache.ignite.lang.*;
+import org.apache.ignite.internal.*;
 import org.apache.ignite.hadoop.*;
 import org.apache.ignite.internal.processors.hadoop.examples.*;
 
@@ -117,7 +117,7 @@ public class GridHadoopMapReduceEmbeddedSelfTest extends GridHadoopMapReduceTest
 
             job.setJarByClass(GridHadoopWordCount2.class);
 
-            IgniteFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
+            IgniteInternalFuture<?> fut = grid(0).hadoop().submit(new GridHadoopJobId(UUID.randomUUID(), 1),
                     createJobInfo(job.getConfiguration()));
 
             fut.get();


[37/50] [abbrv] incubator-ignite git commit: GridEx -> IgniteEx GridKernal -> IgniteKernal

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java
index c689be0..f4ba945 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/affinity/GridAffinityUtils.java
@@ -158,7 +158,7 @@ class GridAffinityUtils {
             assert ignite != null;
             assert log != null;
 
-            GridKernal kernal = ((GridKernal) ignite);
+            IgniteKernal kernal = ((IgniteKernal) ignite);
 
             GridCacheContext<Object, Object> cctx = kernal.internalCache(cacheName).context();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
index 92d2039..35b34f0 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheAdapter.java
@@ -5249,7 +5249,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         /** {@inheritDoc} */
         @Override public Object call() throws Exception {
-            ((GridEx) ignite).cachex(cacheName).clearAll();
+            ((IgniteEx) ignite).cachex(cacheName).clearAll();
 
             return null;
         }
@@ -5302,7 +5302,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         /** {@inheritDoc} */
         @Override public Integer apply(Object o) {
-            GridCache<Object, Object> cache = ((GridEx)ignite).cachex(cacheName);
+            GridCache<Object, Object> cache = ((IgniteEx)ignite).cachex(cacheName);
 
             return primaryOnly ? cache.primarySize() : cache.size();
         }
@@ -5533,7 +5533,7 @@ public abstract class GridCacheAdapter<K, V> implements GridCache<K, V>,
 
         /** {@inheritDoc} */
         @Override public Void call() throws Exception {
-            GridCacheAdapter<K, V> cache = ((GridKernal)ignite).context().cache().internalCache(cacheName);
+            GridCacheAdapter<K, V> cache = ((IgniteKernal)ignite).context().cache().internalCache(cacheName);
 
             assert cache != null : cacheName;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
index 0336832..0f5b06d 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/GridCacheContext.java
@@ -530,7 +530,7 @@ public class GridCacheContext<K, V> implements Externalizable {
     /**
      * @return Grid instance.
      */
-    public GridEx grid() {
+    public IgniteEx grid() {
         return ctx.grid();
     }
 
@@ -1888,7 +1888,7 @@ public class GridCacheContext<K, V> implements Externalizable {
         try {
             IgniteBiTuple<String, String> t = stash.get();
 
-            GridKernal grid = GridGainEx.gridx(t.get1());
+            IgniteKernal grid = GridGainEx.gridx(t.get1());
 
             GridCacheAdapter<K, V> cache = grid.internalCache(t.get2());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
index d49ea9f..2793bb2 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheDataStructuresManager.java
@@ -1493,7 +1493,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
 
         /** {@inheritDoc} */
         @Override public Void call() throws IgniteCheckedException {
-            GridCacheAdapter cache = ((GridKernal) ignite).context().cache().internalCache(cacheName);
+            GridCacheAdapter cache = ((IgniteKernal) ignite).context().cache().internalCache(cacheName);
 
             assert cache != null;
 
@@ -1561,7 +1561,7 @@ public final class GridCacheDataStructuresManager<K, V> extends GridCacheManager
 
         /** {@inheritDoc} */
         @Override public Void call() throws IgniteCheckedException {
-            GridCacheAdapter cache = ((GridKernal) ignite).context().cache().internalCache(cacheName);
+            GridCacheAdapter cache = ((IgniteKernal) ignite).context().cache().internalCache(cacheName);
 
             assert cache != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
index d113cda..4ba6149 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/GridCacheQueryManager.java
@@ -1882,7 +1882,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         /** {@inheritDoc} */
         @Override public Collection<CacheSqlMetadata> call() {
-            final GridKernalContext ctx = ((GridKernal) ignite).context();
+            final GridKernalContext ctx = ((IgniteKernal) ignite).context();
 
             Collection<String> cacheNames = F.viewReadOnly(ctx.cache().caches(),
                 new C1<GridCache<?, ?>, String>() {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
index 64fc88e..b3f99a3 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
@@ -104,7 +104,7 @@ public class GridCacheQueryJdbcMetadataTask extends ComputeTaskAdapter<String, b
             byte[] data;
 
             try {
-                GridCache<?, ?> cache = ((GridEx) ignite).cachex(cacheName);
+                GridCache<?, ?> cache = ((IgniteEx) ignite).cachex(cacheName);
 
                 assert cache != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
index 5371504..acd5051 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
@@ -192,7 +192,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
                 assert args != null;
                 assert futId == null;
 
-                GridCache<?, ?> cache = ((GridEx) ignite).cachex(cacheName);
+                GridCache<?, ?> cache = ((IgniteEx) ignite).cachex(cacheName);
 
                 CacheQuery<List<?>> qry =
                     ((GridCacheQueriesEx<?, ?>)cache.queries()).createSqlFieldsQuery(sql, true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java
index 03216a2..6e7e385 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/dr/GridDrDataLoadCacheUpdater.java
@@ -40,7 +40,7 @@ public class GridDrDataLoadCacheUpdater<K, V> implements IgniteDataLoadCacheUpda
         throws IgniteCheckedException {
         String cacheName = cache0.getConfiguration(CacheConfiguration.class).getName();
 
-        GridKernalContext ctx = ((GridKernal)cache0.unwrap(Ignite.class)).context();
+        GridKernalContext ctx = ((IgniteKernal)cache0.unwrap(Ignite.class)).context();
         IgniteLogger log = ctx.log(GridDrDataLoadCacheUpdater.class);
         GridCacheAdapter<K, V> cache = ctx.cache().internalCache(cacheName);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java
index 2e20e62..01c3edc 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/fs/GridGgfsImpl.java
@@ -2013,7 +2013,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                     private Ignite g;
 
                     @Nullable @Override public IgniteBiTuple<Long, Long> execute() throws IgniteCheckedException {
-                        IgniteFs ggfs = ((GridKernal)g).context().ggfs().ggfs(ggfsName);
+                        IgniteFs ggfs = ((IgniteKernal)g).context().ggfs().ggfs(ggfsName);
 
                         if (ggfs == null)
                             return F.t(0L, 0L);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
index fb55ab9..063509b 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/resource/GridResourceProcessor.java
@@ -60,7 +60,7 @@ public class GridResourceProcessor extends GridProcessorAdapter {
         IgniteServiceResource.class);
 
     /** Grid instance injector. */
-    private GridResourceBasicInjector<GridEx> gridInjector;
+    private GridResourceBasicInjector<IgniteEx> gridInjector;
 
     /** Spring application context injector. */
     private GridResourceInjector springCtxInjector;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index 9497936..e9c0d99 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -433,7 +433,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         final Long d = delta;
 
-        return ((GridKernal)cache.gridProjection().ignite()).context().closure().callLocalSafe(new Callable<Object>() {
+        return ((IgniteKernal)cache.gridProjection().ignite()).context().closure().callLocalSafe(new Callable<Object>() {
             @Override public Object call() throws Exception {
                 return l.addAndGet(decr ? -d : d);
             }
@@ -719,7 +719,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
             // Need to apply both operation and response transformation remotely
             // as cache could be inaccessible on local node and
             // exception processing should be consistent with local execution.
-            return op.apply((CacheProjection<Object, Object>)prj, ((GridKernal)g).context()).
+            return op.apply((CacheProjection<Object, Object>)prj, ((IgniteKernal)g).context()).
                 chain(resultWrapper((CacheProjection<Object, Object>)prj, key)).get();
         }
     }
@@ -768,7 +768,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
             // Need to apply both operation and response transformation remotely
             // as cache could be inaccessible on local node and
             // exception processing should be consistent with local execution.
-            return op.apply(cache, ((GridKernal)g).context()).chain(resultWrapper(cache, key)).get();
+            return op.apply(cache, ((IgniteKernal)g).context()).chain(resultWrapper(cache, key)).get();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
index 5178847..87d5dba 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/rest/handlers/task/GridTaskCommandHandler.java
@@ -590,7 +590,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
 
         /** */
         @IgniteInstanceResource
-        private GridEx g;
+        private IgniteEx g;
 
         /**
          * Required by {@link Externalizable}.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
index 86818d5..1515a8a 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/processors/service/GridServiceProxy.java
@@ -341,7 +341,7 @@ class GridServiceProxy<T> implements Serializable {
 
         /** {@inheritDoc} */
         @Override public Object call() throws Exception {
-            ManagedServiceContextImpl svcCtx = ((GridKernal) ignite).context().service().serviceContext(svcName);
+            ManagedServiceContextImpl svcCtx = ((IgniteKernal) ignite).context().service().serviceContext(svcName);
 
             if (svcCtx == null)
                 throw new GridServiceNotFoundException(svcName);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java
index 35f11d9..0950a4e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorJob.java
@@ -31,7 +31,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.*;
  */
 public abstract class VisorJob<A, R> extends ComputeJobAdapter {
     @IgniteInstanceResource
-    protected transient GridEx g;
+    protected transient IgniteEx g;
 
     /** Job start time. */
     protected transient long start;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
index 081213f..16ead12 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/VisorMultiNodeTask.java
@@ -37,7 +37,7 @@ import static org.apache.ignite.internal.visor.util.VisorTaskUtils.*;
  */
 public abstract class VisorMultiNodeTask<A, R, J> implements ComputeTask<VisorTaskArgument<A>, R> {
     @IgniteInstanceResource
-    protected GridEx g;
+    protected IgniteEx g;
 
     /** Debug flag. */
     protected boolean debug;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
index f49ab87..6967969 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/cache/VisorCache.java
@@ -103,7 +103,7 @@ public class VisorCache implements Serializable {
 
         String cacheName = c.name();
 
-        GridCacheAdapter ca = ((GridKernal)g).internalCache(cacheName);
+        GridCacheAdapter ca = ((IgniteKernal)g).internalCache(cacheName);
 
         long swapSize;
         long swapKeys;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java
index 2cabdcc..1a2dd79 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/compute/VisorComputeMonitoringHolder.java
@@ -47,7 +47,7 @@ public class VisorComputeMonitoringHolder {
      * @param g grid.
      * @param visorKey unique Visor instance key.
      */
-    public void startCollect(GridEx g, String visorKey) {
+    public void startCollect(IgniteEx g, String visorKey) {
         synchronized(listenVisor) {
             if (cleanupStopped) {
                 scheduleCleanupJob(g);
@@ -66,7 +66,7 @@ public class VisorComputeMonitoringHolder {
      * @param g grid.
      * @return {@code true} if task events should remain enabled.
      */
-    private boolean tryDisableEvents(GridEx g) {
+    private boolean tryDisableEvents(IgniteEx g) {
         if (!listenVisor.values().contains(true)) {
             listenVisor.clear();
 
@@ -82,7 +82,7 @@ public class VisorComputeMonitoringHolder {
      * @param g grid.
      * @param visorKey uniq Visor instance key.
      */
-    public void stopCollect(GridEx g, String visorKey) {
+    public void stopCollect(IgniteEx g, String visorKey) {
         synchronized(listenVisor) {
             listenVisor.remove(visorKey);
 
@@ -94,8 +94,8 @@ public class VisorComputeMonitoringHolder {
      * Schedule cleanup process for events monitoring.
      * @param g grid.
      */
-    private void scheduleCleanupJob(final GridEx g) {
-        ((GridKernal)g).context().timeout().addTimeoutObject(new GridTimeoutObjectAdapter(CLEANUP_TIMEOUT) {
+    private void scheduleCleanupJob(final IgniteEx g) {
+        ((IgniteKernal)g).context().timeout().addTimeoutObject(new GridTimeoutObjectAdapter(CLEANUP_TIMEOUT) {
             @Override public void onTimeout() {
                 synchronized(listenVisor) {
                     if (tryDisableEvents(g)) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
index ae68e38..6498e45 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorBasicConfiguration.java
@@ -107,7 +107,7 @@ public class VisorBasicConfiguration implements Serializable {
      * @param c Grid configuration.
      * @return Data transfer object for node basic configuration properties.
      */
-    public static VisorBasicConfiguration from(GridEx g, IgniteConfiguration c) {
+    public static VisorBasicConfiguration from(IgniteEx g, IgniteConfiguration c) {
         VisorBasicConfiguration cfg = new VisorBasicConfiguration();
 
         cfg.gridName(c.getGridName());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
index f9eb9ca..66db0c6 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorGridConfiguration.java
@@ -90,7 +90,7 @@ public class VisorGridConfiguration implements Serializable {
      * @param g Grid.
      * @return Fill data transfer object with node configuration data.
      */
-    public VisorGridConfiguration from(GridEx g) {
+    public VisorGridConfiguration from(IgniteEx g) {
         assert g != null;
 
         IgniteConfiguration c = g.configuration();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
index f00f47b..753e99e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/node/VisorNodeDataCollectorJob.java
@@ -123,7 +123,7 @@ public class VisorNodeDataCollectorJob extends VisorJob<VisorNodeDataCollectorTa
     /** Collect GGFS. */
     private void ggfs(VisorNodeDataCollectorJobResult res) {
         try {
-            IgniteFsProcessorAdapter ggfsProc = ((GridKernal)g).context().ggfs();
+            IgniteFsProcessorAdapter ggfsProc = ((IgniteKernal)g).context().ggfs();
 
             for (IgniteFs ggfs : ggfsProc.ggfss()) {
                 long start0 = U.currentTimeMillis();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
index 3925dd0..7b60a0e 100644
--- a/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/internal/visor/query/VisorQueryTask.java
@@ -281,7 +281,7 @@ public class VisorQueryTask extends VisorOneNodeTask<VisorQueryTask.VisorQueryAr
          * @param id Uniq query result id.
          */
         private void scheduleResultSetHolderRemoval(final String id) {
-            ((GridKernal)g).context().timeout()
+            ((IgniteKernal)g).context().timeout()
                 .addTimeoutObject(new GridTimeoutObjectAdapter(VisorQueryUtils.RMV_DELAY) {
                     @Override public void onTimeout() {
                         ClusterNodeLocalMap<String, VisorFutureResultSetHolder> storage = g.nodeLocalMap();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java b/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java
index e0bd722..8cb3996 100644
--- a/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java
+++ b/modules/core/src/main/java/org/apache/ignite/streamer/router/StreamerCacheAffinityEventRouter.java
@@ -58,7 +58,7 @@ public class StreamerCacheAffinityEventRouter extends StreamerEventRouterAdapter
         if (evt instanceof CacheAffinityEvent) {
             CacheAffinityEvent e = (CacheAffinityEvent)evt;
 
-            GridCache<Object, Object> c = ((GridEx) ignite).cachex(e.cacheName());
+            GridCache<Object, Object> c = ((IgniteEx) ignite).cachex(e.cacheName());
 
             assert c != null;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/GridExceptionHelpLinksSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/GridExceptionHelpLinksSelfTest.java b/modules/core/src/test/java/org/apache/ignite/GridExceptionHelpLinksSelfTest.java
deleted file mode 100644
index f4bcb23..0000000
--- a/modules/core/src/test/java/org/apache/ignite/GridExceptionHelpLinksSelfTest.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite;
-
-import junit.framework.*;
-import org.apache.ignite.*;
-import org.jetbrains.annotations.*;
-
-import java.util.*;
-
-import static org.apache.ignite.internal.util.GridUtils.*;
-
-/**
- * Tests for proper link output in stack traces.
- */
-public class GridExceptionHelpLinksSelfTest extends TestCase {
-    /**
-     * @throws Exception If failed.
-     */
-    public void testDefaultLinks() throws Exception {
-        assertTrue(hasLinksInMessage(new IgniteCheckedException("test"), DFLT_HELP_LINKS));
-        assertTrue(hasLinksInMessage(new IgniteCheckedException(new Exception()), DFLT_HELP_LINKS));
-        assertTrue(hasLinksInMessage(new IgniteCheckedException("test", new Exception()), DFLT_HELP_LINKS));
-
-        assertTrue(hasLinksInMessage(new IgniteException("test"), DFLT_HELP_LINKS));
-        assertTrue(hasLinksInMessage(new IgniteException(new Exception()), DFLT_HELP_LINKS));
-        assertTrue(hasLinksInMessage(new IgniteException("test", new Exception()), DFLT_HELP_LINKS));
-    }
-
-    /**
-     * Tests default links suppression.
-     */
-    public void testLinksUniqueness() {
-        assertLinksAppearOnce(
-            new IgniteCheckedException("test",
-                new IgniteCheckedException("test nested",
-                    new IgniteCheckedException("last"))),
-            DFLT_HELP_LINKS);
-
-        assertLinksAppearOnce(
-            new IgniteException("test",
-                new IgniteException("test nested",
-                    new IgniteException("last"))),
-            DFLT_HELP_LINKS);
-
-        assertLinksAppearOnce(
-            new IgniteCheckedException("test",
-                new IgniteException("test nested",
-                    new IgniteCheckedException("last"))),
-            DFLT_HELP_LINKS);
-
-        assertLinksAppearOnce(
-            new IgniteException("test",
-                new IgniteCheckedException("test nested",
-                    new IgniteException("last"))),
-            DFLT_HELP_LINKS);
-    }
-
-    /**
-     * @param e Root exception.
-     * @param links Set of links to ensure present only once in full stack trace.
-     */
-    @SuppressWarnings("TypeMayBeWeakened")
-    private void assertLinksAppearOnce(Throwable e, List<String>... links) {
-        Set<List<String>> seen  = new HashSet<>();
-
-        while (e != null) {
-            for (List<String> l : links)
-                if (hasLinksInMessage(e, l))
-                    assertTrue(seen.add(l));
-
-            e = e.getCause();
-        }
-    }
-
-    /**
-     * @param e Exception
-     * @param links List of links.
-     * @return Whether exception has all passed links in it's message.
-     */
-    private boolean hasLinksInMessage(Throwable e, @Nullable Iterable<String> links) {
-        if (links == null)
-            return true;
-
-        for (String link : links)
-            if (!e.getMessage().contains(link))
-                return false;
-
-        return true;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/GridExternalizableAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/GridExternalizableAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/GridExternalizableAbstractTest.java
deleted file mode 100644
index e4cdae9..0000000
--- a/modules/core/src/test/java/org/apache/ignite/GridExternalizableAbstractTest.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite;
-
-import org.apache.ignite.marshaller.*;
-import org.apache.ignite.marshaller.jdk.*;
-import org.apache.ignite.marshaller.optimized.*;
-import org.apache.ignite.testframework.junits.common.*;
-import java.util.*;
-
-/**
- * Base externalizable test class.
- */
-public class GridExternalizableAbstractTest extends GridCommonAbstractTest {
-    /**
-     * @return Marshallers.
-     */
-    protected List<IgniteMarshaller> getMarshallers() {
-        List<IgniteMarshaller> marshallers = new ArrayList<>();
-
-        marshallers.add(new IgniteJdkMarshaller());
-        marshallers.add(new IgniteOptimizedMarshaller());
-
-        return marshallers;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/IgniteExceptionHelpLinksSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/IgniteExceptionHelpLinksSelfTest.java b/modules/core/src/test/java/org/apache/ignite/IgniteExceptionHelpLinksSelfTest.java
new file mode 100644
index 0000000..3fbf4ec
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/IgniteExceptionHelpLinksSelfTest.java
@@ -0,0 +1,106 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite;
+
+import junit.framework.*;
+import org.apache.ignite.*;
+import org.jetbrains.annotations.*;
+
+import java.util.*;
+
+import static org.apache.ignite.internal.util.GridUtils.*;
+
+/**
+ * Tests for proper link output in stack traces.
+ */
+public class IgniteExceptionHelpLinksSelfTest extends TestCase {
+    /**
+     * @throws Exception If failed.
+     */
+    public void testDefaultLinks() throws Exception {
+        assertTrue(hasLinksInMessage(new IgniteCheckedException("test"), DFLT_HELP_LINKS));
+        assertTrue(hasLinksInMessage(new IgniteCheckedException(new Exception()), DFLT_HELP_LINKS));
+        assertTrue(hasLinksInMessage(new IgniteCheckedException("test", new Exception()), DFLT_HELP_LINKS));
+
+        assertTrue(hasLinksInMessage(new IgniteException("test"), DFLT_HELP_LINKS));
+        assertTrue(hasLinksInMessage(new IgniteException(new Exception()), DFLT_HELP_LINKS));
+        assertTrue(hasLinksInMessage(new IgniteException("test", new Exception()), DFLT_HELP_LINKS));
+    }
+
+    /**
+     * Tests default links suppression.
+     */
+    public void testLinksUniqueness() {
+        assertLinksAppearOnce(
+            new IgniteCheckedException("test",
+                new IgniteCheckedException("test nested",
+                    new IgniteCheckedException("last"))),
+            DFLT_HELP_LINKS);
+
+        assertLinksAppearOnce(
+            new IgniteException("test",
+                new IgniteException("test nested",
+                    new IgniteException("last"))),
+            DFLT_HELP_LINKS);
+
+        assertLinksAppearOnce(
+            new IgniteCheckedException("test",
+                new IgniteException("test nested",
+                    new IgniteCheckedException("last"))),
+            DFLT_HELP_LINKS);
+
+        assertLinksAppearOnce(
+            new IgniteException("test",
+                new IgniteCheckedException("test nested",
+                    new IgniteException("last"))),
+            DFLT_HELP_LINKS);
+    }
+
+    /**
+     * @param e Root exception.
+     * @param links Set of links to ensure present only once in full stack trace.
+     */
+    @SuppressWarnings("TypeMayBeWeakened")
+    private void assertLinksAppearOnce(Throwable e, List<String>... links) {
+        Set<List<String>> seen  = new HashSet<>();
+
+        while (e != null) {
+            for (List<String> l : links)
+                if (hasLinksInMessage(e, l))
+                    assertTrue(seen.add(l));
+
+            e = e.getCause();
+        }
+    }
+
+    /**
+     * @param e Exception
+     * @param links List of links.
+     * @return Whether exception has all passed links in it's message.
+     */
+    private boolean hasLinksInMessage(Throwable e, @Nullable Iterable<String> links) {
+        if (links == null)
+            return true;
+
+        for (String link : links)
+            if (!e.getMessage().contains(link))
+                return false;
+
+        return true;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/IgniteExternalizableAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/IgniteExternalizableAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/IgniteExternalizableAbstractTest.java
new file mode 100644
index 0000000..1bae7cd
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/IgniteExternalizableAbstractTest.java
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite;
+
+import org.apache.ignite.marshaller.*;
+import org.apache.ignite.marshaller.jdk.*;
+import org.apache.ignite.marshaller.optimized.*;
+import org.apache.ignite.testframework.junits.common.*;
+import java.util.*;
+
+/**
+ * Base externalizable test class.
+ */
+public class IgniteExternalizableAbstractTest extends GridCommonAbstractTest {
+    /**
+     * @return Marshallers.
+     */
+    protected List<IgniteMarshaller> getMarshallers() {
+        List<IgniteMarshaller> marshallers = new ArrayList<>();
+
+        marshallers.add(new IgniteJdkMarshaller());
+        marshallers.add(new IgniteOptimizedMarshaller());
+
+        return marshallers;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java
index 92a8598..28e5325 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/affinity/fair/GridCachePartitionFairAffinityNodesSelfTest.java
@@ -192,13 +192,13 @@ public class GridCachePartitionFairAffinityNodesSelfTest extends GridCommonAbstr
 
                 info("Grid 0: " + grid(0).localNode().id());
 
-                ((GridKernal)grid(0)).internalCache().context().affinity().affinityReadyFuture(topVer).get();
+                ((IgniteKernal)grid(0)).internalCache().context().affinity().affinityReadyFuture(topVer).get();
 
                 for (int i : started) {
                     if (i != 0) {
-                        GridEx grid = grid(i);
+                        IgniteEx grid = grid(i);
 
-                        ((GridKernal)grid).internalCache().context().affinity().affinityReadyFuture(topVer).get();
+                        ((IgniteKernal)grid).internalCache().context().affinity().affinityReadyFuture(topVer).get();
 
                         info("Grid " + i + ": " + grid.localNode().id());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
index 09c4e4f..7b080af 100644
--- a/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/cache/store/jdbc/GridCacheJdbcBlobStoreMultithreadedSelfTest.java
@@ -232,7 +232,7 @@ public class GridCacheJdbcBlobStoreMultithreadedSelfTest extends GridCommonAbstr
         assertEquals(GRID_CNT, Ignition.allGrids().size());
 
         for (Ignite ignite : Ignition.allGrids()) {
-            GridCacheContext cctx = ((GridKernal)ignite).internalCache().context();
+            GridCacheContext cctx = ((IgniteKernal)ignite).internalCache().context();
 
             CacheStore store = cctx.store().configuredStore();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java
index dacb3c8..0756be1 100644
--- a/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/fs/GridGgfsFragmentizerSelfTest.java
@@ -243,7 +243,7 @@ public class GridGgfsFragmentizerSelfTest extends GridGgfsFragmentizerAbstractSe
         GridTestUtils.retryAssert(log, 50, 100, new CA() {
             @Override public void apply() {
                 for (int i = 0; i < NODE_CNT; i++) {
-                    GridEx g = grid(i);
+                    IgniteEx g = grid(i);
 
                     GridCache<Object, Object> cache = g.cachex(DATA_CACHE_NAME);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java
index d57c8ee..106cd2a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoveryEventSelfTest.java
@@ -21,7 +21,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.events.*;
-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.*;
@@ -403,7 +402,7 @@ public class GridDiscoveryEventSelfTest extends GridCommonAbstractTest {
 
             daemon = true;
 
-            GridKernal daemon = (GridKernal)startGrid(3);
+            IgniteKernal daemon = (IgniteKernal)startGrid(3);
 
             IgniteDiscoveryEvent join = daemon.context().discovery().localJoinEvent();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
index 38575c3..15cfa6c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridDiscoverySelfTest.java
@@ -127,7 +127,7 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
                 nodes.add(new GridDiscoveryTestNode());
 
             @SuppressWarnings("deprecation")
-            long hash = ((GridKernal) ignite).context().discovery().topologyHash(nodes);
+            long hash = ((IgniteKernal) ignite).context().discovery().topologyHash(nodes);
 
             boolean isHashed = hashes.add(hash);
 
@@ -236,7 +236,7 @@ public class GridDiscoverySelfTest extends GridCommonAbstractTest {
      */
     public void testCacheNodes() throws Exception {
         // Validate only original node is available.
-        GridDiscoveryManager discoMgr = ((GridKernal) ignite).context().discovery();
+        GridDiscoveryManager discoMgr = ((IgniteKernal) ignite).context().discovery();
 
         Collection<ClusterNode> nodes = discoMgr.allNodes();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridExecutorServiceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridExecutorServiceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridExecutorServiceTest.java
deleted file mode 100644
index ac20e67..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridExecutorServiceTest.java
+++ /dev/null
@@ -1,315 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal;
-
-import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.internal.executor.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import java.io.*;
-import java.util.*;
-import java.util.concurrent.*;
-
-/**
- * Grid distributed executor test.
- */
-@GridCommonTest(group = "Thread Tests")
-public class GridExecutorServiceTest extends GridCommonAbstractTest {
-    /** */
-    public GridExecutorServiceTest() {
-        super(true);
-    }
-
-    /**
-     * @throws Exception Thrown in case of test failure.
-     */
-    public void testExecute() throws Exception {
-        Ignite ignite = G.ignite(getTestGridName());
-
-        ExecutorService srvc = createExecutorService(ignite);
-
-        srvc.execute(new Runnable() {
-            @IgniteInstanceResource
-            private Ignite ignite;
-
-            @Override public void run() {
-                System.out.println("Test message.");
-
-                assert this.ignite != null;
-            }
-        });
-
-        srvc.execute(new TestRunnable());
-
-        srvc.shutdown();
-    }
-
-    /**
-     * @throws Exception Thrown in case of test failure.
-     */
-    public void testSubmit() throws Exception {
-        Ignite ignite = G.ignite(getTestGridName());
-
-        ExecutorService srvc = createExecutorService(ignite);
-
-        Future<?> fut = srvc.submit(new TestRunnable());
-
-        Object res = fut.get();
-
-        info("Default Runnable result:" + res);
-
-        assert res == null : "Failed to get valid default result for submitted Runnable: " + res;
-
-        String val = "test-value";
-
-        fut = srvc.submit(new TestRunnable(), val);
-
-        res = fut.get();
-
-        info("Defined Runnable result:" + res);
-
-        assert val.equals(res) : "Failed to get valid predefined result for submitted Runnable: " + res;
-
-        fut = srvc.submit(new TestCallable<>(val));
-
-        res = fut.get();
-
-        info("Callable result:" + res);
-
-        assert val.equals(res) : "Failed to get valid result for submitted Callable: " + res;
-
-        srvc.shutdown();
-    }
-
-    /**
-     * @throws Exception Thrown in case of test failure.
-     */
-    public void testSubmitWithFutureTimeout() throws Exception {
-        Ignite ignite = G.ignite(getTestGridName());
-
-        ExecutorService srvc = createExecutorService(ignite);
-
-        Future<Integer> fut = srvc.submit(new TestCallable<>(3000)); // Just sleep for 3 seconds.
-
-        boolean ok = true;
-
-        try {
-            fut.get(1, TimeUnit.SECONDS);
-
-            ok = false;
-        }
-        catch (TimeoutException e) {
-            info("Task timeout elapsed: " + e.getMessage());
-        }
-
-        assert ok : "Timeout must be thrown.";
-
-        srvc.shutdown();
-    }
-
-    /**
-     * @throws Exception Thrown in case of test failure.
-     */
-    @SuppressWarnings("TooBroadScope")
-    public void testInvokeAll() throws Exception {
-        Ignite ignite = G.ignite(getTestGridName());
-
-        ExecutorService srvc = createExecutorService(ignite);
-
-        Collection<Callable<String>> cmds = new ArrayList<>(2);
-
-        String val1 = "test-value-1";
-        String val2 = "test-value-2";
-
-        cmds.add(new TestCallable<>(val1));
-        cmds.add(new TestCallable<>(val2));
-
-        List<Future<String>> futs = srvc.invokeAll(cmds);
-
-        assert futs != null;
-        assert futs.size() == 2;
-
-        String res1 = futs.get(0).get();
-        String res2 = futs.get(1).get();
-
-        assert val1.equals(res1) : "Failed to get valid result for first command: " + res1;
-        assert val2.equals(res2) : "Failed to get valid result for second command: " + res2;
-
-        srvc.shutdown();
-    }
-
-    /**
-     * @throws Exception Thrown in case of test failure.
-     */
-    @SuppressWarnings("TooBroadScope")
-    public void testInvokeAllWithTimeout() throws Exception {
-        Ignite ignite = G.ignite(getTestGridName());
-
-        ExecutorService srvc = createExecutorService(ignite);
-
-        Collection<Callable<Integer>> cmds = new ArrayList<>();
-
-        cmds.add(new TestCallable<>(3000)); // Just sleeps for 3 seconds.
-        cmds.add(new TestCallable<>(3000)); // Just sleeps for 3 seconds.
-
-        List<Future<Integer>> fut = srvc.invokeAll(cmds, 1, TimeUnit.SECONDS);
-
-        assert fut != null;
-        assert fut.size() == 2;
-
-        boolean ok = true;
-
-        try {
-            fut.get(0).get();
-
-            ok = false;
-        }
-        catch (CancellationException e) {
-            info("First timeout task is cancelled: " + e.getMessage());
-        }
-
-        assert ok : "First task must be cancelled.";
-
-        try {
-            fut.get(1).get();
-
-            ok = false;
-        }
-        catch (CancellationException e) {
-            info("Second timeout task is cancelled: " + e.getMessage());
-        }
-
-        assert ok : "Second task must be cancelled.";
-
-        srvc.shutdown();
-    }
-
-    /**
-     * @throws Exception Thrown in case of test failure.
-     */
-    @SuppressWarnings("TooBroadScope")
-    public void testInvokeAny() throws Exception {
-        Ignite ignite = G.ignite(getTestGridName());
-
-        ExecutorService srvc = createExecutorService(ignite);
-
-        Collection<Callable<String>> cmds = new ArrayList<>(2);
-
-        String val1 = "test-value-1";
-        String val2 = "test-value-2";
-
-        cmds.add(new TestCallable<>(val1));
-        cmds.add(new TestCallable<>(val2));
-
-        String res = srvc.invokeAny(cmds);
-
-        info("Result: " + res);
-
-        assert val1.equals(res) : "Failed to get valid result: " + res;
-
-        srvc.shutdown();
-    }
-
-    /**
-     * @throws Exception Thrown in case of test failure.
-     */
-    @SuppressWarnings("TooBroadScope")
-    public void testInvokeAnyWithTimeout() throws Exception {
-        Ignite ignite = G.ignite(getTestGridName());
-
-        ExecutorService srvc = createExecutorService(ignite);
-
-        Collection<Callable<Integer>> timeoutCmds = new ArrayList<>(2);
-
-        timeoutCmds.add(new TestCallable<>(5000));
-        timeoutCmds.add(new TestCallable<>(5000));
-
-        boolean ok = true;
-
-        try {
-            srvc.invokeAny(timeoutCmds, 1, TimeUnit.SECONDS);
-
-            ok = false;
-        }
-        catch (TimeoutException e) {
-            info("Task timeout elapsed: " + e.getMessage());
-        }
-
-        assert ok : "Timeout must be thrown.";
-
-        srvc.shutdown();
-    }
-
-    /**
-     * @param ignite Grid instance.
-     * @return Thrown in case of test failure.
-     */
-    private ExecutorService createExecutorService(Ignite ignite) {
-        assert ignite != null;
-
-        return new GridExecutorService((ClusterGroupAdapter) ignite, log());
-    }
-
-    /**
-     * @param <T> Type of the {@link Callable} argument.
-     */
-    private static class TestCallable<T> implements Callable<T>, Serializable {
-        /** */
-        private T data;
-
-        /** */
-        @IgniteInstanceResource
-        private Ignite ignite;
-
-        /**
-         * @param data Data.
-         */
-        TestCallable(T data) {
-            this.data = data;
-        }
-
-        /** {@inheritDoc} */
-        @Override public T call() throws Exception {
-            System.out.println("Test callable message.");
-
-            assert ignite != null;
-
-            if (data instanceof Integer)
-                Thread.sleep((Integer)data);
-
-            return data;
-        }
-    }
-
-    /** */
-    private static class TestRunnable implements Runnable, Serializable {
-        /** */
-        @IgniteInstanceResource
-        private Ignite ignite;
-
-        /** {@inheritDoc} */
-        @Override public void run() {
-            System.out.println("Test Runnable message.");
-
-            assert ignite != null;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridExplicitImplicitDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridExplicitImplicitDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridExplicitImplicitDeploymentSelfTest.java
deleted file mode 100644
index f19622e..0000000
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridExplicitImplicitDeploymentSelfTest.java
+++ /dev/null
@@ -1,476 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *      http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.ignite.internal;
-
-import org.apache.ignite.*;
-import org.apache.ignite.cluster.*;
-import org.apache.ignite.compute.*;
-import org.apache.ignite.configuration.*;
-import org.apache.ignite.resources.*;
-import org.apache.ignite.internal.util.typedef.*;
-import org.apache.ignite.internal.util.typedef.internal.*;
-import org.apache.ignite.testframework.*;
-import org.apache.ignite.testframework.junits.common.*;
-
-import java.io.*;
-import java.util.*;
-
-/**
- *
- */
-@GridCommonTest(group = "Kernal Self")
-public class GridExplicitImplicitDeploymentSelfTest extends GridCommonAbstractTest {
-    /** */
-    public GridExplicitImplicitDeploymentSelfTest() {
-        super(/*start grid*/false);
-    }
-
-    /** {@inheritDoc} */
-    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
-        IgniteConfiguration cfg = super.getConfiguration(gridName);
-
-        // Override P2P configuration to exclude Task and Job classes
-        cfg.setPeerClassLoadingLocalClassPathExclude(GridDeploymentResourceTestTask.class.getName(),
-            GridDeploymentResourceTestJob.class.getName());
-
-        cfg.setDeploymentMode(IgniteDeploymentMode.ISOLATED);
-
-        return cfg;
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    public void testImplicitDeployLocally() throws Exception {
-        execImplicitDeployLocally(true, true, true);
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    public void testImplicitDeployP2P() throws Exception {
-        execImplicitDeployP2P(true, true, true);
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    public void testExplicitDeployLocally() throws Exception {
-        execExplicitDeployLocally(true, true, true);
-    }
-
-    /**
-     * @throws Exception If test failed.
-     */
-    public void testExplicitDeployP2P() throws Exception {
-        execExplicitDeployP2P(true, true, true);
-    }
-
-    /**
-     * @param ignite Grid.
-     */
-    @SuppressWarnings({"CatchGenericClass"})
-    private void stopGrid(Ignite ignite) {
-        try {
-            if (ignite != null)
-                G.stop(ignite.name(), true);
-        }
-        catch (Throwable e) {
-            error("Got error when stopping grid.", e);
-        }
-    }
-
-    /**
-     * @param byCls If {@code true} than executes task by Class.
-     * @param byTask If {@code true} than executes task instance.
-     * @param byName If {@code true} than executes task by class name.
-     * @throws Exception If test failed.
-     */
-    @SuppressWarnings("unchecked")
-    private void execExplicitDeployLocally(boolean byCls, boolean byTask, boolean byName) throws Exception {
-        Ignite ignite = null;
-
-        try {
-            ignite = startGrid();
-
-            // Explicit Deployment. Task execution should return 0.
-            // Say resource class loader - different to task one.
-            ClassLoader ldr1 = new GridTestClassLoader(
-                Collections.singletonMap("testResource", "1"),
-                getClass().getClassLoader());
-
-            // Assume that users task and job were loaded with this class loader
-            ClassLoader ldr2 = new GridTestClassLoader(
-                Collections.singletonMap("testResource", "2"),
-                getClass().getClassLoader(),
-                GridDeploymentResourceTestTask.class.getName(),
-                GridDeploymentResourceTestJob.class.getName()
-            );
-
-            info("Loader1: " + ldr1);
-            info("Loader2: " + ldr2);
-
-            Class<? extends ComputeTask<String, Integer>> taskCls = (Class<? extends ComputeTask<String, Integer>>)
-                ldr2.loadClass(GridDeploymentResourceTestTask.class.getName());
-
-            // Check auto-deploy. It should pick up resource class loader.
-            if (byCls) {
-                ignite.compute().localDeployTask(taskCls, ldr1);
-
-                Integer res = ignite.compute().execute(taskCls, null);
-
-                assert res != null;
-                assert res == 2 : "Invalid response: " + res;
-            }
-
-            if (byTask) {
-                ignite.compute().localDeployTask(taskCls, ldr1);
-
-                Integer res = ignite.compute().execute(taskCls.newInstance(), null);
-
-                assert res != null;
-                assert res == 2 : "Invalid response: " + res;
-            }
-
-            if (byName) {
-                ignite.compute().localDeployTask(taskCls, ldr1);
-
-                Integer res = (Integer) ignite.compute().execute(taskCls.getName(), null);
-
-                assert res != null;
-                assert res == 1 : "Invalid response: " + res;
-            }
-        }
-        finally {
-            stopGrid(ignite);
-        }
-    }
-
-    /**
-     * @param byCls If {@code true} than executes task by Class.
-     * @param byTask If {@code true} than executes task instance.
-     * @param byName If {@code true} than executes task by class name.
-     * @throws Exception If test failed.
-     */
-   @SuppressWarnings("unchecked")
-   private void execImplicitDeployLocally(boolean byCls, boolean byTask, boolean byName) throws Exception {
-       Ignite ignite = null;
-
-       try {
-           ignite = startGrid();
-
-           // First task class loader.
-           ClassLoader ldr1 = new GridTestClassLoader(
-               Collections.singletonMap("testResource", "1"),
-               getClass().getClassLoader(),
-               GridDeploymentResourceTestTask.class.getName(),
-               GridDeploymentResourceTestJob.class.getName()
-           );
-
-           // Second task class loader
-           ClassLoader ldr2 = new GridTestClassLoader(
-               Collections.singletonMap("testResource", "2"),
-               getClass().getClassLoader(),
-               GridDeploymentResourceTestTask.class.getName(),
-               GridDeploymentResourceTestJob.class.getName()
-           );
-
-           // The same name but different classes/ class loaders.
-           Class<? extends ComputeTask<String, Integer>> taskCls1 = (Class<? extends ComputeTask<String, Integer>>)
-               ldr1.loadClass(GridDeploymentResourceTestTask.class.getName());
-
-           Class<? extends ComputeTask<String, Integer>> taskCls2 = (Class<? extends ComputeTask<String, Integer>>)
-               ldr2.loadClass(GridDeploymentResourceTestTask.class.getName());
-
-           if (byCls) {
-               Integer res1 = ignite.compute().execute(taskCls1, null);
-               Integer res2 = ignite.compute().execute(taskCls2, null);
-
-               assert res1 != null;
-               assert res2 != null;
-
-               assert res1 == 1 : "Invalid res1: " + res1;
-               assert res2 == 2 : "Invalid res2: " + res2;
-           }
-
-           if (byTask) {
-               Integer res1 = ignite.compute().execute(taskCls1.newInstance(), null);
-               Integer res2 = ignite.compute().execute(taskCls2.newInstance(), null);
-
-               assert res1 != null;
-               assert res2 != null;
-
-               assert res1 == 1 : "Invalid res1: " + res1;
-               assert res2 == 2 : "Invalid res2: " + res2;
-           }
-
-           if (byName) {
-               ignite.compute().localDeployTask(taskCls1, ldr1);
-
-               Integer res1 = (Integer) ignite.compute().execute(taskCls1.getName(), null);
-
-               ignite.compute().localDeployTask(taskCls2, ldr2);
-
-               Integer res2 = (Integer) ignite.compute().execute(taskCls2.getName(), null);
-
-               assert res1 != null;
-               assert res2 != null;
-
-               assert res1 == 1 : "Invalid res1: " + res1;
-               assert res2 == 2 : "Invalid res2: " + res2;
-           }
-       }
-       finally {
-           stopGrid(ignite);
-       }
-   }
-
-    /**
-     * @param byCls If {@code true} than executes task by Class.
-     * @param byTask If {@code true} than executes task instance.
-     * @param byName If {@code true} than executes task by class name.
-     * @throws Exception If test failed.
-     */
-    @SuppressWarnings("unchecked")
-    private void execExplicitDeployP2P(boolean byCls, boolean byTask, boolean byName) throws Exception {
-       Ignite ignite1 = null;
-       Ignite ignite2 = null;
-
-       try {
-           ignite1 = startGrid(1);
-           ignite2 = startGrid(2);
-
-           ClassLoader ldr1 = new GridTestClassLoader(
-               Collections.singletonMap("testResource", "1"),
-               getClass().getClassLoader(),
-               GridDeploymentResourceTestTask.class.getName(),
-               GridDeploymentResourceTestJob.class.getName()
-           );
-
-           ClassLoader ldr2 = new GridTestClassLoader(
-               Collections.singletonMap("testResource", "2"),
-               getClass().getClassLoader(),
-               GridDeploymentResourceTestTask.class.getName(),
-               GridDeploymentResourceTestJob.class.getName()
-           );
-
-           Class<? extends ComputeTask<String, Integer>> taskCls = (Class<? extends ComputeTask<String, Integer>>)
-               ldr2.loadClass(GridDeploymentResourceTestTask.class.getName());
-
-           if (byCls) {
-               ignite1.compute().localDeployTask(taskCls, ldr1);
-
-               // Even though the task is deployed with resource class loader,
-               // when we execute it, it will be redeployed with task class-loader.
-               Integer res = ignite1.compute().execute(taskCls, null);
-
-               assert res != null;
-               assert res == 2 : "Invalid response: " + res;
-           }
-
-
-           if (byTask) {
-               ignite1.compute().localDeployTask(taskCls, ldr1);
-
-               // Even though the task is deployed with resource class loader,
-               // when we execute it, it will be redeployed with task class-loader.
-               Integer res = ignite1.compute().execute(taskCls.newInstance(), null);
-
-               assert res != null;
-               assert res == 2 : "Invalid response: " + res;
-           }
-
-           if (byName) {
-               ignite1.compute().localDeployTask(taskCls, ldr1);
-
-               // Even though the task is deployed with resource class loader,
-               // when we execute it, it will be redeployed with task class-loader.
-               Integer res = (Integer) ignite1.compute().execute(taskCls.getName(), null);
-
-               assert res != null;
-               assert res == 1 : "Invalid response: " + res;
-           }
-       }
-       finally {
-           stopGrid(ignite2);
-           stopGrid(ignite1);
-       }
-    }
-
-    /**
-     * @param byCls If {@code true} than executes task by Class.
-     * @param byTask If {@code true} than executes task instance.
-     * @param byName If {@code true} than executes task by class name.
-     * @throws Exception If test failed.
-     */
-   @SuppressWarnings("unchecked")
-   private void execImplicitDeployP2P(boolean byCls, boolean byTask, boolean byName) throws Exception {
-      Ignite ignite1 = null;
-      Ignite ignite2 = null;
-
-      try {
-          ignite1 = startGrid(1);
-          ignite2 = startGrid(2);
-
-          ClassLoader ldr1 = new GridTestClassLoader(
-              Collections.singletonMap("testResource", "1"),
-              getClass().getClassLoader(),
-              GridDeploymentResourceTestTask.class.getName(),
-              GridDeploymentResourceTestJob.class.getName()
-          );
-
-          ClassLoader ldr2 = new GridTestClassLoader(
-              Collections.singletonMap("testResource", "2"),
-              getClass().getClassLoader(),
-              GridDeploymentResourceTestTask.class.getName(),
-              GridDeploymentResourceTestJob.class.getName()
-          );
-
-          Class<? extends ComputeTask<String, Integer>> taskCls1 = (Class<? extends ComputeTask<String, Integer>>)
-              ldr1.loadClass(GridDeploymentResourceTestTask.class.getName());
-
-          Class<? extends ComputeTask<String, Integer>> taskCls2 = (Class<? extends ComputeTask<String, Integer>>)
-              ldr2.loadClass(GridDeploymentResourceTestTask.class.getName());
-
-          if (byCls) {
-              Integer res1 = ignite1.compute().execute(taskCls1, null);
-              Integer res2 = ignite1.compute().execute(taskCls2, null);
-
-              assert res1 != null;
-              assert res2 != null;
-
-              assert res1 == 1 : "Invalid res1: " + res1;
-              assert res2 == 2 : "Invalid res2: " + res2;
-          }
-
-          if (byTask) {
-              Integer res1 = ignite1.compute().execute(taskCls1.newInstance(), null);
-              Integer res2 = ignite1.compute().execute(taskCls2.newInstance(), null);
-
-              assert res1 != null;
-              assert res2 != null;
-
-              assert res1 == 1 : "Invalid res1: " + res1;
-              assert res2 == 2 : "Invalid res2: " + res2;
-          }
-
-          if (byName) {
-              ignite1.compute().localDeployTask(taskCls1, ldr1);
-
-              Integer res1 = (Integer) ignite1.compute().execute(taskCls1.getName(), null);
-
-              ignite1.compute().localDeployTask(taskCls2, ldr2);
-
-              Integer res2 = (Integer) ignite1.compute().execute(taskCls2.getName(), null);
-
-              assert res1 != null;
-              assert res2 != null;
-
-              assert res1 == 1 : "Invalid res1: " + res1;
-              assert res2 == 2 : "Invalid res2: " + res2;
-          }
-      }
-      finally {
-          stopGrid(ignite1);
-          stopGrid(ignite2);
-      }
-   }
-
-    /**
-     * We use custom name to avoid auto-deployment in the same VM.
-     */
-    @SuppressWarnings({"PublicInnerClass"})
-    @ComputeTaskName("GridDeploymentResourceTestTask")
-    public static class GridDeploymentResourceTestTask extends ComputeTaskAdapter<String, Integer> {
-        /** */
-        @IgniteInstanceResource
-        private Ignite ignite;
-
-        /** {@inheritDoc} */
-        @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, String arg) throws IgniteCheckedException {
-            Map<ComputeJobAdapter, ClusterNode> map = new HashMap<>(subgrid.size());
-
-            boolean ignoreLocNode = false;
-
-            UUID locId = ignite.configuration().getNodeId();
-
-            if (subgrid.size() == 1)
-                assert subgrid.get(0).id().equals(locId) : "Wrong node id.";
-            else
-                ignoreLocNode = true;
-
-            for (ClusterNode node : subgrid) {
-                // Ignore local node.
-                if (ignoreLocNode && node.id().equals(locId))
-                    continue;
-
-                map.put(new GridDeploymentResourceTestJob(), node);
-            }
-
-            return map;
-        }
-
-        /** {@inheritDoc} */
-        @Override public Integer reduce(List<ComputeJobResult> results) throws IgniteCheckedException {
-            return results.get(0).getData();
-        }
-    }
-
-    /**
-     * Simple job for this test.
-     */
-    @SuppressWarnings({"PublicInnerClass"})
-    public static final class GridDeploymentResourceTestJob extends ComputeJobAdapter {
-        /** */
-        @IgniteLoggerResource
-        private IgniteLogger log;
-
-        /** {@inheritDoc} */
-        @Override public Serializable execute() throws IgniteCheckedException {
-            if (log.isInfoEnabled())
-                log.info("Executing grid job: " + this);
-
-            try {
-                ClassLoader ldr = Thread.currentThread().getContextClassLoader();
-
-                if (log.isInfoEnabled())
-                    log.info("Loader (inside job): " + ldr);
-
-                InputStream in = ldr.getResourceAsStream("testResource");
-
-                if (in != null) {
-                    Reader reader = new InputStreamReader(in);
-
-                    try {
-                        char res = (char)reader.read();
-
-                        return Integer.parseInt(Character.toString(res));
-                    }
-                    finally {
-                        U.close(in, null);
-                    }
-                }
-
-                return null;
-            }
-            catch (IOException e) {
-                throw new IgniteCheckedException("Failed to execute job.", e);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java b/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java
index 6a473f9..0c92b43 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridKernalTestUtils.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal;
 
 import org.apache.ignite.Ignite;
-import org.apache.ignite.internal.*;
 
 /**
  * Test kernal utils.
@@ -40,6 +39,6 @@ public class GridKernalTestUtils {
     public static GridKernalContext context(Ignite ignite) {
         assert ignite != null;
 
-        return ((GridKernal) ignite).context();
+        return ((IgniteKernal) ignite).context();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java
index 96926f5..343e48d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridListenActorSelfTest.java
@@ -19,7 +19,6 @@ package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
-import org.apache.ignite.internal.*;
 import org.apache.ignite.messaging.*;
 import org.apache.ignite.internal.util.typedef.*;
 import org.apache.ignite.testframework.*;
@@ -52,7 +51,7 @@ public class GridListenActorSelfTest extends GridCommonAbstractTest {
     /** {@inheritDoc} */
     @SuppressWarnings("deprecation")
     @Override protected void afterTest() throws Exception {
-        ((GridKernal)grid()).context().io().
+        ((IgniteKernal)grid()).context().io().
             removeMessageListener(GridTopic.TOPIC_COMM_USER.name());
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java
index b17bd1b..9d97559 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTopicExternalizableSelfTest.java
@@ -18,7 +18,6 @@
 package org.apache.ignite.internal;
 
 import org.apache.ignite.*;
-import org.apache.ignite.internal.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
 
@@ -28,7 +27,7 @@ import java.util.*;
 /**
  * Grid topic externalization test.
  */
-public class GridTopicExternalizableSelfTest extends GridExternalizableAbstractTest {
+public class GridTopicExternalizableSelfTest extends IgniteExternalizableAbstractTest {
     /** */
     private static final IgniteUuid A_GRID_UUID = IgniteUuid.randomUuid();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java
index 686640c..78b866a 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridTopologyBuildVersionSelfTest.java
@@ -80,7 +80,7 @@ public class GridTopologyBuildVersionSelfTest extends GridCommonAbstractTest {
 
         try {
             for (int i = 3; i >= 0; i--) {
-                GridKernal g = (GridKernal)grid(i);
+                IgniteKernal g = (IgniteKernal)grid(i);
 
                 NavigableMap<IgniteProductVersion, Collection<ClusterNode>> verMap = g.context().discovery()
                     .topologyVersionMap();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
index 6edfa1b..51142d0 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/GridUpdateNotifierSelfTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.ignite.internal;
 
-import org.apache.ignite.internal.*;
 import org.apache.ignite.testframework.junits.common.*;
 import org.jetbrains.annotations.*;
 
@@ -38,7 +37,7 @@ public class GridUpdateNotifierSelfTest extends GridCommonAbstractTest {
      */
     public void testNotifier() throws Exception {
         GridUpdateNotifier ntf = new GridUpdateNotifier(null, GridProperties.get("ignite.version"),
-            GridKernal.SITE, TEST_GATEWAY, false);
+            IgniteKernal.SITE, TEST_GATEWAY, false);
 
         ntf.checkForNewVersion(new SelfExecutor(), log);
 


[36/50] [abbrv] incubator-ignite git commit: GridEx -> IgniteEx GridKernal -> IgniteKernal

Posted by vo...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/IgniteExecutorServiceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteExecutorServiceTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteExecutorServiceTest.java
new file mode 100644
index 0000000..d76fe95
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteExecutorServiceTest.java
@@ -0,0 +1,315 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.internal.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.internal.executor.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.io.*;
+import java.util.*;
+import java.util.concurrent.*;
+
+/**
+ * Grid distributed executor test.
+ */
+@GridCommonTest(group = "Thread Tests")
+public class IgniteExecutorServiceTest extends GridCommonAbstractTest {
+    /** */
+    public IgniteExecutorServiceTest() {
+        super(true);
+    }
+
+    /**
+     * @throws Exception Thrown in case of test failure.
+     */
+    public void testExecute() throws Exception {
+        Ignite ignite = G.ignite(getTestGridName());
+
+        ExecutorService srvc = createExecutorService(ignite);
+
+        srvc.execute(new Runnable() {
+            @IgniteInstanceResource
+            private Ignite ignite;
+
+            @Override public void run() {
+                System.out.println("Test message.");
+
+                assert this.ignite != null;
+            }
+        });
+
+        srvc.execute(new TestRunnable());
+
+        srvc.shutdown();
+    }
+
+    /**
+     * @throws Exception Thrown in case of test failure.
+     */
+    public void testSubmit() throws Exception {
+        Ignite ignite = G.ignite(getTestGridName());
+
+        ExecutorService srvc = createExecutorService(ignite);
+
+        Future<?> fut = srvc.submit(new TestRunnable());
+
+        Object res = fut.get();
+
+        info("Default Runnable result:" + res);
+
+        assert res == null : "Failed to get valid default result for submitted Runnable: " + res;
+
+        String val = "test-value";
+
+        fut = srvc.submit(new TestRunnable(), val);
+
+        res = fut.get();
+
+        info("Defined Runnable result:" + res);
+
+        assert val.equals(res) : "Failed to get valid predefined result for submitted Runnable: " + res;
+
+        fut = srvc.submit(new TestCallable<>(val));
+
+        res = fut.get();
+
+        info("Callable result:" + res);
+
+        assert val.equals(res) : "Failed to get valid result for submitted Callable: " + res;
+
+        srvc.shutdown();
+    }
+
+    /**
+     * @throws Exception Thrown in case of test failure.
+     */
+    public void testSubmitWithFutureTimeout() throws Exception {
+        Ignite ignite = G.ignite(getTestGridName());
+
+        ExecutorService srvc = createExecutorService(ignite);
+
+        Future<Integer> fut = srvc.submit(new TestCallable<>(3000)); // Just sleep for 3 seconds.
+
+        boolean ok = true;
+
+        try {
+            fut.get(1, TimeUnit.SECONDS);
+
+            ok = false;
+        }
+        catch (TimeoutException e) {
+            info("Task timeout elapsed: " + e.getMessage());
+        }
+
+        assert ok : "Timeout must be thrown.";
+
+        srvc.shutdown();
+    }
+
+    /**
+     * @throws Exception Thrown in case of test failure.
+     */
+    @SuppressWarnings("TooBroadScope")
+    public void testInvokeAll() throws Exception {
+        Ignite ignite = G.ignite(getTestGridName());
+
+        ExecutorService srvc = createExecutorService(ignite);
+
+        Collection<Callable<String>> cmds = new ArrayList<>(2);
+
+        String val1 = "test-value-1";
+        String val2 = "test-value-2";
+
+        cmds.add(new TestCallable<>(val1));
+        cmds.add(new TestCallable<>(val2));
+
+        List<Future<String>> futs = srvc.invokeAll(cmds);
+
+        assert futs != null;
+        assert futs.size() == 2;
+
+        String res1 = futs.get(0).get();
+        String res2 = futs.get(1).get();
+
+        assert val1.equals(res1) : "Failed to get valid result for first command: " + res1;
+        assert val2.equals(res2) : "Failed to get valid result for second command: " + res2;
+
+        srvc.shutdown();
+    }
+
+    /**
+     * @throws Exception Thrown in case of test failure.
+     */
+    @SuppressWarnings("TooBroadScope")
+    public void testInvokeAllWithTimeout() throws Exception {
+        Ignite ignite = G.ignite(getTestGridName());
+
+        ExecutorService srvc = createExecutorService(ignite);
+
+        Collection<Callable<Integer>> cmds = new ArrayList<>();
+
+        cmds.add(new TestCallable<>(3000)); // Just sleeps for 3 seconds.
+        cmds.add(new TestCallable<>(3000)); // Just sleeps for 3 seconds.
+
+        List<Future<Integer>> fut = srvc.invokeAll(cmds, 1, TimeUnit.SECONDS);
+
+        assert fut != null;
+        assert fut.size() == 2;
+
+        boolean ok = true;
+
+        try {
+            fut.get(0).get();
+
+            ok = false;
+        }
+        catch (CancellationException e) {
+            info("First timeout task is cancelled: " + e.getMessage());
+        }
+
+        assert ok : "First task must be cancelled.";
+
+        try {
+            fut.get(1).get();
+
+            ok = false;
+        }
+        catch (CancellationException e) {
+            info("Second timeout task is cancelled: " + e.getMessage());
+        }
+
+        assert ok : "Second task must be cancelled.";
+
+        srvc.shutdown();
+    }
+
+    /**
+     * @throws Exception Thrown in case of test failure.
+     */
+    @SuppressWarnings("TooBroadScope")
+    public void testInvokeAny() throws Exception {
+        Ignite ignite = G.ignite(getTestGridName());
+
+        ExecutorService srvc = createExecutorService(ignite);
+
+        Collection<Callable<String>> cmds = new ArrayList<>(2);
+
+        String val1 = "test-value-1";
+        String val2 = "test-value-2";
+
+        cmds.add(new TestCallable<>(val1));
+        cmds.add(new TestCallable<>(val2));
+
+        String res = srvc.invokeAny(cmds);
+
+        info("Result: " + res);
+
+        assert val1.equals(res) : "Failed to get valid result: " + res;
+
+        srvc.shutdown();
+    }
+
+    /**
+     * @throws Exception Thrown in case of test failure.
+     */
+    @SuppressWarnings("TooBroadScope")
+    public void testInvokeAnyWithTimeout() throws Exception {
+        Ignite ignite = G.ignite(getTestGridName());
+
+        ExecutorService srvc = createExecutorService(ignite);
+
+        Collection<Callable<Integer>> timeoutCmds = new ArrayList<>(2);
+
+        timeoutCmds.add(new TestCallable<>(5000));
+        timeoutCmds.add(new TestCallable<>(5000));
+
+        boolean ok = true;
+
+        try {
+            srvc.invokeAny(timeoutCmds, 1, TimeUnit.SECONDS);
+
+            ok = false;
+        }
+        catch (TimeoutException e) {
+            info("Task timeout elapsed: " + e.getMessage());
+        }
+
+        assert ok : "Timeout must be thrown.";
+
+        srvc.shutdown();
+    }
+
+    /**
+     * @param ignite Grid instance.
+     * @return Thrown in case of test failure.
+     */
+    private ExecutorService createExecutorService(Ignite ignite) {
+        assert ignite != null;
+
+        return new GridExecutorService((ClusterGroupAdapter) ignite, log());
+    }
+
+    /**
+     * @param <T> Type of the {@link Callable} argument.
+     */
+    private static class TestCallable<T> implements Callable<T>, Serializable {
+        /** */
+        private T data;
+
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /**
+         * @param data Data.
+         */
+        TestCallable(T data) {
+            this.data = data;
+        }
+
+        /** {@inheritDoc} */
+        @Override public T call() throws Exception {
+            System.out.println("Test callable message.");
+
+            assert ignite != null;
+
+            if (data instanceof Integer)
+                Thread.sleep((Integer)data);
+
+            return data;
+        }
+    }
+
+    /** */
+    private static class TestRunnable implements Runnable, Serializable {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public void run() {
+            System.out.println("Test Runnable message.");
+
+            assert ignite != null;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/IgniteExplicitImplicitDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/IgniteExplicitImplicitDeploymentSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/IgniteExplicitImplicitDeploymentSelfTest.java
new file mode 100644
index 0000000..22a3639
--- /dev/null
+++ b/modules/core/src/test/java/org/apache/ignite/internal/IgniteExplicitImplicitDeploymentSelfTest.java
@@ -0,0 +1,476 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.ignite.internal;
+
+import org.apache.ignite.*;
+import org.apache.ignite.cluster.*;
+import org.apache.ignite.compute.*;
+import org.apache.ignite.configuration.*;
+import org.apache.ignite.resources.*;
+import org.apache.ignite.internal.util.typedef.*;
+import org.apache.ignite.internal.util.typedef.internal.*;
+import org.apache.ignite.testframework.*;
+import org.apache.ignite.testframework.junits.common.*;
+
+import java.io.*;
+import java.util.*;
+
+/**
+ *
+ */
+@GridCommonTest(group = "Kernal Self")
+public class IgniteExplicitImplicitDeploymentSelfTest extends GridCommonAbstractTest {
+    /** */
+    public IgniteExplicitImplicitDeploymentSelfTest() {
+        super(/*start grid*/false);
+    }
+
+    /** {@inheritDoc} */
+    @Override protected IgniteConfiguration getConfiguration(String gridName) throws Exception {
+        IgniteConfiguration cfg = super.getConfiguration(gridName);
+
+        // Override P2P configuration to exclude Task and Job classes
+        cfg.setPeerClassLoadingLocalClassPathExclude(GridDeploymentResourceTestTask.class.getName(),
+            GridDeploymentResourceTestJob.class.getName());
+
+        cfg.setDeploymentMode(IgniteDeploymentMode.ISOLATED);
+
+        return cfg;
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testImplicitDeployLocally() throws Exception {
+        execImplicitDeployLocally(true, true, true);
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testImplicitDeployP2P() throws Exception {
+        execImplicitDeployP2P(true, true, true);
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testExplicitDeployLocally() throws Exception {
+        execExplicitDeployLocally(true, true, true);
+    }
+
+    /**
+     * @throws Exception If test failed.
+     */
+    public void testExplicitDeployP2P() throws Exception {
+        execExplicitDeployP2P(true, true, true);
+    }
+
+    /**
+     * @param ignite Grid.
+     */
+    @SuppressWarnings({"CatchGenericClass"})
+    private void stopGrid(Ignite ignite) {
+        try {
+            if (ignite != null)
+                G.stop(ignite.name(), true);
+        }
+        catch (Throwable e) {
+            error("Got error when stopping grid.", e);
+        }
+    }
+
+    /**
+     * @param byCls If {@code true} than executes task by Class.
+     * @param byTask If {@code true} than executes task instance.
+     * @param byName If {@code true} than executes task by class name.
+     * @throws Exception If test failed.
+     */
+    @SuppressWarnings("unchecked")
+    private void execExplicitDeployLocally(boolean byCls, boolean byTask, boolean byName) throws Exception {
+        Ignite ignite = null;
+
+        try {
+            ignite = startGrid();
+
+            // Explicit Deployment. Task execution should return 0.
+            // Say resource class loader - different to task one.
+            ClassLoader ldr1 = new GridTestClassLoader(
+                Collections.singletonMap("testResource", "1"),
+                getClass().getClassLoader());
+
+            // Assume that users task and job were loaded with this class loader
+            ClassLoader ldr2 = new GridTestClassLoader(
+                Collections.singletonMap("testResource", "2"),
+                getClass().getClassLoader(),
+                GridDeploymentResourceTestTask.class.getName(),
+                GridDeploymentResourceTestJob.class.getName()
+            );
+
+            info("Loader1: " + ldr1);
+            info("Loader2: " + ldr2);
+
+            Class<? extends ComputeTask<String, Integer>> taskCls = (Class<? extends ComputeTask<String, Integer>>)
+                ldr2.loadClass(GridDeploymentResourceTestTask.class.getName());
+
+            // Check auto-deploy. It should pick up resource class loader.
+            if (byCls) {
+                ignite.compute().localDeployTask(taskCls, ldr1);
+
+                Integer res = ignite.compute().execute(taskCls, null);
+
+                assert res != null;
+                assert res == 2 : "Invalid response: " + res;
+            }
+
+            if (byTask) {
+                ignite.compute().localDeployTask(taskCls, ldr1);
+
+                Integer res = ignite.compute().execute(taskCls.newInstance(), null);
+
+                assert res != null;
+                assert res == 2 : "Invalid response: " + res;
+            }
+
+            if (byName) {
+                ignite.compute().localDeployTask(taskCls, ldr1);
+
+                Integer res = (Integer) ignite.compute().execute(taskCls.getName(), null);
+
+                assert res != null;
+                assert res == 1 : "Invalid response: " + res;
+            }
+        }
+        finally {
+            stopGrid(ignite);
+        }
+    }
+
+    /**
+     * @param byCls If {@code true} than executes task by Class.
+     * @param byTask If {@code true} than executes task instance.
+     * @param byName If {@code true} than executes task by class name.
+     * @throws Exception If test failed.
+     */
+   @SuppressWarnings("unchecked")
+   private void execImplicitDeployLocally(boolean byCls, boolean byTask, boolean byName) throws Exception {
+       Ignite ignite = null;
+
+       try {
+           ignite = startGrid();
+
+           // First task class loader.
+           ClassLoader ldr1 = new GridTestClassLoader(
+               Collections.singletonMap("testResource", "1"),
+               getClass().getClassLoader(),
+               GridDeploymentResourceTestTask.class.getName(),
+               GridDeploymentResourceTestJob.class.getName()
+           );
+
+           // Second task class loader
+           ClassLoader ldr2 = new GridTestClassLoader(
+               Collections.singletonMap("testResource", "2"),
+               getClass().getClassLoader(),
+               GridDeploymentResourceTestTask.class.getName(),
+               GridDeploymentResourceTestJob.class.getName()
+           );
+
+           // The same name but different classes/ class loaders.
+           Class<? extends ComputeTask<String, Integer>> taskCls1 = (Class<? extends ComputeTask<String, Integer>>)
+               ldr1.loadClass(GridDeploymentResourceTestTask.class.getName());
+
+           Class<? extends ComputeTask<String, Integer>> taskCls2 = (Class<? extends ComputeTask<String, Integer>>)
+               ldr2.loadClass(GridDeploymentResourceTestTask.class.getName());
+
+           if (byCls) {
+               Integer res1 = ignite.compute().execute(taskCls1, null);
+               Integer res2 = ignite.compute().execute(taskCls2, null);
+
+               assert res1 != null;
+               assert res2 != null;
+
+               assert res1 == 1 : "Invalid res1: " + res1;
+               assert res2 == 2 : "Invalid res2: " + res2;
+           }
+
+           if (byTask) {
+               Integer res1 = ignite.compute().execute(taskCls1.newInstance(), null);
+               Integer res2 = ignite.compute().execute(taskCls2.newInstance(), null);
+
+               assert res1 != null;
+               assert res2 != null;
+
+               assert res1 == 1 : "Invalid res1: " + res1;
+               assert res2 == 2 : "Invalid res2: " + res2;
+           }
+
+           if (byName) {
+               ignite.compute().localDeployTask(taskCls1, ldr1);
+
+               Integer res1 = (Integer) ignite.compute().execute(taskCls1.getName(), null);
+
+               ignite.compute().localDeployTask(taskCls2, ldr2);
+
+               Integer res2 = (Integer) ignite.compute().execute(taskCls2.getName(), null);
+
+               assert res1 != null;
+               assert res2 != null;
+
+               assert res1 == 1 : "Invalid res1: " + res1;
+               assert res2 == 2 : "Invalid res2: " + res2;
+           }
+       }
+       finally {
+           stopGrid(ignite);
+       }
+   }
+
+    /**
+     * @param byCls If {@code true} than executes task by Class.
+     * @param byTask If {@code true} than executes task instance.
+     * @param byName If {@code true} than executes task by class name.
+     * @throws Exception If test failed.
+     */
+    @SuppressWarnings("unchecked")
+    private void execExplicitDeployP2P(boolean byCls, boolean byTask, boolean byName) throws Exception {
+       Ignite ignite1 = null;
+       Ignite ignite2 = null;
+
+       try {
+           ignite1 = startGrid(1);
+           ignite2 = startGrid(2);
+
+           ClassLoader ldr1 = new GridTestClassLoader(
+               Collections.singletonMap("testResource", "1"),
+               getClass().getClassLoader(),
+               GridDeploymentResourceTestTask.class.getName(),
+               GridDeploymentResourceTestJob.class.getName()
+           );
+
+           ClassLoader ldr2 = new GridTestClassLoader(
+               Collections.singletonMap("testResource", "2"),
+               getClass().getClassLoader(),
+               GridDeploymentResourceTestTask.class.getName(),
+               GridDeploymentResourceTestJob.class.getName()
+           );
+
+           Class<? extends ComputeTask<String, Integer>> taskCls = (Class<? extends ComputeTask<String, Integer>>)
+               ldr2.loadClass(GridDeploymentResourceTestTask.class.getName());
+
+           if (byCls) {
+               ignite1.compute().localDeployTask(taskCls, ldr1);
+
+               // Even though the task is deployed with resource class loader,
+               // when we execute it, it will be redeployed with task class-loader.
+               Integer res = ignite1.compute().execute(taskCls, null);
+
+               assert res != null;
+               assert res == 2 : "Invalid response: " + res;
+           }
+
+
+           if (byTask) {
+               ignite1.compute().localDeployTask(taskCls, ldr1);
+
+               // Even though the task is deployed with resource class loader,
+               // when we execute it, it will be redeployed with task class-loader.
+               Integer res = ignite1.compute().execute(taskCls.newInstance(), null);
+
+               assert res != null;
+               assert res == 2 : "Invalid response: " + res;
+           }
+
+           if (byName) {
+               ignite1.compute().localDeployTask(taskCls, ldr1);
+
+               // Even though the task is deployed with resource class loader,
+               // when we execute it, it will be redeployed with task class-loader.
+               Integer res = (Integer) ignite1.compute().execute(taskCls.getName(), null);
+
+               assert res != null;
+               assert res == 1 : "Invalid response: " + res;
+           }
+       }
+       finally {
+           stopGrid(ignite2);
+           stopGrid(ignite1);
+       }
+    }
+
+    /**
+     * @param byCls If {@code true} than executes task by Class.
+     * @param byTask If {@code true} than executes task instance.
+     * @param byName If {@code true} than executes task by class name.
+     * @throws Exception If test failed.
+     */
+   @SuppressWarnings("unchecked")
+   private void execImplicitDeployP2P(boolean byCls, boolean byTask, boolean byName) throws Exception {
+      Ignite ignite1 = null;
+      Ignite ignite2 = null;
+
+      try {
+          ignite1 = startGrid(1);
+          ignite2 = startGrid(2);
+
+          ClassLoader ldr1 = new GridTestClassLoader(
+              Collections.singletonMap("testResource", "1"),
+              getClass().getClassLoader(),
+              GridDeploymentResourceTestTask.class.getName(),
+              GridDeploymentResourceTestJob.class.getName()
+          );
+
+          ClassLoader ldr2 = new GridTestClassLoader(
+              Collections.singletonMap("testResource", "2"),
+              getClass().getClassLoader(),
+              GridDeploymentResourceTestTask.class.getName(),
+              GridDeploymentResourceTestJob.class.getName()
+          );
+
+          Class<? extends ComputeTask<String, Integer>> taskCls1 = (Class<? extends ComputeTask<String, Integer>>)
+              ldr1.loadClass(GridDeploymentResourceTestTask.class.getName());
+
+          Class<? extends ComputeTask<String, Integer>> taskCls2 = (Class<? extends ComputeTask<String, Integer>>)
+              ldr2.loadClass(GridDeploymentResourceTestTask.class.getName());
+
+          if (byCls) {
+              Integer res1 = ignite1.compute().execute(taskCls1, null);
+              Integer res2 = ignite1.compute().execute(taskCls2, null);
+
+              assert res1 != null;
+              assert res2 != null;
+
+              assert res1 == 1 : "Invalid res1: " + res1;
+              assert res2 == 2 : "Invalid res2: " + res2;
+          }
+
+          if (byTask) {
+              Integer res1 = ignite1.compute().execute(taskCls1.newInstance(), null);
+              Integer res2 = ignite1.compute().execute(taskCls2.newInstance(), null);
+
+              assert res1 != null;
+              assert res2 != null;
+
+              assert res1 == 1 : "Invalid res1: " + res1;
+              assert res2 == 2 : "Invalid res2: " + res2;
+          }
+
+          if (byName) {
+              ignite1.compute().localDeployTask(taskCls1, ldr1);
+
+              Integer res1 = (Integer) ignite1.compute().execute(taskCls1.getName(), null);
+
+              ignite1.compute().localDeployTask(taskCls2, ldr2);
+
+              Integer res2 = (Integer) ignite1.compute().execute(taskCls2.getName(), null);
+
+              assert res1 != null;
+              assert res2 != null;
+
+              assert res1 == 1 : "Invalid res1: " + res1;
+              assert res2 == 2 : "Invalid res2: " + res2;
+          }
+      }
+      finally {
+          stopGrid(ignite1);
+          stopGrid(ignite2);
+      }
+   }
+
+    /**
+     * We use custom name to avoid auto-deployment in the same VM.
+     */
+    @SuppressWarnings({"PublicInnerClass"})
+    @ComputeTaskName("GridDeploymentResourceTestTask")
+    public static class GridDeploymentResourceTestTask extends ComputeTaskAdapter<String, Integer> {
+        /** */
+        @IgniteInstanceResource
+        private Ignite ignite;
+
+        /** {@inheritDoc} */
+        @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, String arg) throws IgniteCheckedException {
+            Map<ComputeJobAdapter, ClusterNode> map = new HashMap<>(subgrid.size());
+
+            boolean ignoreLocNode = false;
+
+            UUID locId = ignite.configuration().getNodeId();
+
+            if (subgrid.size() == 1)
+                assert subgrid.get(0).id().equals(locId) : "Wrong node id.";
+            else
+                ignoreLocNode = true;
+
+            for (ClusterNode node : subgrid) {
+                // Ignore local node.
+                if (ignoreLocNode && node.id().equals(locId))
+                    continue;
+
+                map.put(new GridDeploymentResourceTestJob(), node);
+            }
+
+            return map;
+        }
+
+        /** {@inheritDoc} */
+        @Override public Integer reduce(List<ComputeJobResult> results) throws IgniteCheckedException {
+            return results.get(0).getData();
+        }
+    }
+
+    /**
+     * Simple job for this test.
+     */
+    @SuppressWarnings({"PublicInnerClass"})
+    public static final class GridDeploymentResourceTestJob extends ComputeJobAdapter {
+        /** */
+        @IgniteLoggerResource
+        private IgniteLogger log;
+
+        /** {@inheritDoc} */
+        @Override public Serializable execute() throws IgniteCheckedException {
+            if (log.isInfoEnabled())
+                log.info("Executing grid job: " + this);
+
+            try {
+                ClassLoader ldr = Thread.currentThread().getContextClassLoader();
+
+                if (log.isInfoEnabled())
+                    log.info("Loader (inside job): " + ldr);
+
+                InputStream in = ldr.getResourceAsStream("testResource");
+
+                if (in != null) {
+                    Reader reader = new InputStreamReader(in);
+
+                    try {
+                        char res = (char)reader.read();
+
+                        return Integer.parseInt(Character.toString(res));
+                    }
+                    finally {
+                        U.close(in, null);
+                    }
+                }
+
+                return null;
+            }
+            catch (IOException e) {
+                throw new IgniteCheckedException("Failed to execute job.", e);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 6b3ffc8..a4d6f6f 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
@@ -112,7 +112,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
     private GridCheckpointManager checkpoints(Ignite ignite) {
         assert ignite != null;
 
-        return ((GridKernal) ignite).context().checkpoint();
+        return ((IgniteKernal) ignite).context().checkpoint();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
index aeed502..46a380f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/communication/GridCommunicationSendMessageSelfTest.java
@@ -109,8 +109,8 @@ public class GridCommunicationSendMessageSelfTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     private void doSend() throws Exception {
-        GridIoManager mgr0 = ((GridKernal)grid(0)).context().io();
-        GridIoManager mgr1 = ((GridKernal)grid(1)).context().io();
+        GridIoManager mgr0 = ((IgniteKernal)grid(0)).context().io();
+        GridIoManager mgr1 = ((IgniteKernal)grid(1)).context().io();
 
         String topic = "test-topic";
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
index 761809b..deb3edf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerAliveCacheSelfTest.java
@@ -165,7 +165,7 @@ public class GridDiscoveryManagerAliveCacheSelfTest extends GridCommonAbstractTe
             assertEquals(PERM_NODES_CNT, g.cluster().nodes().size());
 
         for (final Ignite g : alive) {
-            GridKernal k = (GridKernal)g;
+            IgniteKernal k = (IgniteKernal)g;
 
             GridDiscoveryManager discoMgr = k.context().discovery();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
index 3c07eb4..090bee8 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/discovery/GridDiscoveryManagerSelfTest.java
@@ -86,7 +86,7 @@ public class GridDiscoveryManagerSelfTest extends GridCommonAbstractTest {
      * @throws Exception If failed.
      */
     public void testHasNearCache() throws Exception {
-        GridKernal g0 = (GridKernal)startGrid(0); // PARTITIONED_ONLY cache.
+        IgniteKernal g0 = (IgniteKernal)startGrid(0); // PARTITIONED_ONLY cache.
 
         assertFalse(g0.context().discovery().hasNearCache(CACHE_NAME, 0));
         assertFalse(g0.context().discovery().hasNearCache(null, 0));
@@ -94,7 +94,7 @@ public class GridDiscoveryManagerSelfTest extends GridCommonAbstractTest {
         assertFalse(g0.context().discovery().hasNearCache(CACHE_NAME, 1));
         assertFalse(g0.context().discovery().hasNearCache(null, 1));
 
-        GridKernal g1 = (GridKernal)startGrid(1); // NEAR_ONLY cache.
+        IgniteKernal g1 = (IgniteKernal)startGrid(1); // NEAR_ONLY cache.
 
         assertFalse(g0.context().discovery().hasNearCache(CACHE_NAME, 1));
         assertTrue(g0.context().discovery().hasNearCache(CACHE_NAME, 2));
@@ -104,7 +104,7 @@ public class GridDiscoveryManagerSelfTest extends GridCommonAbstractTest {
         assertTrue(g1.context().discovery().hasNearCache(CACHE_NAME, 2));
         assertTrue(g1.context().discovery().hasNearCache(null, 2));
 
-        GridKernal g2 = (GridKernal)startGrid(2); // NEAR_PARTITIONED cache.
+        IgniteKernal g2 = (IgniteKernal)startGrid(2); // NEAR_PARTITIONED cache.
 
         assertFalse(g0.context().discovery().hasNearCache(CACHE_NAME, 1));
         assertTrue(g0.context().discovery().hasNearCache(CACHE_NAME, 2));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
index b422e7a..4158f18 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/managers/swapspace/GridSwapSpaceManagerSelfTest.java
@@ -67,7 +67,7 @@ public class GridSwapSpaceManagerSelfTest extends GridCommonAbstractTest {
     private GridSwapSpaceManager getSwapSpaceManager(Ignite ignite) {
         assert ignite != null;
 
-        return ((GridKernal) ignite).context().swap();
+        return ((IgniteKernal) ignite).context().swap();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
index e4de7c1..5d3877d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/affinity/GridAffinityProcessorAbstractSelfTest.java
@@ -111,8 +111,8 @@ public abstract class GridAffinityProcessorAbstractSelfTest extends GridCommonAb
     public void testAffinityProcessor() throws Exception {
         Random rnd = new Random();
 
-        final GridKernal grid1 = (GridKernal)grid(rnd.nextInt(NODES_CNT)); // With cache.
-        GridKernal grid2 = (GridKernal)grid(NODES_CNT + rnd.nextInt(NODES_CNT)); // Without cache.
+        final IgniteKernal grid1 = (IgniteKernal)grid(rnd.nextInt(NODES_CNT)); // With cache.
+        IgniteKernal grid2 = (IgniteKernal)grid(NODES_CNT + rnd.nextInt(NODES_CNT)); // Without cache.
 
         assertEquals(NODES_CNT * 2, grid1.nodes().size());
         assertEquals(NODES_CNT * 2, grid2.nodes().size());
@@ -168,7 +168,7 @@ public abstract class GridAffinityProcessorAbstractSelfTest extends GridCommonAb
      * @throws Exception In case of any exception.
      */
     public void testPerformance() throws Exception {
-        GridKernal grid = (GridKernal)grid(0);
+        IgniteKernal grid = (IgniteKernal)grid(0);
         GridAffinityProcessor aff = grid.context().affinity();
 
         int keysSize = 1000000;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 5c4e4d5..4f55e2f 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
@@ -3242,7 +3242,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testRemoveAfterClear() throws Exception {
-        GridEx grid = grid(0);
+        IgniteEx grid = grid(0);
 
         CacheDistributionMode distroMode = grid.cache(null).configuration().getDistributionMode();
 
@@ -3290,7 +3290,7 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
      * @throws Exception If failed.
      */
     public void testRemoveFilteredAfterClear() throws Exception {
-        GridEx grid = grid(0);
+        IgniteEx grid = grid(0);
 
         CacheDistributionMode distroMode = grid.cache(null).configuration().getDistributionMode();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
index a18bb9a..0f9a11f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAbstractSelfTest.java
@@ -388,7 +388,7 @@ public abstract class GridCacheAbstractSelfTest extends GridCommonAbstractTest {
      * @return Cache context.
      */
     protected GridCacheContext<String, Integer> context(int idx) {
-        return ((GridKernal)grid(idx)).<String, Integer>internalCache().context();
+        return ((IgniteKernal)grid(idx)).<String, Integer>internalCache().context();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java
index 24933cf..1bf6d8e 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheAffinityApiSelfTest.java
@@ -60,14 +60,14 @@ public class GridCacheAffinityApiSelfTest extends GridCacheAbstractSelfTest {
      * @return Affinity.
      */
     private CacheAffinityFunction affinity() {
-        return ((GridKernal)grid(0)).internalCache().configuration().getAffinity();
+        return ((IgniteKernal)grid(0)).internalCache().configuration().getAffinity();
     }
 
     /**
      * @return Affinity mapper.
      */
     private CacheAffinityKeyMapper affinityMapper() {
-        return ((GridKernal)grid(0)).internalCache().configuration().getAffinityMapper();
+        return ((IgniteKernal)grid(0)).internalCache().configuration().getAffinityMapper();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
index 3e537c5..570c133 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
@@ -442,7 +442,7 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
 
             doWork();
 
-            GridNearCacheAdapter near = (GridNearCacheAdapter)((GridKernal) ignite).internalCache();
+            GridNearCacheAdapter near = (GridNearCacheAdapter)((IgniteKernal) ignite).internalCache();
             GridDhtCacheAdapter dht = near.dht();
 
             long start = cntrs.get2().get();
@@ -570,7 +570,7 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
                     for (Ignite g : G.allGrids()) {
                         if (g.name().contains("server")) {
                             GridNearCacheAdapter<CacheAffinityKey<String>, Object> near =
-                                (GridNearCacheAdapter<CacheAffinityKey<String>, Object>)((GridKernal)g).
+                                (GridNearCacheAdapter<CacheAffinityKey<String>, Object>)((IgniteKernal)g).
                                     <CacheAffinityKey<String>, Object>internalCache();
                             GridDhtCacheAdapter<CacheAffinityKey<String>, Object> dht = near.dht();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryVersionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryVersionSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryVersionSelfTest.java
index 77b0c0b..6e9ee08 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryVersionSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheEntryVersionSelfTest.java
@@ -103,7 +103,7 @@ public class GridCacheEntryVersionSelfTest extends GridCommonAbstractTest {
             grid(0).cache(null).putAll(map);
 
             for (int g = 0; g < 3; g++) {
-                GridKernal grid = (GridKernal)grid(g);
+                IgniteKernal grid = (IgniteKernal)grid(g);
 
                 for (Integer key : map.keySet()) {
                     GridCacheAdapter<Object, Object> cache = grid.internalCache();
@@ -130,7 +130,7 @@ public class GridCacheEntryVersionSelfTest extends GridCommonAbstractTest {
             grid(0).cache(null).putAll(map);
 
             for (int g = 0; g < 4; g++) {
-                GridKernal grid = (GridKernal)grid(g);
+                IgniteKernal grid = (IgniteKernal)grid(g);
 
                 for (Integer key : map.keySet()) {
                     GridCacheAdapter<Object, Object> cache = grid.internalCache();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 90487ba..a6b566a 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
@@ -44,7 +44,7 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
     private static final int GRID_CNT = 1;
 
     /** Grid kernal. */
-    private GridKernal grid;
+    private IgniteKernal grid;
 
     /** {@inheritDoc} */
     @Override protected int gridCount() {
@@ -53,7 +53,7 @@ public class GridCacheFinishPartitionsSelfTest extends GridCacheAbstractSelfTest
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        grid = (GridKernal)grid(0);
+        grid = (IgniteKernal)grid(0);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 40aa2bf..4bddd4a 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
@@ -461,7 +461,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
     private void checkSanityCheckDisabled(final IgniteTxConcurrency concurrency) throws Exception {
         assert !sanityCheckEnabled();
 
-        GridEx grid = grid(0);
+        IgniteEx grid = grid(0);
 
         final UUID affinityKey = primaryKeyForCache(grid);
 
@@ -545,7 +545,7 @@ public abstract class GridCacheGroupLockAbstractSelfTest extends GridCommonAbstr
             assertEquals("Unexpected number of unlock events: " + unlocks.affectedKeys(), 0,
                 unlocks.affectedKeys().size());
 
-            GridCacheAdapter<Object, Object> cacheAdapter = ((GridKernal)grid(0)).internalCache();
+            GridCacheAdapter<Object, Object> cacheAdapter = ((IgniteKernal)grid(0)).internalCache();
 
             GridCacheAffinityManager<Object, Object> affMgr = cacheAdapter.context().affinity();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLeakTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLeakTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLeakTest.java
index 1d9c1f6..8f1c3af 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLeakTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheLeakTest.java
@@ -125,7 +125,7 @@ public class GridCacheLeakTest extends GridCommonAbstractTest {
 
                 if (i % 5000 == 0) {
                     for (int g = 0; g < 3; g++) {
-                        GridCacheConcurrentMap<Object, Object> map = ((GridKernal)grid(g)).internalCache(CACHE_NAME).map();
+                        GridCacheConcurrentMap<Object, Object> map = ((IgniteKernal)grid(g)).internalCache(CACHE_NAME).map();
 
                         info("Map size for cache [g=" + g + ", size=" + map.size() +
                             ", pubSize=" + map.publicSize() + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 b7f8239..9143160 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
@@ -135,7 +135,7 @@ public class GridCacheMixedPartitionExchangeSelfTest extends GridCommonAbstractT
 
             // Check all grids have all exchange futures completed.
             for (int i = 0; i < 4; i++) {
-                GridKernal grid = (GridKernal)grid(i);
+                IgniteKernal grid = (IgniteKernal)grid(i);
 
                 GridCacheContext<Object, Object> cctx = grid.internalCache(null).context();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 7eebab8..f9add9e 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
@@ -107,7 +107,7 @@ public class GridCacheMultiUpdateLockSelfTest extends GridCommonAbstractTest {
         startGrids(3);
 
         try {
-            GridKernal g = (GridKernal)grid(0);
+            IgniteKernal g = (IgniteKernal)grid(0);
 
             GridCacheContext<Object, Object> cctx = g.internalCache().context();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
index e46511b..0a676be 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccFlagsTest.java
@@ -32,7 +32,7 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridCacheMvccFlagsTest extends GridCommonAbstractTest {
     /** Grid. */
-    private GridKernal grid;
+    private IgniteKernal grid;
 
     /**
      *
@@ -43,7 +43,7 @@ public class GridCacheMvccFlagsTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        grid = (GridKernal)grid();
+        grid = (IgniteKernal)grid();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManagerSelfTest.java
index b6d1fdb..b12b8bc 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccManagerSelfTest.java
@@ -104,8 +104,8 @@ public class GridCacheMvccManagerSelfTest extends GridCommonAbstractTest {
             tx.commit();
 
             for (int i = 0; i < gridCnt; i++) {
-                assert ((GridKernal)grid(i)).internalCache().context().mvcc().localCandidates().isEmpty();
-                assert ((GridKernal)grid(i)).internalCache().context().mvcc().remoteCandidates().isEmpty();
+                assert ((IgniteKernal)grid(i)).internalCache().context().mvcc().localCandidates().isEmpty();
+                assert ((IgniteKernal)grid(i)).internalCache().context().mvcc().remoteCandidates().isEmpty();
             }
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
index f2d17af..d97c583 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccPartitionedSelfTest.java
@@ -37,7 +37,7 @@ import static org.apache.ignite.cache.CacheDistributionMode.*;
  */
 public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
     /** Grid. */
-    private GridKernal grid;
+    private IgniteKernal grid;
 
     /** VM ip finder for TCP discovery. */
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
@@ -51,7 +51,7 @@ public class GridCacheMvccPartitionedSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        grid = (GridKernal)grid();
+        grid = (IgniteKernal)grid();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
index 86e43b0..2044753 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheMvccSelfTest.java
@@ -36,7 +36,7 @@ import static org.apache.ignite.cache.CacheMode.*;
  */
 public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
     /** Grid. */
-    private GridKernal grid;
+    private IgniteKernal grid;
 
     /** VM ip finder for TCP discovery. */
     private static TcpDiscoveryIpFinder ipFinder = new TcpDiscoveryVmIpFinder(true);
@@ -50,7 +50,7 @@ public class GridCacheMvccSelfTest extends GridCommonAbstractTest {
 
     /** {@inheritDoc} */
     @Override protected void beforeTest() throws Exception {
-        grid = (GridKernal)grid();
+        grid = (IgniteKernal)grid();
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheNestedTxAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheNestedTxAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheNestedTxAbstractTest.java
index 7121ee1..53c5e93 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheNestedTxAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheNestedTxAbstractTest.java
@@ -108,7 +108,7 @@ public class GridCacheNestedTxAbstractTest extends GridCommonAbstractTest {
     public void testTwoTx() throws Exception {
         final GridCache<String, Integer> c = grid(0).cache(null);
 
-        GridKernalContext ctx = ((GridKernal)grid(0)).context();
+        GridKernalContext ctx = ((IgniteKernal)grid(0)).context();
 
         c.put(CNTR_KEY, 0);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheObjectToStringSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheObjectToStringSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheObjectToStringSelfTest.java
index 7d4b5cb..a60ffc6 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheObjectToStringSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheObjectToStringSelfTest.java
@@ -160,7 +160,7 @@ public class GridCacheObjectToStringSelfTest extends GridCommonAbstractTest {
                 cache.put(i, i);
 
             for (int i = 0; i < 10; i++) {
-                GridCacheEntryEx<Object, Object> entry = ((GridKernal)g).context().cache().internalCache().peekEx(i);
+                GridCacheEntryEx<Object, Object> entry = ((IgniteKernal)g).context().cache().internalCache().peekEx(i);
 
                 if (entry != null)
                     assertFalse("Entry is locked after implicit transaction commit: " + entry, entry.lockedByAny());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOrderedPreloadingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOrderedPreloadingSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOrderedPreloadingSelfTest.java
index c2cab27..d7a8a75 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOrderedPreloadingSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheOrderedPreloadingSelfTest.java
@@ -137,7 +137,7 @@ public class GridCacheOrderedPreloadingSelfTest extends GridCommonAbstractTest {
 
             // For first node in topology replicated preloader gets completed right away.
             for (int i = 1; i < GRID_CNT; i++) {
-                GridKernal kernal = (GridKernal)grid(i);
+                IgniteKernal kernal = (IgniteKernal)grid(i);
 
                 GridFutureAdapter<?> fut1 = (GridFutureAdapter<?>)kernal.internalCache(FIRST_CACHE_NAME).preloader()
                     .syncFuture();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheP2PUndeploySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheP2PUndeploySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheP2PUndeploySelfTest.java
index a819ab8..3ef9906 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheP2PUndeploySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheP2PUndeploySelfTest.java
@@ -184,7 +184,7 @@ public class GridCacheP2PUndeploySelfTest extends GridCommonAbstractTest {
      * @return Size.
      * @throws IgniteCheckedException If failed.
      */
-    private long size(String cacheName, GridKernal g) throws IgniteCheckedException {
+    private long size(String cacheName, IgniteKernal g) throws IgniteCheckedException {
         if (offheap)
             return g.cache(cacheName).offHeapEntriesCount();
 
@@ -204,7 +204,7 @@ public class GridCacheP2PUndeploySelfTest extends GridCommonAbstractTest {
 
         try {
             Ignite ignite1 = startGrid(1);
-            GridKernal grid2 = (GridKernal)startGrid(2);
+            IgniteKernal grid2 = (IgniteKernal)startGrid(2);
 
             GridCache<Integer, Object> cache1 = ignite1.cache(cacheName);
             GridCache<Integer, Object> cache2 = grid2.cache(cacheName);
@@ -273,7 +273,7 @@ public class GridCacheP2PUndeploySelfTest extends GridCommonAbstractTest {
      * @param grid Kernal.
      * @return Name for swap space.
      */
-    private String swapSpaceName(String cacheName, GridKernal grid) {
+    private String swapSpaceName(String cacheName, IgniteKernal grid) {
         GridCacheContext<Object, Object> cctx = grid.internalCache(cacheName).context();
 
         return CU.swapSpaceName(cctx.isNear() ? cctx.near().dht().context() : cctx);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
index f25218a..492c621 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCachePartitionedGetSelfTest.java
@@ -230,7 +230,7 @@ public class GridCachePartitionedGetSelfTest extends GridCommonAbstractTest {
                 g.cache(null).put(KEY, VAL);
 
                 // Register listener.
-                ((GridKernal)g).context().io().addMessageListener(
+                ((IgniteKernal)g).context().io().addMessageListener(
                     TOPIC_CACHE,
                     new GridMessageListener() {
                         @Override public void onMessage(UUID nodeId, Object msg) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 6902ce2..5cf7383 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
@@ -205,7 +205,7 @@ public class GridCachePreloadingEvictionsSelfTest extends GridCommonAbstractTest
      * @return Random entry from cache.
      */
     @Nullable private CacheEntry<Integer, Object> randomEntry(Ignite g) {
-        GridKernal g1 = (GridKernal)g;
+        IgniteKernal g1 = (IgniteKernal)g;
 
         return g1.<Integer, Object>internalCache().randomEntry();
     }
@@ -216,8 +216,8 @@ public class GridCachePreloadingEvictionsSelfTest extends GridCommonAbstractTest
      * @throws Exception If failed.
      */
     private void checkCachesConsistency(Ignite ignite1, Ignite ignite2) throws Exception {
-        GridKernal g1 = (GridKernal) ignite1;
-        GridKernal g2 = (GridKernal) ignite2;
+        IgniteKernal g1 = (IgniteKernal) ignite1;
+        IgniteKernal g2 = (IgniteKernal) ignite2;
 
         GridCacheAdapter<Integer, Object> cache1 = g1.internalCache();
         GridCacheAdapter<Integer, Object> cache2 = g2.internalCache();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryInternalKeysSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryInternalKeysSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryInternalKeysSelfTest.java
index 10e60c2..ade174f 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryInternalKeysSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheQueryInternalKeysSelfTest.java
@@ -85,7 +85,7 @@ public class GridCacheQueryInternalKeysSelfTest extends GridCacheAbstractSelfTes
                     assertNotNull(g);
 
                     assertTrue("Affinity node doesn't contain internal key [key=" + internalKey + ", node=" + n + ']',
-                        ((GridNearCacheAdapter)((GridKernal)g).internalCache()).dht().containsKey(internalKey, null));
+                        ((GridNearCacheAdapter)((IgniteKernal)g).internalCache()).dht().containsKey(internalKey, null));
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSlowTxWarnTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSlowTxWarnTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSlowTxWarnTest.java
index 140063a..4d031ba 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSlowTxWarnTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSlowTxWarnTest.java
@@ -74,7 +74,7 @@ public class GridCacheSlowTxWarnTest extends GridCommonAbstractTest {
      */
     public void testWarningOutput() throws Exception {
         try {
-            GridKernal g = (GridKernal)startGrid(1);
+            IgniteKernal g = (IgniteKernal)startGrid(1);
 
             info(">>> Slow tx timeout is not set, long-live txs simulated.");
 
@@ -109,7 +109,7 @@ public class GridCacheSlowTxWarnTest extends GridCommonAbstractTest {
     private void checkCache(Ignite g, String cacheName, boolean simulateTimeout,
         boolean configureTimeout) throws Exception {
         if (configureTimeout) {
-            GridCacheAdapter<Integer, Integer> cache = ((GridKernal)g).internalCache(cacheName);
+            GridCacheAdapter<Integer, Integer> cache = ((IgniteKernal)g).internalCache(cacheName);
 
             cache.context().tm().slowTxWarnTimeout(500);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreValueBytesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreValueBytesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreValueBytesSelfTest.java
index 1fcda7e..94fd587 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreValueBytesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheStoreValueBytesSelfTest.java
@@ -82,7 +82,7 @@ public class GridCacheStoreValueBytesSelfTest extends GridCommonAbstractTest {
 
         c.put(1, "Cached value");
 
-        GridCacheEntryEx<Object, Object> entry = ((GridKernal)g1).internalCache().peekEx(1);
+        GridCacheEntryEx<Object, Object> entry = ((IgniteKernal)g1).internalCache().peekEx(1);
 
         assert entry != null;
         assert entry.valueBytes().isNull();
@@ -103,7 +103,7 @@ public class GridCacheStoreValueBytesSelfTest extends GridCommonAbstractTest {
 
         c.put(1, "Cached value");
 
-        GridCacheEntryEx<Object, Object> entry = ((GridKernal)g1).internalCache().peekEx(1);
+        GridCacheEntryEx<Object, Object> entry = ((IgniteKernal)g1).internalCache().peekEx(1);
 
         assert entry != null;
         assert entry.valueBytes() != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
index b983c07..7f49e27 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheSwapReloadSelfTest.java
@@ -207,7 +207,7 @@ public class GridCacheSwapReloadSelfTest extends GridCommonAbstractTest {
      * @return Swap space name.
      */
     private String spaceName() {
-        return CU.swapSpaceName(((GridKernal)grid()).internalCache().context());
+        return CU.swapSpaceName(((IgniteKernal)grid()).internalCache().context());
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 b0b1e50..6b97f9a 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
@@ -38,7 +38,7 @@ public class GridCacheTtlManagerLoadTest extends GridCacheTtlManagerSelfTest {
     public void testLoad() throws Exception {
         cacheMode = REPLICATED;
 
-        final GridKernal g = (GridKernal)startGrid(0);
+        final IgniteKernal g = (IgniteKernal)startGrid(0);
 
         try {
             final AtomicBoolean stop = new AtomicBoolean();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerSelfTest.java
index 1709fac..adb2d40 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/GridCacheTtlManagerSelfTest.java
@@ -91,7 +91,7 @@ public class GridCacheTtlManagerSelfTest extends GridCommonAbstractTest {
     private void checkTtl(CacheMode mode) throws Exception {
         cacheMode = mode;
 
-        final GridKernal g = (GridKernal)startGrid(0);
+        final IgniteKernal g = (IgniteKernal)startGrid(0);
 
         try {
             final String key = "key";

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/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 6f0b1d6..095e621 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
@@ -77,7 +77,7 @@ public abstract class IgniteCacheEntryListenerAbstractTest extends IgniteCacheAb
         super.afterTest();
 
         for (int i = 0; i < gridCount(); i++) {
-            GridContinuousProcessor proc = ((GridKernal)grid(i)).context().continuous();
+            GridContinuousProcessor proc = ((IgniteKernal)grid(i)).context().continuous();
 
             ConcurrentMap<?, ?> syncMsgFuts = GridTestUtils.getFieldValue(proc, "syncMsgFuts");
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
index c4dca58..00d0ccf 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxAbstractTest.java
@@ -280,7 +280,7 @@ abstract class IgniteTxAbstractTest extends GridCommonAbstractTest {
         if (printMemoryStats()) {
             if (cntr.getAndIncrement() % 100 == 0)
                 // Print transaction memory stats.
-                ((GridKernal)grid(gridIdx)).internalCache().context().tm().printMemoryStats();
+                ((IgniteKernal)grid(gridIdx)).internalCache().context().tm().printMemoryStats();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConcurrentGetAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConcurrentGetAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConcurrentGetAbstractTest.java
index ee07f72..421c195 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConcurrentGetAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxConcurrentGetAbstractTest.java
@@ -20,7 +20,6 @@ package org.apache.ignite.internal.processors.cache;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.internal.*;
-import org.apache.ignite.internal.processors.cache.*;
 import org.apache.ignite.transactions.*;
 import org.apache.ignite.internal.processors.cache.distributed.dht.*;
 import org.apache.ignite.internal.processors.cache.distributed.near.*;
@@ -74,7 +73,7 @@ public abstract class IgniteTxConcurrentGetAbstractTest extends GridCommonAbstra
      * @return Near cache.
      */
     GridNearCacheAdapter<String, Integer> near(Ignite g) {
-        return (GridNearCacheAdapter<String, Integer>)((GridKernal)g).<String, Integer>internalCache();
+        return (GridNearCacheAdapter<String, Integer>)((IgniteKernal)g).<String, Integer>internalCache();
     }
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
index 6aa8acb..cecb254 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxExceptionAbstractSelfTest.java
@@ -346,7 +346,7 @@ public abstract class IgniteTxExceptionAbstractSelfTest extends GridCacheAbstrac
         info("Check key: " + key);
 
         for (int i = 0; i < gridCount(); i++) {
-            GridKernal grid = (GridKernal) grid(i);
+            IgniteKernal grid = (IgniteKernal) grid(i);
 
             GridCacheAdapter cache = grid.internalCache(null);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiNodeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiNodeAbstractTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiNodeAbstractTest.java
index 719b7fc..724929d 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiNodeAbstractTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxMultiNodeAbstractTest.java
@@ -129,7 +129,7 @@ public abstract class IgniteTxMultiNodeAbstractTest extends GridCommonAbstractTe
         Ignite g = G.ignite(nodeId);
 
         GridDhtCacheAdapter<Object, Integer> dht =
-            ((GridKernal)g).<Object, Integer>internalCache().context().near().dht();
+            ((IgniteKernal)g).<Object, Integer>internalCache().context().near().dht();
 
         return dht.peekEx(key);
     }
@@ -142,7 +142,7 @@ public abstract class IgniteTxMultiNodeAbstractTest extends GridCommonAbstractTe
     @Nullable private static GridCacheEntryEx<Object, Integer> nearEntry(UUID nodeId, Object key) {
         Ignite g = G.ignite(nodeId);
 
-        GridNearCacheAdapter<Object, Integer> near = ((GridKernal)g).<Object, Integer>internalCache().context().near();
+        GridNearCacheAdapter<Object, Integer> near = ((IgniteKernal)g).<Object, Integer>internalCache().context().near();
 
         return near.peekEx(key);
     }
@@ -460,7 +460,7 @@ public abstract class IgniteTxMultiNodeAbstractTest extends GridCommonAbstractTe
             onRemoveItemQueried(putCntr, ignite, i);
 
             if (i % 50 == 0)
-                ((GridKernal) ignite).internalCache().context().tm().printMemoryStats();
+                ((IgniteKernal) ignite).internalCache().context().tm().printMemoryStats();
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
index afe9afe..ccd0b61 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/IgniteTxStoreExceptionAbstractSelfTest.java
@@ -349,7 +349,7 @@ public abstract class IgniteTxStoreExceptionAbstractSelfTest extends GridCacheAb
         info("Check key: " + key);
 
         for (int i = 0; i < gridCount(); i++) {
-            GridKernal grid = (GridKernal) grid(i);
+            IgniteKernal grid = (IgniteKernal) grid(i);
 
             GridCacheAdapter cache = grid.internalCache(null);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java
index a29d461..30bae26 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheAbstractQueueFailoverDataConsistencySelfTest.java
@@ -365,7 +365,7 @@ public abstract class GridCacheAbstractQueueFailoverDataConsistencySelfTest exte
         CacheAffinity<Object> aff = grid(0).cache(null).affinity();
 
         for (int i = 0; i < gridCount(); i++) {
-            for (GridCacheEntryEx e : ((GridKernal)grid(i)).context().cache().internalCache().map().allEntries0()) {
+            for (GridCacheEntryEx e : ((IgniteKernal)grid(i)).context().cache().internalCache().map().allEntries0()) {
                 if (aff.isPrimary(grid(i).localNode(), e.key()) && e.key() instanceof GridCacheQueueHeaderKey)
                     return i;
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java
index 763135d..e00fa58 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheCountDownLatchSelfTest.java
@@ -392,7 +392,7 @@ public class GridCacheCountDownLatchSelfTest extends GridCommonAbstractTest impl
 
         // Ensure latch is removed on all nodes.
         for (Ignite g : G.allGrids())
-            assert ((GridKernal)g).internalCache(cacheName).context().dataStructures().
+            assert ((IgniteKernal)g).internalCache(cacheName).context().dataStructures().
                 countDownLatch(latchName, 10, true, false) == null;
 
         checkRemovedLatch(latch);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java
index 131e7b4..412f2e4 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheQueueCleanupSelfTest.java
@@ -178,7 +178,7 @@ public class GridCacheQueueCleanupSelfTest extends GridCacheAbstractSelfTest {
 
                 for (int i = 0; i < gridCount(); i++) {
                     Iterator<GridCacheEntryEx<Object, Object>> entries =
-                        ((GridKernal)grid(i)).context().cache().internalCache().map().allEntries0().iterator();
+                        ((IgniteKernal)grid(i)).context().cache().internalCache().map().allEntries0().iterator();
 
                     while (entries.hasNext()) {
                         cnt++;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
index 47d4daa..723a7ab 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetAbstractSelfTest.java
@@ -93,7 +93,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf
         assertSetIteratorsCleared();
 
         for (int i = 0; i < gridCount(); i++) {
-            GridKernal grid = (GridKernal)grid(i);
+            IgniteKernal grid = (IgniteKernal)grid(i);
 
             GridCacheDataStructuresManager ds = grid.internalCache(null).context().dataStructures();
 
@@ -112,7 +112,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf
      */
     private void assertSetIteratorsCleared() {
         for (int i = 0; i < gridCount(); i++) {
-            GridKernal grid = (GridKernal) grid(i);
+            IgniteKernal grid = (IgniteKernal) grid(i);
 
             GridCacheQueryManager queries = grid.internalCache(null).context().queries();
 
@@ -623,7 +623,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf
             UUID setNodeId = null;
 
             for (int i = 0; i < gridCount(); i++) {
-                GridKernal grid = (GridKernal)grid(i);
+                IgniteKernal grid = (IgniteKernal)grid(i);
 
                 Iterator<GridCacheEntryEx<Object, Object>> entries =
                     grid.context().cache().internalCache("noBackupsCache").map().allEntries0().iterator();
@@ -805,7 +805,7 @@ public abstract class GridCacheSetAbstractSelfTest extends GridCacheAbstractSelf
 
         for (int i = 0; i < gridCount(); i++) {
             Iterator<GridCacheEntryEx<Object, Object>> entries =
-                    ((GridKernal)grid(i)).context().cache().internalCache().map().allEntries0().iterator();
+                    ((IgniteKernal)grid(i)).context().cache().internalCache().map().allEntries0().iterator();
 
             while (entries.hasNext()) {
                 GridCacheEntryEx<Object, Object> entry = entries.next();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/c323a8e7/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
index efcc227..c6a173c 100644
--- a/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/internal/processors/cache/datastructures/GridCacheSetFailoverAbstractSelfTest.java
@@ -189,7 +189,7 @@ public class GridCacheSetFailoverAbstractSelfTest extends GridCacheAbstractSelfT
 
             for (int i = 0; i < gridCount(); i++) {
                 Iterator<GridCacheEntryEx<Object, Object>> entries =
-                    ((GridKernal)grid(i)).context().cache().internalCache().map().allEntries0().iterator();
+                    ((IgniteKernal)grid(i)).context().cache().internalCache().map().allEntries0().iterator();
 
                 while (entries.hasNext()) {
                     GridCacheEntryEx<Object, Object> entry = entries.next();