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

[37/64] [abbrv] [partial] incubator-ignite git commit: Rename GridException to IgniteCheckedException, GridRuntimeException to IgniteException.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryEx.java
index bb96abf..e94016f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryEx.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
@@ -25,9 +26,9 @@ import java.util.*;
 public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
     /**
      * @return Memory size.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public int memorySize() throws GridException;
+    public int memorySize() throws IgniteCheckedException;
 
     /**
      * @return {@code True} if entry is internal cache entry.
@@ -103,9 +104,9 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      *        temporary object can used for filter evaluation or transform closure execution and
      *        should not be returned to user.
      * @return Value (unmarshalled if needed).
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public V rawGetOrUnmarshal(boolean tmp) throws GridException;
+    public V rawGetOrUnmarshal(boolean tmp) throws IgniteCheckedException;
 
     /**
      * @return {@code True} if has value or value bytes.
@@ -131,9 +132,9 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * Wraps this map entry into cache entry for filter evaluation inside entry lock.
      *
      * @return Wrapped entry.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public GridCacheEntry<K, V> wrapFilterLocked() throws GridException;
+    public GridCacheEntry<K, V> wrapFilterLocked() throws IgniteCheckedException;
 
     /**
      * @return Entry which is safe to pass into eviction policy.
@@ -174,41 +175,41 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param curVer Current version to match ({@code null} means always match).
      * @param newVer New version to set.
      * @return {@code true} if entry is obsolete.
-     * @throws GridException If swap could not be released.
+     * @throws IgniteCheckedException If swap could not be released.
      */
-    public boolean invalidate(@Nullable GridCacheVersion curVer, GridCacheVersion newVer) throws GridException;
+    public boolean invalidate(@Nullable GridCacheVersion curVer, GridCacheVersion newVer) throws IgniteCheckedException;
 
     /**
      * Invalidates this entry if it passes given filter.
      *
      * @param filter Optional filter that entry should pass before invalidation.
      * @return {@code true} if entry was actually invalidated.
-     * @throws GridException If swap could not be released.
+     * @throws IgniteCheckedException If swap could not be released.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     public boolean invalidate(@Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter)
-        throws GridCacheEntryRemovedException, GridException;
+        throws GridCacheEntryRemovedException, IgniteCheckedException;
 
     /**
      * Optimizes the size of this entry.
      *
      * @param filter Optional filter that entry should pass before invalidation.
      * @throws GridCacheEntryRemovedException If entry was removed.
-     * @throws GridException If operation failed.
+     * @throws IgniteCheckedException If operation failed.
      * @return {@code true} if entry was not being used and could be removed.
      */
     public boolean compact(@Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter)
-        throws GridCacheEntryRemovedException, GridException;
+        throws GridCacheEntryRemovedException, IgniteCheckedException;
 
     /**
      * @param swap Swap flag.
      * @param obsoleteVer Version for eviction.
      * @param filter Optional filter.
      * @return {@code True} if entry could be evicted.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     public boolean evictInternal(boolean swap, GridCacheVersion obsoleteVer,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException;
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException;
 
     /**
      * Evicts entry when batch evict is performed. When called, does not write entry data to swap, but instead
@@ -216,9 +217,9 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      *
      * @param obsoleteVer Version to mark obsolete with.
      * @return Swap entry if this entry was marked obsolete, {@code null} if entry was not evicted.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public GridCacheBatchSwapEntry<K, V> evictInBatchInternal(GridCacheVersion obsoleteVer) throws GridException;
+    public GridCacheBatchSwapEntry<K, V> evictInBatchInternal(GridCacheVersion obsoleteVer) throws IgniteCheckedException;
 
     /**
      * This method should be called each time entry is marked obsolete
@@ -273,7 +274,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      *      together with getting the value is an atomic operation.
      * @param transformClo Transform closure to record event.
      * @return Cached value.
-     * @throws GridException If loading value failed.
+     * @throws IgniteCheckedException If loading value failed.
      * @throws GridCacheEntryRemovedException If entry was removed.
      * @throws GridCacheFilterFailedException If filter failed.
      */
@@ -289,17 +290,17 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
         Object transformClo,
         String taskName,
         IgnitePredicate<GridCacheEntry<K, V>>[] filter)
-        throws GridException, GridCacheEntryRemovedException, GridCacheFilterFailedException;
+        throws IgniteCheckedException, GridCacheEntryRemovedException, GridCacheFilterFailedException;
 
     /**
      * Reloads entry from underlying storage.
      *
      * @param filter Filter for entries.
      * @return Reloaded value.
-     * @throws GridException If reload failed.
+     * @throws IgniteCheckedException If reload failed.
      * @throws GridCacheEntryRemovedException If entry has been removed.
      */
-    @Nullable public V innerReload(IgnitePredicate<GridCacheEntry<K, V>>... filter) throws GridException,
+    @Nullable public V innerReload(IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException,
         GridCacheEntryRemovedException;
 
     /**
@@ -322,7 +323,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param taskName Task name.
      * @return Tuple containing success flag and old value. If success is {@code false},
      *      then value is {@code null}.
-     * @throws GridException If storing value failed.
+     * @throws IgniteCheckedException If storing value failed.
      * @throws GridCacheEntryRemovedException If entry has been removed.
      */
     public GridCacheUpdateTxResult<V> innerSet(
@@ -343,7 +344,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
         @Nullable GridCacheVersion explicitVer,
         @Nullable UUID subjId,
         String taskName
-    ) throws GridException, GridCacheEntryRemovedException;
+    ) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * @param tx Cache transaction.
@@ -361,7 +362,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param taskName Task name.
      * @return Tuple containing success flag and old value. If success is {@code false},
      *      then value is {@code null}.
-     * @throws GridException If remove failed.
+     * @throws IgniteCheckedException If remove failed.
      * @throws GridCacheEntryRemovedException If entry has been removed.
      */
     public GridCacheUpdateTxResult<V> innerRemove(
@@ -378,7 +379,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
         @Nullable GridCacheVersion explicitVer,
         @Nullable UUID subjId,
         String taskName
-    ) throws GridException, GridCacheEntryRemovedException;
+    ) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * @param ver Cache version to set. Entry will be updated only if current version is less then passed version.
@@ -409,7 +410,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      *      fourth is the version to enqueue for deferred delete the fifth is DR conflict context
      *      or {@code null} if conflict resolution was not performed, the last boolean - whether update should be
      *      propagated to backups or not.
-     * @throws GridException If update failed.
+     * @throws IgniteCheckedException If update failed.
      * @throws GridCacheEntryRemovedException If entry is obsolete.
      */
     public GridCacheUpdateAtomicResult<K, V> innerUpdate(
@@ -435,7 +436,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
         boolean intercept,
         @Nullable UUID subjId,
         String taskName
-    ) throws GridException, GridCacheEntryRemovedException;
+    ) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * Update method for local cache in atomic mode.
@@ -453,7 +454,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param subjId Subject ID initiated this update.
      * @param taskName Task name.
      * @return Tuple containing success flag and old value.
-     * @throws GridException If update failed.
+     * @throws IgniteCheckedException If update failed.
      * @throws GridCacheEntryRemovedException If entry is obsolete.
      */
     public IgniteBiTuple<Boolean, V> innerUpdateLocal(
@@ -469,7 +470,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
         boolean intercept,
         @Nullable UUID subjId,
         String taskName
-    ) throws GridException, GridCacheEntryRemovedException;
+    ) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
 
     /**
@@ -479,11 +480,11 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param ver Obsolete version.
      * @param readers Flag to clear readers as well.
      * @param filter Optional entry filter.
-     * @throws GridException If failed to remove from swap.
+     * @throws IgniteCheckedException If failed to remove from swap.
      * @return {@code True} if entry was not being used, passed the filter and could be removed.
      */
     public boolean clear(GridCacheVersion ver, boolean readers,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException;
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException;
 
     /**
      * This locks is called by transaction manager during prepare step
@@ -528,9 +529,9 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      *
      * @param ver Version to set as obsolete.
      * @return {@code True} if entry was marked obsolete.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public boolean markObsoleteIfEmpty(@Nullable GridCacheVersion ver) throws GridException;
+    public boolean markObsoleteIfEmpty(@Nullable GridCacheVersion ver) throws IgniteCheckedException;
 
     /**
      * Sets obsolete flag if entry version equals to {@code ver}.
@@ -547,9 +548,9 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
 
     /**
      * @return Key bytes.
-     * @throws GridException If marshalling failed.
+     * @throws IgniteCheckedException If marshalling failed.
      */
-    public byte[] getOrMarshalKeyBytes() throws GridException;
+    public byte[] getOrMarshalKeyBytes() throws IgniteCheckedException;
 
     /**
      * @return Version.
@@ -598,14 +599,14 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param filter Filter.
      * @param tx Transaction to peek value at (if mode is TX value).
      * @return Peeked value.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      * @throws GridCacheEntryRemovedException If removed.
      * @throws GridCacheFilterFailedException If filter failed.
      */
     @SuppressWarnings({"RedundantTypeArguments"})
     @Nullable public GridTuple<V> peek0(boolean failFast, GridCachePeekMode mode,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, @Nullable GridCacheTxEx<K, V> tx)
-        throws GridCacheEntryRemovedException, GridCacheFilterFailedException, GridException;
+        throws GridCacheEntryRemovedException, GridCacheFilterFailedException, IgniteCheckedException;
 
     /**
      * This method overwrites current in-memory value with new value.
@@ -615,10 +616,10 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      *
      * @param val Value to set.
      * @return Previous value.
-     * @throws GridException If poke operation failed.
+     * @throws IgniteCheckedException If poke operation failed.
      * @throws GridCacheEntryRemovedException if entry was unexpectedly removed.
      */
-    public V poke(V val) throws GridCacheEntryRemovedException, GridException;
+    public V poke(V val) throws GridCacheEntryRemovedException, IgniteCheckedException;
 
     /**
      * Sets new value if current version is <tt>0</tt>
@@ -632,11 +633,11 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param topVer Topology version.
      * @param drType DR type.
      * @return {@code True} if initial value was set.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     public boolean initialValue(V val, @Nullable byte[] valBytes, GridCacheVersion ver, long ttl, long expireTime,
-        boolean preload, long topVer, GridDrType drType) throws GridException, GridCacheEntryRemovedException;
+        boolean preload, long topVer, GridDrType drType) throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * Sets new value if current version is <tt>0</tt> using swap entry data.
@@ -645,11 +646,11 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param key Key.
      * @param unswapped Swap entry to set entry state from.
      * @return {@code True} if  initial value was set.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     public boolean initialValue(K key, GridCacheSwapEntry<V> unswapped)
-        throws GridException, GridCacheEntryRemovedException;
+        throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * Sets new value if passed in version matches the current version
@@ -659,11 +660,11 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param curVer Version to match or {@code null} if match is not required.
      * @param newVer Version to set.
      * @return {@code True} if versioned matched.
-     * @throws GridException If index could not be updated.
+     * @throws IgniteCheckedException If index could not be updated.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     public boolean versionedValue(V val, @Nullable GridCacheVersion curVer, @Nullable GridCacheVersion newVer)
-        throws GridException, GridCacheEntryRemovedException;
+        throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * Checks if the candidate is either owner or pending.
@@ -827,11 +828,11 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      *
      * @param ver Version for which to get value bytes.
      * @return Serialized value bytes.
-     * @throws GridException If serialization failed.
+     * @throws IgniteCheckedException If serialization failed.
      * @throws GridCacheEntryRemovedException If entry was removed.
      */
     @Nullable public GridCacheValueBytes valueBytes(@Nullable GridCacheVersion ver)
-        throws GridException, GridCacheEntryRemovedException;
+        throws IgniteCheckedException, GridCacheEntryRemovedException;
 
     /**
      * @return Expire time, without accounting for transactions or removals.
@@ -870,9 +871,9 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
 
     /**
      * @return Value.
-     * @throws GridException If failed to read from swap storage.
+     * @throws IgniteCheckedException If failed to read from swap storage.
      */
-    @Nullable public V unswap() throws GridException;
+    @Nullable public V unswap() throws IgniteCheckedException;
 
     /**
      * Unswap ignoring flags.
@@ -880,7 +881,7 @@ public interface GridCacheEntryEx<K, V> extends GridMetadataAware {
      * @param ignoreFlags Whether to ignore swap flags.
      * @param needVal If {@code false} then do not need to deserialize value during unswap.
      * @return Value.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    @Nullable public V unswap(boolean ignoreFlags, boolean needVal) throws GridException;
+    @Nullable public V unswap(boolean ignoreFlags, boolean needVal) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
index e1a512a..452e074 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryImpl.java
@@ -9,14 +9,15 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.dht.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.tostring.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -140,7 +141,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
 
         // Try create only if cache allows empty entries.
         if (cached == null)
-            throw new GridRuntimeException("Failed to access cache entry metadata (entry is not present). " +
+            throw new IgniteException("Failed to access cache entry metadata (entry is not present). " +
                 "Put value to cache before accessing metadata: " + key);
 
         this.cached = cached = entryEx(true, topVer);
@@ -177,8 +178,8 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
         try {
             return get();
         }
-        catch (GridException e) {
-            throw new GridRuntimeException(e);
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
     }
 
@@ -187,8 +188,8 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
         try {
             return set(val, CU.<K, V>empty());
         }
-        catch (GridException e) {
-            throw new GridRuntimeException(e);
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
     }
 
@@ -247,14 +248,14 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
         try {
             return peek(MODES_SMART);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             // Should never happen.
-            throw new GridRuntimeException("Unable to perform entry peek() operation.", e);
+            throw new IgniteException("Unable to perform entry peek() operation.", e);
         }
     }
 
     /** {@inheritDoc} */
-    @Override public V peek(@Nullable Collection<GridCachePeekMode> modes) throws GridException {
+    @Override public V peek(@Nullable Collection<GridCachePeekMode> modes) throws IgniteCheckedException {
         return peek0(modes, CU.<K, V>empty(), ctx.atomic() ? null : ctx.tm().localTxx());
     }
 
@@ -263,12 +264,12 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
      * @param filter Optional entry filter.
      * @param tx Transaction to peek at (if mode is TX).
      * @return Peeked value.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings({"unchecked"})
     @Nullable private V peek0(@Nullable GridCachePeekMode mode,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, @Nullable GridCacheTxEx<K, V> tx)
-        throws GridException {
+        throws IgniteCheckedException {
         assert tx == null || tx.local();
 
         if (mode == null)
@@ -336,10 +337,10 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
      * @param filter Optional entry filter.
      * @param tx Transaction to peek at (if modes contains TX value).
      * @return Peeked value.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @Nullable private V peek0(@Nullable Collection<GridCachePeekMode> modes,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, GridCacheTxEx<K, V> tx) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, GridCacheTxEx<K, V> tx) throws IgniteCheckedException {
         if (F.isEmpty(modes))
             return peek0(SMART, filter, tx);
 
@@ -356,7 +357,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V reload() throws GridException {
+    @Nullable @Override public V reload() throws IgniteCheckedException {
         GridCacheProjectionImpl<K, V> old = ctx.gate().enter(proxy.gateProjection());
 
         try {
@@ -390,12 +391,12 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public boolean compact() throws GridException {
+    @Override public boolean compact() throws IgniteCheckedException {
         return proxy.compact(key);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V get() throws GridException {
+    @Nullable @Override public V get() throws IgniteCheckedException {
         return proxy.get(key, isNearEnabled(ctx) ? null : cached, true);
     }
 
@@ -405,7 +406,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V set(V val, IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Nullable @Override public V set(V val, IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         // Should not pass dht entries as to near cache.
         return proxy.put(key, val, isNearEnabled(ctx) ? null : cached, ttl, filter);
     }
@@ -417,7 +418,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public boolean setx(V val, IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Override public boolean setx(V val, IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         // Should not pass dht entries as to near cache.
         return proxy.putx(key, val, isNearEnabled(ctx) ? null : cached, ttl, filter);
     }
@@ -429,7 +430,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V replace(V val) throws GridException {
+    @Nullable @Override public V replace(V val) throws IgniteCheckedException {
         return set(val, ctx.hasPeekArray());
     }
 
@@ -439,7 +440,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replace(V oldVal, V newVal) throws GridException {
+    @Override public boolean replace(V oldVal, V newVal) throws IgniteCheckedException {
         return setx(newVal, ctx.equalsPeekArray(newVal));
     }
 
@@ -485,7 +486,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V setIfAbsent(V val) throws GridException {
+    @Nullable @Override public V setIfAbsent(V val) throws IgniteCheckedException {
         return set(val, ctx.noPeekArray());
     }
 
@@ -495,7 +496,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public boolean setxIfAbsent(V val) throws GridException {
+    @Override public boolean setxIfAbsent(V val) throws IgniteCheckedException {
         return setx(val, ctx.noPeekArray());
     }
 
@@ -505,7 +506,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public void transform(IgniteClosure<V, V> transformer) throws GridException {
+    @Override public void transform(IgniteClosure<V, V> transformer) throws IgniteCheckedException {
         transformAsync(transformer).get();
     }
 
@@ -515,7 +516,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replacex(V val) throws GridException {
+    @Override public boolean replacex(V val) throws IgniteCheckedException {
         return setx(val, ctx.hasPeekArray());
     }
 
@@ -525,7 +526,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V remove(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Nullable @Override public V remove(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         return proxy.remove(key, isNearEnabled(ctx) ? null : cached, filter);
     }
 
@@ -535,7 +536,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public boolean removex(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Override public boolean removex(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         return proxy.removex(key, isNearEnabled(ctx) ? null : cached, filter);
     }
 
@@ -545,7 +546,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(V val) throws GridException {
+    @Override public boolean remove(V val) throws IgniteCheckedException {
         return proxy.remove(key, val);
     }
 
@@ -694,7 +695,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
 
     /** {@inheritDoc} */
     @Override public boolean lock(long timeout,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         return proxy.lock(key, timeout, filter);
     }
 
@@ -705,7 +706,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public void unlock(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Override public void unlock(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         proxy.unlock(key, filter);
     }
 
@@ -732,7 +733,7 @@ public class GridCacheEntryImpl<K, V> implements GridCacheEntry<K, V>, Externali
     }
 
     /** {@inheritDoc} */
-    @Override public int memorySize() throws GridException {
+    @Override public int memorySize() throws IgniteCheckedException {
         GridCacheEntryEx<K, V> cached = this.cached;
 
         if (cached == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfo.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfo.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfo.java
index 3fea7ce..e840de5 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfo.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfo.java
@@ -9,11 +9,11 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
-import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.tostring.*;
+import org.gridgain.grid.util.typedef.internal.*;
 
 import java.io.*;
 
@@ -199,18 +199,18 @@ public class GridCacheEntryInfo<K, V> implements Externalizable {
     /**
      * @param ctx Context.
      * @param ldr Loader.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void unmarshalValue(GridCacheContext<K, V> ctx, ClassLoader ldr) throws GridException {
+    public void unmarshalValue(GridCacheContext<K, V> ctx, ClassLoader ldr) throws IgniteCheckedException {
         if (val == null && valBytes != null)
             val = ctx.marshaller().unmarshal(valBytes, ldr);
     }
 
     /**
      * @param ctx Cache context.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    public void marshal(GridCacheSharedContext<K, V> ctx) throws GridException {
+    public void marshal(GridCacheSharedContext<K, V> ctx) throws IgniteCheckedException {
         boolean depEnabled = ctx.gridDeploy().enabled();
 
         boolean valIsByteArr = val != null && val instanceof byte[];
@@ -231,9 +231,9 @@ public class GridCacheEntryInfo<K, V> implements Externalizable {
      *
      * @param ctx Cache context.
      * @param clsLdr Class loader.
-     * @throws GridException If unmarshalling failed.
+     * @throws IgniteCheckedException If unmarshalling failed.
      */
-    public void unmarshal(GridCacheContext<K, V> ctx, ClassLoader clsLdr) throws GridException {
+    public void unmarshal(GridCacheContext<K, V> ctx, ClassLoader clsLdr) throws IgniteCheckedException {
         IgniteMarshaller mrsh = ctx.marshaller();
 
         if (key == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfoCollectSwapListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfoCollectSwapListener.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfoCollectSwapListener.java
index e3d1538..34f95a8 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfoCollectSwapListener.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntryInfoCollectSwapListener.java
@@ -67,7 +67,7 @@ public class GridCacheEntryInfoCollectSwapListener<K, V> implements GridCacheSwa
 
             swappedEntries.put(key, info);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to process unswapped entry", e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntrySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntrySet.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntrySet.java
index a677a2d..5d5af0c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntrySet.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEntrySet.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
@@ -75,8 +76,8 @@ public class GridCacheEntrySet<K, V> extends AbstractSet<GridCacheEntry<K, V>> {
             try {
                 e.removex();
             }
-            catch (GridException ex) {
-                throw new GridRuntimeException(ex);
+            catch (IgniteCheckedException ex) {
+                throw new IgniteException(ex);
             }
 
             return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
index a051a20..b15b958 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionEntry.java
@@ -123,14 +123,14 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
         try {
             return peek0(SMART, null, cached.context().tm().localTxx());
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             // Should never happen.
-            throw new GridRuntimeException("Unable to perform entry peek() operation.", e);
+            throw new IgniteException("Unable to perform entry peek() operation.", e);
         }
     }
 
     /** {@inheritDoc} */
-    @Override public V peek(@Nullable Collection<GridCachePeekMode> modes) throws GridException {
+    @Override public V peek(@Nullable Collection<GridCachePeekMode> modes) throws IgniteCheckedException {
         return peek0(modes, CU.<K, V>empty(), cached.context().tm().localTxx());
     }
 
@@ -139,12 +139,12 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
      * @param filter Optional entry filter.
      * @param tx Transaction to peek at (if mode is TX).
      * @return Peeked value.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings({"unchecked"})
     @Nullable private V peek0(@Nullable GridCachePeekMode mode,
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, @Nullable GridCacheTxEx<K, V> tx)
-        throws GridException {
+        throws IgniteCheckedException {
         assert tx == null || tx.local();
 
         if (mode == null)
@@ -172,10 +172,10 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
      * @param filter Optional entry filter.
      * @param tx Transaction to peek at (if modes contains TX value).
      * @return Peeked value.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @Nullable private V peek0(@Nullable Collection<GridCachePeekMode> modes,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, GridCacheTxEx<K, V> tx) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, GridCacheTxEx<K, V> tx) throws IgniteCheckedException {
         if (F.isEmpty(modes))
             return peek0(SMART, filter, tx);
 
@@ -199,7 +199,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V reload() throws GridException {
+    @Nullable @Override public V reload() throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -218,7 +218,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
 
             return ctx.evicts().evict(cached, null, false, null);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to evict entry from cache: " + cached, e);
 
             return false;
@@ -231,12 +231,12 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public boolean compact() throws GridException {
+    @Override public boolean compact() throws IgniteCheckedException {
         throw unsupported();
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V get() throws GridException {
+    @Nullable @Override public V get() throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -246,7 +246,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V set(V val, IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Nullable @Override public V set(V val, IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -256,7 +256,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public boolean setx(V val, IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Override public boolean setx(V val, IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -266,7 +266,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public void transform(IgniteClosure<V, V> transformer) throws GridException {
+    @Override public void transform(IgniteClosure<V, V> transformer) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -276,7 +276,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V replace(V val) throws GridException {
+    @Nullable @Override public V replace(V val) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -286,7 +286,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replace(V oldVal, V newVal) throws GridException {
+    @Override public boolean replace(V oldVal, V newVal) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -306,7 +306,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V setIfAbsent(V val) throws GridException {
+    @Nullable @Override public V setIfAbsent(V val) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -316,7 +316,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public boolean setxIfAbsent(V val) throws GridException {
+    @Override public boolean setxIfAbsent(V val) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -326,7 +326,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replacex(V val) throws GridException {
+    @Override public boolean replacex(V val) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -336,7 +336,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V remove(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Nullable @Override public V remove(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -346,7 +346,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public boolean removex(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Override public boolean removex(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -356,7 +356,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(V val) throws GridException {
+    @Override public boolean remove(V val) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -449,7 +449,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
 
     /** {@inheritDoc} */
     @Override public boolean lock(long timeout,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -460,7 +460,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public void unlock(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Override public void unlock(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         throw unsupported();
     }
 
@@ -485,7 +485,7 @@ public class GridCacheEvictionEntry<K, V> implements GridCacheEntry<K, V>, Exter
     }
 
     /** {@inheritDoc} */
-    @Override public int memorySize() throws GridException{
+    @Override public int memorySize() throws IgniteCheckedException{
         return cached.memorySize();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionManager.java
index cd921c2..e07211c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionManager.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
@@ -38,9 +39,9 @@ import java.util.concurrent.locks.*;
 import java.util.concurrent.locks.Lock;
 
 import static java.util.concurrent.TimeUnit.*;
+import static org.apache.ignite.events.IgniteEventType.*;
 import static org.gridgain.grid.cache.GridCacheMemoryMode.*;
 import static org.gridgain.grid.cache.GridCacheMode.*;
-import static org.apache.ignite.events.IgniteEventType.*;
 import static org.gridgain.grid.kernal.processors.cache.GridCacheUtils.*;
 import static org.gridgain.grid.kernal.processors.cache.distributed.dht.GridDhtPartitionState.*;
 import static org.jdk8.backport.ConcurrentLinkedDeque8.*;
@@ -116,7 +117,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
     private volatile boolean firstEvictWarn;
 
     /** {@inheritDoc} */
-    @Override public void start0() throws GridException {
+    @Override public void start0() throws IgniteCheckedException {
         GridCacheConfiguration cfg = cctx.config();
 
         plc = cctx.isNear() ? cfg.<K, V>getNearEvictionPolicy() : cfg.<K, V>getEvictionPolicy();
@@ -128,10 +129,10 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
         filter = cfg.getEvictionFilter();
 
         if (cfg.getEvictMaxOverflowRatio() < 0)
-            throw new GridException("Configuration parameter 'maxEvictOverflowRatio' cannot be negative.");
+            throw new IgniteCheckedException("Configuration parameter 'maxEvictOverflowRatio' cannot be negative.");
 
         if (cfg.getEvictSynchronizedKeyBufferSize() < 0)
-            throw new GridException("Configuration parameter 'evictSynchronizedKeyBufferSize' cannot be negative.");
+            throw new IgniteCheckedException("Configuration parameter 'evictSynchronizedKeyBufferSize' cannot be negative.");
 
         if (!cctx.isLocal()) {
             evictSync = cfg.isEvictSynchronized() && !cctx.isNear() && !cctx.isSwapOrOffheapEnabled();
@@ -147,7 +148,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
         }
 
         if (cctx.isDht() && !nearSync && evictSync && isNearEnabled(cctx))
-            throw new GridException("Illegal configuration (may lead to data inconsistency) " +
+            throw new IgniteCheckedException("Illegal configuration (may lead to data inconsistency) " +
                 "[evictSync=true, evictNearSync=false]");
 
         reportConfigurationProblems();
@@ -175,10 +176,10 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
 
         if (evictSyncAgr) {
             if (cfg.getEvictSynchronizedTimeout() <= 0)
-                throw new GridException("Configuration parameter 'evictSynchronousTimeout' should be positive.");
+                throw new IgniteCheckedException("Configuration parameter 'evictSynchronousTimeout' should be positive.");
 
             if (cfg.getEvictSynchronizedConcurrencyLevel() <= 0)
-                throw new GridException("Configuration parameter 'evictSynchronousConcurrencyLevel' " +
+                throw new IgniteCheckedException("Configuration parameter 'evictSynchronousConcurrencyLevel' " +
                     "should be positive.");
 
             maxActiveFuts = cfg.getEvictSynchronizedConcurrencyLevel();
@@ -237,7 +238,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
     }
 
     /** {@inheritDoc} */
-    @Override protected void onKernalStart0() throws GridException {
+    @Override protected void onKernalStart0() throws IgniteCheckedException {
         super.onKernalStart0();
 
         if (plcEnabled && evictSync && !cctx.isNear()) {
@@ -466,7 +467,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
                 log.debug("Failed to send eviction response since initiating node left grid " +
                     "[node=" + nodeId + ", localNode=" + cctx.nodeId() + ']');
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to send eviction response to node [node=" + nodeId +
                 ", localNode=" + cctx.nodeId() + ", res" + res + ']', e);
         }
@@ -616,7 +617,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
             // as well because versions may change outside the transaction.
             return evict0(cache, entry, obsoleteVer, null, false);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to evict entry on remote node [key=" + key + ", localNode=" + cctx.nodeId() + ']', e);
 
             return false;
@@ -630,10 +631,10 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
      * @param filter Filter.
      * @param explicit If eviction is initiated by user.
      * @return {@code true} if entry has been evicted.
-     * @throws GridException If failed to evict entry.
+     * @throws IgniteCheckedException If failed to evict entry.
      */
     private boolean evict0(GridCacheAdapter<K, V> cache, GridCacheEntryEx<K, V> entry, GridCacheVersion obsoleteVer,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, boolean explicit) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter, boolean explicit) throws IgniteCheckedException {
         assert cache != null;
         assert entry != null;
         assert obsoleteVer != null;
@@ -690,7 +691,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
             if (e.markObsoleteIfEmpty(null) || e.obsolete())
                 e.context().cache().removeEntry(e);
         }
-        catch (GridException ex) {
+        catch (IgniteCheckedException ex) {
             U.error(log, "Failed to evict entry from cache: " + e, ex);
         }
 
@@ -698,7 +699,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
             try {
                 evict0(cctx.cache(), e, cctx.versions().next(), null, false);
             }
-            catch (GridException ex) {
+            catch (IgniteCheckedException ex) {
                 U.error(log, "Failed to evict entry from on heap memory: " + e, ex);
             }
         }
@@ -722,7 +723,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
             if (e.markObsoleteIfEmpty(null) || e.obsolete())
                 e.context().cache().removeEntry(e);
         }
-        catch (GridException ex) {
+        catch (IgniteCheckedException ex) {
             U.error(log, "Failed to evict entry from cache: " + e, ex);
         }
 
@@ -730,7 +731,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
             try {
                 evict0(cctx.cache(), e, cctx.versions().next(), null, false);
             }
-            catch (GridException ex) {
+            catch (IgniteCheckedException ex) {
                 U.error(log, "Failed to evict entry from on heap memory: " + e, ex);
             }
 
@@ -826,10 +827,10 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
      * @param explicit {@code True} if evict is called explicitly, {@code false} if it's called
      *      from eviction policy.
      * @return {@code True} if entry was marked for eviction.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     public boolean evict(@Nullable GridCacheEntryEx<K, V> entry, @Nullable GridCacheVersion obsoleteVer,
-        boolean explicit, @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+        boolean explicit, @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         if (entry == null)
             return true;
 
@@ -879,9 +880,9 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
     /**
      * @param keys Keys to evict.
      * @param obsoleteVer Obsolete version.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    public void batchEvict(Collection<? extends K> keys, @Nullable GridCacheVersion obsoleteVer) throws GridException {
+    public void batchEvict(Collection<? extends K> keys, @Nullable GridCacheVersion obsoleteVer) throws IgniteCheckedException {
         assert !evictSyncAgr;
         assert cctx.isSwapOrOffheapEnabled();
 
@@ -1107,7 +1108,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
 
                 return;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 U.error(log, "Eviction future finished with error (all entries will be touched): " + fut, e);
 
                 if (plcEnabled) {
@@ -1155,7 +1156,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
                     // They will be touched within updating transactions.
                     evict0(cctx.cache(), entry, obsoleteVer, versionFilter(info), false);
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(log, "Failed to evict entry [entry=" + entry +
                         ", localNode=" + cctx.nodeId() + ']', e);
                 }
@@ -1678,7 +1679,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
                                 plcEnabled)
                                 touch0(info.entry());
                         }
-                        catch (GridException e) {
+                        catch (IgniteCheckedException e) {
                             U.error(log, "Failed to evict entry: " + info.entry(), e);
                         }
                     }
@@ -1711,7 +1712,7 @@ public class GridCacheEvictionManager<K, V> extends GridCacheManagerAdapter<K, V
                     // Node left the topology.
                     onNodeLeft(nodeId);
                 }
-                catch (GridException ex) {
+                catch (IgniteCheckedException ex) {
                     U.error(log, "Failed to send eviction request to node [node=" + nodeId + ", req=" + req + ']', ex);
 
                     rejectEntries(nodeId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionRequest.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionRequest.java
index 825390c..fd36e15 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionRequest.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionRequest.java
@@ -9,13 +9,13 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.util.direct.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.lang.*;
 import org.gridgain.grid.util.tostring.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
 
 import java.io.*;
 import java.nio.*;
@@ -71,7 +71,7 @@ public class GridCacheEvictionRequest<K, V> extends GridCacheMessage<K, V> imple
 
     /** {@inheritDoc}
      * @param ctx*/
-    @Override public void prepareMarshal(GridCacheSharedContext<K, V> ctx) throws GridException {
+    @Override public void prepareMarshal(GridCacheSharedContext<K, V> ctx) throws IgniteCheckedException {
         super.prepareMarshal(ctx);
 
         if (entries != null) {
@@ -83,7 +83,7 @@ public class GridCacheEvictionRequest<K, V> extends GridCacheMessage<K, V> imple
     }
 
     /** {@inheritDoc} */
-    @Override public void finishUnmarshal(GridCacheSharedContext<K, V> ctx, ClassLoader ldr) throws GridException {
+    @Override public void finishUnmarshal(GridCacheSharedContext<K, V> ctx, ClassLoader ldr) throws IgniteCheckedException {
         super.finishUnmarshal(ctx, ldr);
 
         if (entriesBytes != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionResponse.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionResponse.java
index 5a420b8..b241043 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionResponse.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheEvictionResponse.java
@@ -9,11 +9,11 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.util.direct.*;
-import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.tostring.*;
+import org.gridgain.grid.util.typedef.internal.*;
 
 import java.io.*;
 import java.nio.*;
@@ -70,14 +70,14 @@ public class GridCacheEvictionResponse<K, V> extends GridCacheMessage<K, V> {
 
     /** {@inheritDoc}
      * @param ctx*/
-    @Override public void prepareMarshal(GridCacheSharedContext<K, V> ctx) throws GridException {
+    @Override public void prepareMarshal(GridCacheSharedContext<K, V> ctx) throws IgniteCheckedException {
         super.prepareMarshal(ctx);
 
         rejectedKeyBytes = marshalCollection(rejectedKeys, ctx);
     }
 
     /** {@inheritDoc} */
-    @Override public void finishUnmarshal(GridCacheSharedContext<K, V> ctx, ClassLoader ldr) throws GridException {
+    @Override public void finishUnmarshal(GridCacheSharedContext<K, V> ctx, ClassLoader ldr) throws IgniteCheckedException {
         super.finishUnmarshal(ctx, ldr);
 
         rejectedKeys = unmarshalCollection(rejectedKeyBytes, ctx, ldr);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java
index 1249641..6862a5e 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheFilterEvaluationEntry.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
@@ -104,7 +105,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V get() throws GridException {
+    @Nullable @Override public V get() throws IgniteCheckedException {
         throw new UnsupportedOperationException("get");
     }
 
@@ -119,7 +120,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V reload() throws GridException {
+    @Nullable @Override public V reload() throws IgniteCheckedException {
         throw new UnsupportedOperationException("reload");
     }
 
@@ -146,7 +147,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
         catch (GridCacheEntryRemovedException e) {
             assert false : "Entry should not become obsolete while holding lock";
 
-            throw new GridRuntimeException(e);
+            throw new IgniteException(e);
         }
     }
 
@@ -158,7 +159,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
         catch (GridCacheEntryRemovedException e) {
             assert false : "Entry should not become obsolete while holding lock";
 
-            throw new GridRuntimeException(e);
+            throw new IgniteException(e);
         }
     }
 
@@ -170,7 +171,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
         catch (GridCacheEntryRemovedException e) {
             assert false : "Entry should not become obsolete while holding lock";
 
-            throw new GridRuntimeException(e);
+            throw new IgniteException(e);
         }
     }
 
@@ -196,7 +197,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
 
     /** {@inheritDoc} */
     @Nullable @Override public V set(V val, @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter)
-        throws GridException {
+        throws IgniteCheckedException {
         throw new UnsupportedOperationException("set");
     }
 
@@ -206,7 +207,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V setIfAbsent(V val) throws GridException {
+    @Nullable @Override public V setIfAbsent(V val) throws IgniteCheckedException {
         throw new UnsupportedOperationException("setIfAbsent");
     }
 
@@ -217,7 +218,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
 
     /** {@inheritDoc} */
     @Override public boolean setx(V val, @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter)
-        throws GridException {
+        throws IgniteCheckedException {
         throw new UnsupportedOperationException("setx");
     }
 
@@ -228,7 +229,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean setxIfAbsent(@Nullable V val) throws GridException {
+    @Override public boolean setxIfAbsent(@Nullable V val) throws IgniteCheckedException {
         throw new UnsupportedOperationException("setxIfAbsent");
     }
 
@@ -238,7 +239,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public void transform(IgniteClosure<V, V> transformer) throws GridException {
+    @Override public void transform(IgniteClosure<V, V> transformer) throws IgniteCheckedException {
         throw new UnsupportedOperationException("transform");
     }
 
@@ -248,7 +249,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public V replace(V val) throws GridException {
+    @Nullable @Override public V replace(V val) throws IgniteCheckedException {
         throw new UnsupportedOperationException("replace");
     }
 
@@ -258,7 +259,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replacex(V val) throws GridException {
+    @Override public boolean replacex(V val) throws IgniteCheckedException {
         throw new UnsupportedOperationException("replacex");
     }
 
@@ -268,7 +269,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replace(V oldVal, V newVal) throws GridException {
+    @Override public boolean replace(V oldVal, V newVal) throws IgniteCheckedException {
         throw new UnsupportedOperationException("replace");
     }
 
@@ -279,7 +280,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
 
     /** {@inheritDoc} */
     @Nullable @Override public V remove(@Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter)
-        throws GridException {
+        throws IgniteCheckedException {
         throw new UnsupportedOperationException("remove");
     }
 
@@ -289,7 +290,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean removex(@Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws GridException {
+    @Override public boolean removex(@Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
         throw new UnsupportedOperationException("removex");
     }
 
@@ -299,7 +300,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(V val) throws GridException {
+    @Override public boolean remove(V val) throws IgniteCheckedException {
         throw new UnsupportedOperationException("remove");
     }
 
@@ -320,13 +321,13 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
 
     /** {@inheritDoc} */
     @Override public boolean compact()
-        throws GridException {
+        throws IgniteCheckedException {
         throw new UnsupportedOperationException("compact");
     }
 
     /** {@inheritDoc} */
     @Override public boolean lock(long timeout, @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter)
-        throws GridException {
+        throws IgniteCheckedException {
         throw new UnsupportedOperationException("lock");
     }
 
@@ -337,7 +338,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public void unlock(IgnitePredicate<GridCacheEntry<K, V>>... filter) throws GridException {
+    @Override public void unlock(IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
         throw new UnsupportedOperationException("unlock");
     }
 
@@ -347,7 +348,7 @@ public class GridCacheFilterEvaluationEntry<K, V> implements GridCacheEntry<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public int memorySize() throws GridException {
+    @Override public int memorySize() throws IgniteCheckedException {
         return impl.memorySize();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheGateway.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheGateway.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheGateway.java
index 18f4829..08e97e8 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheGateway.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheGateway.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
-import org.gridgain.grid.*;
-import org.gridgain.grid.util.typedef.internal.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.util.tostring.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 /**
@@ -89,8 +89,8 @@ public class GridCacheGateway<K, V> {
 
             preldr.startFuture().get();
         }
-        catch (GridException e) {
-            throw new GridRuntimeException("Failed to wait for cache preloader start [cacheName=" +
+        catch (IgniteCheckedException e) {
+            throw new IgniteException("Failed to wait for cache preloader start [cacheName=" +
                 ctx.name() + "]", e);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIndexUpdateException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIndexUpdateException.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIndexUpdateException.java
index 9e3df0d..628b436 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIndexUpdateException.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIndexUpdateException.java
@@ -9,14 +9,14 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.jetbrains.annotations.*;
 
 /**
  * Exception indicating that index update failed during cache entry update. If this exception is thrown, entry
  * is kept in it's original state (no updates performed).
  */
-public class GridCacheIndexUpdateException extends GridException {
+public class GridCacheIndexUpdateException extends IgniteCheckedException {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIoManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIoManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIoManager.java
index daaacb8..a222c32 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIoManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIoManager.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.deployment.*;
 import org.gridgain.grid.util.*;
@@ -128,7 +128,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
     };
 
     /** {@inheritDoc} */
-    @Override public void start0() throws GridException {
+    @Override public void start0() throws IgniteCheckedException {
         retryDelay = cctx.gridConfig().getNetworkSendRetryDelay();
         retryCnt = cctx.gridConfig().getNetworkSendRetryCount();
 
@@ -234,7 +234,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
 
                                 processMessage(nodeId, cacheMsg, c);
                             }
-                            catch (GridException e) {
+                            catch (IgniteCheckedException e) {
                                 // Log once.
                                 if (startErr.compareAndSet(false, true))
                                     U.error(log, "Failed to complete preload start future (will ignore message) " +
@@ -312,9 +312,9 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
      *
      * @param msg Message to send.
      * @param destNodeId Destination node ID.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private void onSend(GridCacheMessage<K, V> msg, @Nullable UUID destNodeId) throws GridException {
+    private void onSend(GridCacheMessage<K, V> msg, @Nullable UUID destNodeId) throws IgniteCheckedException {
         if (msg.messageId() < 0)
             // Generate and set message ID.
             msg.messageId(idGen.incrementAndGet());
@@ -332,10 +332,10 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
      *
      * @param node Node to send the message to.
      * @param msg Message to send.
-     * @throws GridException If sending failed.
+     * @throws IgniteCheckedException If sending failed.
      * @throws org.apache.ignite.cluster.ClusterTopologyException If receiver left.
      */
-    public void send(ClusterNode node, GridCacheMessage<K, V> msg) throws GridException {
+    public void send(ClusterNode node, GridCacheMessage<K, V> msg) throws IgniteCheckedException {
         send(node, msg, SYSTEM_POOL);
     }
 
@@ -344,10 +344,10 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
      *
      * @param node Node to send the message to.
      * @param msg Message to send.
-     * @throws GridException If sending failed.
+     * @throws IgniteCheckedException If sending failed.
      * @throws org.apache.ignite.cluster.ClusterTopologyException If receiver left.
      */
-    public void send(ClusterNode node, GridCacheMessage<K, V> msg, GridIoPolicy plc) throws GridException {
+    public void send(ClusterNode node, GridCacheMessage<K, V> msg, GridIoPolicy plc) throws IgniteCheckedException {
         assert !node.isLocal();
 
         onSend(msg, node.id());
@@ -376,7 +376,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
 
                 return;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (!cctx.discovery().alive(node.id()) || !cctx.discovery().pingNode(node.id()))
                     throw new ClusterTopologyException("Node left grid while sending message to: " + node.id(), e);
 
@@ -401,11 +401,11 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
      * @param fallback Callback for failed nodes.
      * @return {@code True} if nodes are empty or message was sent, {@code false} if
      *      all nodes have left topology while sending this message.
-     * @throws GridException If send failed.
+     * @throws IgniteCheckedException If send failed.
      */
     @SuppressWarnings( {"BusyWait"})
     public boolean safeSend(Collection<? extends ClusterNode> nodes, GridCacheMessage<K, V> msg,
-        @Nullable IgnitePredicate<ClusterNode> fallback) throws GridException {
+        @Nullable IgnitePredicate<ClusterNode> fallback) throws IgniteCheckedException {
         assert nodes != null;
         assert msg != null;
 
@@ -474,7 +474,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
 
                 break;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 boolean added = false;
 
                 for (ClusterNode n : nodes) {
@@ -524,9 +524,9 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
      *
      * @param nodeId ID of node to send the message to.
      * @param msg Message to send.
-     * @throws GridException If sending failed.
+     * @throws IgniteCheckedException If sending failed.
      */
-    public void send(UUID nodeId, GridCacheMessage<K, V> msg) throws GridException {
+    public void send(UUID nodeId, GridCacheMessage<K, V> msg) throws IgniteCheckedException {
         ClusterNode n = cctx.discovery().node(nodeId);
 
         if (n == null)
@@ -542,10 +542,10 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
      * @param msgId Ordered message ID.
      * @param msg Message to send.
      * @param timeout Timeout to keep a message on receiving queue.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
     public void sendOrderedMessage(ClusterNode node, Object topic, long msgId, GridCacheMessage<K, V> msg,
-        long timeout) throws GridException {
+        long timeout) throws IgniteCheckedException {
         onSend(msg, node.id());
 
         int cnt = 0;
@@ -562,7 +562,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
 
                 return;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (cctx.discovery().node(node.id()) == null)
                     throw new ClusterTopologyException("Node left grid while sending ordered message to: " + node.id(), e);
 
@@ -616,7 +616,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
             }
 
             if (cacheClsHandlers[msgIdx] != null)
-                throw new GridRuntimeException("Duplicate cache message ID found [cacheId=" + cacheId +
+                throw new IgniteException("Duplicate cache message ID found [cacheId=" + cacheId +
                     ", type=" + type + ']');
 
             cacheClsHandlers[msgIdx] = c;
@@ -657,7 +657,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
 
             return msgIdx;
         }
-        catch (GridException ignored) {
+        catch (IgniteCheckedException ignored) {
             return -1;
         }
     }
@@ -734,10 +734,10 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
     /**
      * @param nodeId Sender node ID.
      * @param cacheMsg Message.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("ErrorNotRethrown")
-    private void unmarshall(UUID nodeId, GridCacheMessage<K, V> cacheMsg) throws GridException {
+    private void unmarshall(UUID nodeId, GridCacheMessage<K, V> cacheMsg) throws IgniteCheckedException {
         if (cctx.localNodeId().equals(nodeId))
             return;
 
@@ -757,7 +757,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
         try {
             cacheMsg.finishUnmarshal(cctx, cctx.deploy().globalLoader());
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             if (cacheMsg.ignoreClassErrors() && X.hasCause(e, InvalidClassException.class,
                     ClassNotFoundException.class, NoClassDefFoundError.class, UnsupportedClassVersionError.class))
                 cacheMsg.onClassError(e);
@@ -767,7 +767,7 @@ public class GridCacheIoManager<K, V> extends GridCacheSharedManagerAdapter<K, V
         catch (Error e) {
             if (cacheMsg.ignoreClassErrors() && X.hasCause(e, NoClassDefFoundError.class,
                 UnsupportedClassVersionError.class))
-                    cacheMsg.onClassError(new GridException("Failed to load class during unmarshalling: " + e, e));
+                    cacheMsg.onClassError(new IgniteCheckedException("Failed to load class during unmarshalling: " + e, e));
             else
                 throw e;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIterator.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIterator.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIterator.java
index 1d7878d..30c7f0e 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIterator.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheIterator.java
@@ -9,12 +9,12 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.util.*;
-import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.lang.*;
+import org.gridgain.grid.util.typedef.*;
 
 import java.util.*;
 
@@ -71,7 +71,7 @@ public class GridCacheIterator<K, V, T> implements GridSerializableIterator<T> {
             // Back remove operation by actual cache.
             cur.removex();
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             throw new GridClosureException(e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheKeySet.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheKeySet.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheKeySet.java
index 9fe42db..8f6092d 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheKeySet.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheKeySet.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
@@ -80,8 +81,8 @@ public class GridCacheKeySet<K, V> extends GridSerializableSet<K> {
         try {
             e.removex();
         }
-        catch (GridException ex) {
-            throw new GridRuntimeException(ex);
+        catch (IgniteCheckedException ex) {
+            throw new IgniteException(ex);
         }
 
         return true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMBeanAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMBeanAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMBeanAdapter.java
index 5f0e592..676ebae 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMBeanAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheMBeanAdapter.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.cache.*;
 
 /**
@@ -57,7 +57,7 @@ class GridCacheMBeanAdapter implements GridCacheMBean {
         try {
             return cctx.cache().overflowSize();
         }
-        catch (GridException ignored) {
+        catch (IgniteCheckedException ignored) {
             return -1;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManager.java
index bcc81ab..41cda0c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManager.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 /**
  * Interface for cache managers.
@@ -19,9 +19,9 @@ public interface GridCacheManager<K, V> {
      * Starts manager.
      *
      * @param cctx Context.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void start(GridCacheContext<K, V> cctx) throws GridException;
+    public void start(GridCacheContext<K, V> cctx) throws IgniteCheckedException;
 
     /**
      * Stops manager.
@@ -31,9 +31,9 @@ public interface GridCacheManager<K, V> {
     public void stop(boolean cancel);
 
     /**
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void onKernalStart() throws GridException;
+    public void onKernalStart() throws IgniteCheckedException;
 
     /**
      * @param cancel Cancel flag.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManagerAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManagerAdapter.java
index e3b2882..ed05b98 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManagerAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheManagerAdapter.java
@@ -29,7 +29,7 @@ public class GridCacheManagerAdapter<K, V> implements GridCacheManager<K, V> {
     private final AtomicBoolean starting = new AtomicBoolean(false);
 
     /** {@inheritDoc} */
-    @Override public final void start(GridCacheContext<K, V> cctx) throws GridException {
+    @Override public final void start(GridCacheContext<K, V> cctx) throws IgniteCheckedException {
         if (!starting.compareAndSet(false, true))
             assert false : "Method start is called more than once for manager: " + this;
 
@@ -60,9 +60,9 @@ public class GridCacheManagerAdapter<K, V> implements GridCacheManager<K, V> {
     }
 
     /**
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    protected void start0() throws GridException {
+    protected void start0() throws IgniteCheckedException {
         // No-op.
     }
 
@@ -86,7 +86,7 @@ public class GridCacheManagerAdapter<K, V> implements GridCacheManager<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public final void onKernalStart() throws GridException {
+    @Override public final void onKernalStart() throws IgniteCheckedException {
         onKernalStart0();
 
         if (log != null && log.isDebugEnabled())
@@ -106,9 +106,9 @@ public class GridCacheManagerAdapter<K, V> implements GridCacheManager<K, V> {
     }
 
     /**
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    protected void onKernalStart0() throws GridException {
+    protected void onKernalStart0() throws IgniteCheckedException {
         // No-op.
     }