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

[28/56] [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/local/GridLocalCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalCache.java
index cbbf36c..8edcf9c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalCache.java
@@ -9,12 +9,12 @@
 
 package org.gridgain.grid.kernal.processors.cache.local;
 
+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.*;
-import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.future.*;
+import org.gridgain.grid.util.typedef.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -143,14 +143,14 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
             }
 
             if (!ctx.mvcc().addFuture(fut))
-                fut.onError(new GridException("Duplicate future ID (internal error): " + fut));
+                fut.onError(new IgniteCheckedException("Duplicate future ID (internal error): " + fut));
 
             // Must have future added prior to checking locks.
             fut.checkLocks();
 
             return fut;
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             fut.onError(e);
 
             return fut;
@@ -159,7 +159,7 @@ public class GridLocalCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public void unlockAll(Collection<? extends K> keys,
-        IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+        IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         long topVer = ctx.affinity().affinityTopologyVersion();
 
         for (K key : keys) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalLockFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalLockFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalLockFuture.java
index e0f4cc8..09bb220 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalLockFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalLockFuture.java
@@ -287,7 +287,7 @@ public final class GridLocalLockFuture<K, V> extends GridFutureAdapter<Boolean>
 
             return true;
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             onError(e);
 
             return false;
@@ -398,9 +398,9 @@ public final class GridLocalLockFuture<K, V> extends GridFutureAdapter<Boolean>
     /**
      * Checks for errors.
      *
-     * @throws GridException If execution failed.
+     * @throws IgniteCheckedException If execution failed.
      */
-    private void checkError() throws GridException {
+    private void checkError() throws IgniteCheckedException {
         if (err.get() != null)
             throw U.cast(err.get());
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java
index caee6e7..e769893 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTx.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.cache.local;
 
+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.*;
 import org.gridgain.grid.util.future.*;
@@ -77,7 +77,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public void prepare() throws GridException {
+    @Override public void prepare() throws IgniteCheckedException {
         if (!state(PREPARING)) {
             GridCacheTxState state = state();
 
@@ -87,7 +87,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
 
             setRollbackOnly();
 
-            throw new GridException("Invalid transaction state for prepare [state=" + state + ", tx=" + this + ']');
+            throw new IgniteCheckedException("Invalid transaction state for prepare [state=" + state + ", tx=" + this + ']');
         }
 
         try {
@@ -95,7 +95,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
 
             state(PREPARED);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             setRollbackOnly();
 
             throw e;
@@ -109,7 +109,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
 
             return new GridFinishedFuture<GridCacheTxEx<K, V>>(cctx.kernalContext(), this);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(cctx.kernalContext(), e);
         }
     }
@@ -117,9 +117,9 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
     /**
      * Commits without prepare.
      *
-     * @throws GridException If commit failed.
+     * @throws IgniteCheckedException If commit failed.
      */
-    void commit0() throws GridException {
+    void commit0() throws IgniteCheckedException {
         if (state(COMMITTING)) {
             try {
                 userCommit();
@@ -146,7 +146,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
         try {
             prepare();
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             state(UNKNOWN);
 
             return new GridFinishedFuture<>(cctx.kernalContext(), e);
@@ -168,7 +168,7 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public void rollback() throws GridException {
+    @Override public void rollback() throws IgniteCheckedException {
         rollbackAsync().get();
     }
 
@@ -182,13 +182,13 @@ class GridLocalTx<K, V> extends GridCacheTxLocalAdapter<K, V> {
 
             return new GridFinishedFuture<GridCacheTx>(cctx.kernalContext(), this);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(cctx.kernalContext(), e);
         }
     }
 
     /** {@inheritDoc} */
-    @Override public boolean finish(boolean commit) throws GridException {
+    @Override public boolean finish(boolean commit) throws IgniteCheckedException {
         assert false;
 
         return false;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTxFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTxFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTxFuture.java
index c9d5a37..a40ddbb 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTxFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/GridLocalTxFuture.java
@@ -164,7 +164,7 @@ final class GridLocalTxFuture<K, V> extends GridFutureAdapter<GridCacheTxEx<K, V
                 try {
                     tx.rollback();
                 }
-                catch (GridException ex) {
+                catch (IgniteCheckedException ex) {
                     U.error(log, "Failed to rollback the transaction: " + tx, ex);
                 }
             }
@@ -193,7 +193,7 @@ final class GridLocalTxFuture<K, V> extends GridFutureAdapter<GridCacheTxEx<K, V
                     // Another thread or transaction owns some lock.
                     if (!entry.lockedByThread(tx.threadId())) {
                         if (tx.pessimistic())
-                            onError(new GridException("Pessimistic transaction does not own lock for commit: " + tx));
+                            onError(new IgniteCheckedException("Pessimistic transaction does not own lock for commit: " + tx));
 
                         if (log.isDebugEnabled())
                             log.debug("Transaction entry is not locked by transaction (will wait) [entry=" + entry +
@@ -279,7 +279,7 @@ final class GridLocalTxFuture<K, V> extends GridFutureAdapter<GridCacheTxEx<K, V
             catch (GridCacheTxTimeoutException e) {
                 onError(e);
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (tx.state() == UNKNOWN) {
                     onError(new GridCacheTxHeuristicException("Commit only partially succeeded " +
                         "(entries will be invalidated on remote nodes once transaction timeout passes): " +
@@ -304,7 +304,7 @@ final class GridLocalTxFuture<K, V> extends GridFutureAdapter<GridCacheTxEx<K, V
             try {
                 tx.rollback();
             }
-            catch (GridException ex) {
+            catch (IgniteCheckedException ex) {
                 U.error(log, "Failed to rollback the transaction: " + tx, ex);
             }
 
@@ -328,9 +328,9 @@ final class GridLocalTxFuture<K, V> extends GridFutureAdapter<GridCacheTxEx<K, V
     /**
      * Checks for errors.
      *
-     * @throws GridException If execution failed.
+     * @throws IgniteCheckedException If execution failed.
      */
-    private void checkError() throws GridException {
+    private void checkError() throws IgniteCheckedException {
         if (err.get() != null)
             throw U.cast(err.get());
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
index 46cc579..e4d6b00 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/local/atomic/GridLocalAtomicCache.java
@@ -9,12 +9,12 @@
 
 package org.gridgain.grid.kernal.processors.cache.local.atomic;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
+import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.local.*;
-import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.typedef.*;
@@ -64,7 +64,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
             @Override public GridCacheMapEntry<K, V> create(GridCacheContext<K, V> ctx, long topVer, K key, int hash,
                 V val, @Nullable GridCacheMapEntry<K, V> next, long ttl, int hdrId) {
                 return new GridLocalCacheEntry<K, V>(ctx, key, hash, val, next, ttl, hdrId) {
-                    @Override public GridCacheEntry<K, V> wrapFilterLocked() throws GridException {
+                    @Override public GridCacheEntry<K, V> wrapFilterLocked() throws IgniteCheckedException {
                         assert Thread.holdsLock(this);
 
                         return new GridCacheFilterEvaluationEntry<>(key, rawGetOrUnmarshalUnlocked(true), this);
@@ -75,7 +75,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public void start() throws GridException {
+    @Override public void start() throws IgniteCheckedException {
         // No-op.
     }
 
@@ -95,7 +95,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         V val,
         @Nullable GridCacheEntryEx<K, V> cached,
         long ttl,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         A.notNull(key, "key", val, "val");
 
         ctx.denyOnLocalRead();
@@ -116,7 +116,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         V val,
         @Nullable GridCacheEntryEx<K, V> cached,
         long ttl,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
         A.notNull(key, "key", val, "val");
 
         ctx.denyOnLocalRead();
@@ -134,7 +134,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @Override public boolean putx(K key,
         V val,
-        IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+        IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         A.notNull(key, "key", val, "val");
 
         ctx.denyOnLocalRead();
@@ -179,7 +179,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public V putIfAbsent(K key, V val) throws GridException {
+    @Override public V putIfAbsent(K key, V val) throws IgniteCheckedException {
         return put(key, val, ctx.noPeekArray());
     }
 
@@ -189,7 +189,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean putxIfAbsent(K key, V val) throws GridException {
+    @Override public boolean putxIfAbsent(K key, V val) throws IgniteCheckedException {
         return putx(key, val, ctx.noPeekArray());
     }
 
@@ -200,7 +200,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public V replace(K key, V val) throws GridException {
+    @Override public V replace(K key, V val) throws IgniteCheckedException {
         return put(key, val, ctx.hasPeekArray());
     }
 
@@ -210,7 +210,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replacex(K key, V val) throws GridException {
+    @Override public boolean replacex(K key, V val) throws IgniteCheckedException {
         return putx(key, val, ctx.hasPeekArray());
     }
 
@@ -220,7 +220,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean replace(K key, V oldVal, V newVal) throws GridException {
+    @Override public boolean replace(K key, V oldVal, V newVal) throws IgniteCheckedException {
         return putx(key, newVal, ctx.equalsPeekArray(oldVal));
     }
 
@@ -231,7 +231,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 GridException {
+    @Override public GridCacheReturn<V> replacex(K key, V oldVal, V newVal) throws IgniteCheckedException {
         A.notNull(key, "key");
 
         ctx.denyOnLocalRead();
@@ -248,7 +248,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public GridCacheReturn<V> removex(K key, V val) throws GridException {
+    @Override public GridCacheReturn<V> removex(K key, V val) throws IgniteCheckedException {
         A.notNull(key, "key");
 
         ctx.denyOnLocalRead();
@@ -286,7 +286,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @Override public void putAll(Map<? extends K, ? extends V> m,
-        IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+        IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         ctx.denyOnLocalRead();
 
         updateAllInternal(UPDATE,
@@ -308,7 +308,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public void transform(K key, IgniteClosure<V, V> transformer) throws GridException {
+    @Override public void transform(K key, IgniteClosure<V, V> transformer) throws IgniteCheckedException {
         ctx.denyOnLocalRead();
 
         updateAllInternal(TRANSFORM,
@@ -324,7 +324,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public <R> R transformAndCompute(K key, IgniteClosure<V, IgniteBiTuple<V, R>> transformer)
-        throws GridException {
+        throws IgniteCheckedException {
         return (R)updateAllInternal(TRANSFORM,
             Collections.singleton(key),
             Collections.singleton(new GridCacheTransformComputeClosure<>(transformer)),
@@ -347,7 +347,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("ConstantConditions")
-    @Override public void transformAll(@Nullable Map<? extends K, ? extends IgniteClosure<V, V>> m) throws GridException {
+    @Override public void transformAll(@Nullable Map<? extends K, ? extends IgniteClosure<V, V>> m) throws IgniteCheckedException {
         ctx.denyOnLocalRead();
 
         if (F.isEmpty(m))
@@ -377,7 +377,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     @SuppressWarnings("unchecked")
     @Override public V remove(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
         ctx.denyOnLocalRead();
 
         return (V)updateAllInternal(DELETE,
@@ -403,7 +403,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void removeAll(Collection<? extends K> keys,
-        IgnitePredicate<GridCacheEntry<K, V>>... filter) throws GridException {
+        IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
         ctx.denyOnLocalRead();
 
         updateAllInternal(DELETE,
@@ -428,7 +428,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     @SuppressWarnings("unchecked")
     @Override public boolean removex(K key,
         @Nullable GridCacheEntryEx<K, V> entry,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
         A.notNull(key, "key");
 
         ctx.denyOnLocalRead();
@@ -456,7 +456,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean remove(K key, V val) throws GridException {
+    @Override public boolean remove(K key, V val) throws IgniteCheckedException {
         A.notNull(key, "key");
 
         ctx.denyOnLocalRead();
@@ -478,7 +478,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
-    @Override public void removeAll(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws GridException {
+    @Override public void removeAll(IgnitePredicate<GridCacheEntry<K, V>>[] filter) throws IgniteCheckedException {
         removeAll(keySet(filter));
     }
 
@@ -491,7 +491,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
     @SuppressWarnings("unchecked")
     @Override @Nullable public V get(K key, boolean deserializePortable,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>> filter) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>> filter) throws IgniteCheckedException {
         ctx.denyOnFlag(LOCAL);
 
         String taskName = ctx.kernalContext().job().currentTaskName();
@@ -511,7 +511,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     @SuppressWarnings("unchecked")
     @Override public final Map<K, V> getAll(Collection<? extends K> keys, boolean deserializePortable,
         IgnitePredicate<GridCacheEntry<K, V>> filter)
-        throws GridException {
+        throws IgniteCheckedException {
         ctx.denyOnFlag(LOCAL);
 
         String taskName = ctx.kernalContext().job().currentTaskName();
@@ -560,7 +560,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      * @param storeEnabled Store enabled flag.
      * @param clone {@code True} if returned values should be cloned.
      * @return Key-value map.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("ConstantConditions")
     private Map<K, V> getAllInternal(@Nullable Collection<? extends K> keys,
@@ -569,7 +569,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         boolean storeEnabled,
         boolean clone,
         String taskName,
-        boolean deserializePortable) throws GridException {
+        boolean deserializePortable) throws IgniteCheckedException {
         ctx.checkSecurity(GridSecurityPermission.CACHE_READ);
 
         UUID subjId = ctx.subjectIdPerCall(null);
@@ -736,7 +736,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
      * @param filter Cache entry filter.
      * @param storeEnabled Store enabled flag.
      * @return Update result.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
     private Object updateAllInternal(GridCacheOperation op,
@@ -746,7 +746,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         boolean retval,
         boolean rawRetval,
         IgnitePredicate<GridCacheEntry<K, V>>[] filter,
-        boolean storeEnabled) throws GridException {
+        boolean storeEnabled) throws IgniteCheckedException {
         if (keyCheck)
             validateCacheKeys(keys);
 
@@ -820,7 +820,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
                     entry = null;
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     if (err == null)
                         err = partialUpdateException();
 
@@ -864,7 +864,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
         @Nullable IgnitePredicate<GridCacheEntry<K, V>>[] filter,
         UUID subjId,
         String taskName
-    ) throws GridException {
+    ) throws IgniteCheckedException {
         List<GridCacheEntryEx<K, V>> locked = lockEntries(keys);
 
         try {
@@ -898,7 +898,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                             continue;
                         }
                     }
-                    catch (GridException e) {
+                    catch (IgniteCheckedException e) {
                         if (err == null)
                             err = partialUpdateException();
 
@@ -1046,7 +1046,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
 
                     filtered.add(entry);
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     if (err == null)
                         err = partialUpdateException();
 
@@ -1120,7 +1120,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
                 op = DELETE;
             }
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             if (err == null)
                 err = partialUpdateException();
 
@@ -1169,7 +1169,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
             catch (GridCacheEntryRemovedException ignore) {
                 assert false : "Entry cannot become obsolete while holding lock.";
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (err == null)
                     err = partialUpdateException();
 
@@ -1269,7 +1269,7 @@ public class GridLocalAtomicCache<K, V> extends GridCacheAdapter<K, V> {
     /** {@inheritDoc} */
     @SuppressWarnings("unchecked")
     @Override public void unlockAll(@Nullable Collection<? extends K> keys,
-        @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws GridException {
+        @Nullable IgnitePredicate<GridCacheEntry<K, V>>... filter) throws IgniteCheckedException {
         throw new UnsupportedOperationException("Locks are not supported for " +
             "GridCacheAtomicityMode.ATOMIC mode (use GridCacheAtomicityMode.TRANSACTIONAL instead)");
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
index c3644ae..b3d658b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.processors.cache.*;
@@ -71,7 +72,7 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws GridException {
+    @Override protected void cancelQuery() throws IgniteCheckedException {
         final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
 
         assert qryMgr != null;
@@ -114,7 +115,7 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
                     });
             }
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to send cancel request (will cancel query in any case).", e);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
index 9306eb3..b9d2474 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
@@ -26,8 +27,8 @@ import org.jetbrains.annotations.*;
 import java.util.*;
 import java.util.concurrent.*;
 
-import static org.gridgain.grid.cache.GridCacheMode.*;
 import static org.apache.ignite.events.IgniteEventType.*;
+import static org.gridgain.grid.cache.GridCacheMode.*;
 import static org.gridgain.grid.kernal.GridTopic.*;
 
 /**
@@ -68,7 +69,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
     };
 
     /** {@inheritDoc} */
-    @Override public void start0() throws GridException {
+    @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
         assert cctx.config().getCacheMode() != LOCAL;
@@ -148,7 +149,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
                     GridCacheQueryResponse res = new GridCacheQueryResponse(
                         cctx.cacheId(),
                         req.id(),
-                        new GridException("Received request for incorrect cache [expected=" + cctx.name() +
+                        new IgniteCheckedException("Received request for incorrect cache [expected=" + cctx.name() +
                             ", actual=" + req.cacheName()));
 
                     sendQueryResponse(sndId, res, 0);
@@ -251,7 +252,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
         int attempt = 1;
 
-        GridException err = null;
+        IgniteCheckedException err = null;
 
         while (!Thread.currentThread().isInterrupted()) {
             try {
@@ -276,7 +277,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
                 return false;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (err == null)
                     err = e;
 
@@ -372,7 +373,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
             try {
                 fut.cancel();
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 U.error(log, "Failed to cancel running query future: " + fut, e);
             }
 
@@ -388,7 +389,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
             try {
                 fut.get();
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (log.isDebugEnabled())
                     log.debug("Received query error while waiting for query to finish [queryFuture= " + fut +
                         ", error= " + e + ']');
@@ -485,7 +486,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             fut.execute();
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             fut.onDone(e);
         }
 
@@ -544,7 +545,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             sendRequest(fut, req, nodes);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             fut.onDone(e);
         }
 
@@ -576,7 +577,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             sendRequest(fut, req, nodes);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             fut.onDone(e);
         }
     }
@@ -595,7 +596,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             fut.execute();
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             fut.onDone(e);
         }
 
@@ -653,7 +654,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
 
             sendRequest(fut, req, nodes);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             fut.onDone(e);
         }
 
@@ -666,7 +667,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
      * @param fut Distributed future.
      * @param req Request.
      * @param nodes Nodes.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      * @deprecated Need to remove nodes filtration after breaking compatibility.
      */
     @Deprecated
@@ -675,7 +676,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
         final GridCacheDistributedQueryFuture<?, ?, ?> fut,
         final GridCacheQueryRequest<K, V> req,
         Collection<ClusterNode> nodes
-    ) throws GridException {
+    ) throws IgniteCheckedException {
         assert fut != null;
         assert req != null;
         assert nodes != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
index b53654b..df55b83 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.util.lang.*;
@@ -58,7 +58,7 @@ public class GridCacheLocalQueryFuture<K, V, R> extends GridCacheQueryFutureAdap
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws GridException {
+    @Override protected void cancelQuery() throws IgniteCheckedException {
         if (fut != null)
             fut.cancel();
     }
@@ -97,10 +97,10 @@ public class GridCacheLocalQueryFuture<K, V, R> extends GridCacheQueryFutureAdap
 
         /**
          * @return Query info.
-         * @throws GridException In case of error.
+         * @throws IgniteCheckedException In case of error.
          */
         @SuppressWarnings({"unchecked"})
-        private GridCacheQueryInfo localQueryInfo() throws GridException {
+        private GridCacheQueryInfo localQueryInfo() throws IgniteCheckedException {
             GridCacheQueryBean qry = query();
 
             IgnitePredicate<GridCacheEntry<Object, Object>> prjPred = qry.query().projectionFilter() == null ?

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryManager.java
index 9dd1d7f..fb385a4 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryManager.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.processors.query.*;
 import org.jetbrains.annotations.*;
@@ -65,7 +65,7 @@ public class GridCacheLocalQueryManager<K, V> extends GridCacheQueryManager<K, V
     }
 
     /** {@inheritDoc} */
-    @Override public void start0() throws GridException {
+    @Override public void start0() throws IgniteCheckedException {
         super.start0();
 
         assert cctx.config().getCacheMode() == LOCAL;
@@ -85,7 +85,7 @@ public class GridCacheLocalQueryManager<K, V> extends GridCacheQueryManager<K, V
 
             fut.execute();
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             fut.onDone(e);
         }
 
@@ -96,7 +96,7 @@ public class GridCacheLocalQueryManager<K, V> extends GridCacheQueryManager<K, V
     @Override public GridCacheQueryFuture<?> queryDistributed(GridCacheQueryBean qry, Collection<ClusterNode> nodes) {
         assert cctx.config().getCacheMode() == LOCAL;
 
-        throw new GridRuntimeException("Distributed queries are not available for local cache " +
+        throw new IgniteException("Distributed queries are not available for local cache " +
             "(use 'GridCacheQuery.execute(grid.forLocal())' instead) [cacheName=" + cctx.name() + ']');
     }
 
@@ -119,7 +119,7 @@ public class GridCacheLocalQueryManager<K, V> extends GridCacheQueryManager<K, V
 
             fut.execute();
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             fut.onDone(e);
         }
 
@@ -131,7 +131,7 @@ public class GridCacheLocalQueryManager<K, V> extends GridCacheQueryManager<K, V
         Collection<ClusterNode> nodes) {
         assert cctx.config().getCacheMode() == LOCAL;
 
-        throw new GridRuntimeException("Distributed queries are not available for local cache " +
+        throw new IgniteException("Distributed queries are not available for local cache " +
             "(use 'GridCacheQuery.execute(grid.forLocal())' instead) [cacheName=" + cctx.name() + ']');
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesEx.java
index 803719c..d1732fb 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesEx.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.cache.query.*;
 
 import java.util.*;
@@ -22,9 +22,9 @@ public interface GridCacheQueriesEx<K, V> extends GridCacheQueries<K, V> {
      * Gets SQL metadata.
      *
      * @return SQL metadata.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    public Collection<GridCacheSqlMetadata> sqlMetadata() throws GridException;
+    public Collection<GridCacheSqlMetadata> sqlMetadata() throws IgniteCheckedException;
 
     /**
      * Creates SQL fields query which will include results metadata if needed.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java
index ee02797..3ba1ceb 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesImpl.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.indexing.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.processors.cache.*;
@@ -192,7 +192,7 @@ public class GridCacheQueriesImpl<K, V> implements GridCacheQueriesEx<K, V>, Ext
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<GridCacheSqlMetadata> sqlMetadata() throws GridException {
+    @Override public Collection<GridCacheSqlMetadata> sqlMetadata() throws IgniteCheckedException {
         return ctx.queries().sqlMetadata();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java
index 03f1f03..9edcf6a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueriesProxy.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.processors.cache.*;
@@ -226,7 +226,7 @@ public class GridCacheQueriesProxy<K, V> implements GridCacheQueriesEx<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<GridCacheSqlMetadata> sqlMetadata() throws GridException {
+    @Override public Collection<GridCacheSqlMetadata> sqlMetadata() throws IgniteCheckedException {
         GridCacheProjectionImpl<K, V> prev = gate.enter(prj);
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java
index dbddd02..d15d77c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryAdapter.java
@@ -348,11 +348,11 @@ public class GridCacheQueryAdapter<T> implements GridCacheQuery<T> {
     }
 
     /**
-     * @throws GridException If query is invalid.
+     * @throws IgniteCheckedException If query is invalid.
      */
-    public void validate() throws GridException {
+    public void validate() throws IgniteCheckedException {
         if (type != SCAN && !cctx.config().isQueryIndexEnabled())
-            throw new GridException("Indexing is disabled for cache: " + cctx.cache().name());
+            throw new IgniteCheckedException("Indexing is disabled for cache: " + cctx.cache().name());
     }
 
     /**
@@ -419,7 +419,7 @@ public class GridCacheQueryAdapter<T> implements GridCacheQuery<T> {
                 cctx.deploy().registerClasses(filter, rmtReducer, rmtTransform);
                 cctx.deploy().registerClasses(args);
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 return new GridCacheQueryErrorFuture<>(cctx.kernalContext(), e);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java
index f4dc62e..96e05f7 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryErrorFuture.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.util.future.*;
@@ -33,12 +33,12 @@ public class GridCacheQueryErrorFuture<T> extends GridFinishedFuture<Collection<
     }
 
     /** {@inheritDoc} */
-    @Override public int available() throws GridException {
+    @Override public int available() throws IgniteCheckedException {
         return 0;
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public T next() throws GridException {
+    @Nullable @Override public T next() throws IgniteCheckedException {
         return null;
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java
index dbdeb31..797e31a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -156,8 +156,8 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
         catch (NoSuchElementException ignored) {
             return null;
         }
-        catch (GridException e) {
-            throw new GridRuntimeException(e);
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
     }
 
@@ -165,9 +165,9 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
      * Returns next page for the query.
      *
      * @return Next page or {@code null} if no more pages available.
-     * @throws GridException If fetch failed.
+     * @throws IgniteCheckedException If fetch failed.
      */
-    public Collection<R> nextPage() throws GridException {
+    public Collection<R> nextPage() throws IgniteCheckedException {
         return nextPage(qry.query().timeout(), startTime);
     }
 
@@ -176,9 +176,9 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
      *
      * @param timeout Timeout.
      * @return Next page or {@code null} if no more pages available.
-     * @throws GridException If fetch failed.
+     * @throws IgniteCheckedException If fetch failed.
      */
-    public Collection<R> nextPage(long timeout) throws GridException {
+    public Collection<R> nextPage(long timeout) throws IgniteCheckedException {
         return nextPage(timeout, U.currentTimeMillis());
     }
 
@@ -188,9 +188,9 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
      * @param timeout Timeout.
      * @param startTime Timeout wait start time.
      * @return Next page or {@code null} if no more pages available.
-     * @throws GridException If fetch failed.
+     * @throws IgniteCheckedException If fetch failed.
      */
-    private Collection<R> nextPage(long timeout, long startTime) throws GridException {
+    private Collection<R> nextPage(long timeout, long startTime) throws IgniteCheckedException {
         Collection<R> res = null;
 
         while (res == null) {
@@ -215,7 +215,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
                         catch (InterruptedException e) {
                             Thread.currentThread().interrupt();
 
-                            throw new GridException("Query was interrupted: " + qry, e);
+                            throw new IgniteCheckedException("Query was interrupted: " + qry, e);
                         }
                     }
                 }
@@ -230,21 +230,21 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /**
-     * @throws GridException If future is done with an error.
+     * @throws IgniteCheckedException If future is done with an error.
      */
-    private void checkError() throws GridException {
+    private void checkError() throws IgniteCheckedException {
         if (error() != null) {
             clear();
 
-            throw new GridException("Query execution failed: " + qry, error());
+            throw new IgniteCheckedException("Query execution failed: " + qry, error());
         }
     }
 
     /**
      * @return Iterator.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    private Iterator<R> internalIterator() throws GridException {
+    private Iterator<R> internalIterator() throws IgniteCheckedException {
         checkError();
 
         Iterator<R> it = null;
@@ -288,7 +288,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
                     catch (InterruptedException e) {
                         Thread.currentThread().interrupt();
 
-                        throw new GridException("Query was interrupted: " + qry, e);
+                        throw new IgniteCheckedException("Query was interrupted: " + qry, e);
                     }
                 }
             }
@@ -362,9 +362,9 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
                     onPage(nodeId, true);
 
                     onDone(nodeId != null ?
-                        new GridException("Failed to execute query on node [query=" + qry +
+                        new IgniteCheckedException("Failed to execute query on node [query=" + qry +
                             ", nodeId=" + nodeId + "]", err) :
-                        new GridException("Failed to execute query locally: " + qry, err));
+                        new IgniteCheckedException("Failed to execute query locally: " + qry, err));
 
                     mux.notifyAll();
                 }
@@ -455,7 +455,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<R> get() throws GridException {
+    @Override public Collection<R> get() throws IgniteCheckedException {
         if (!isDone())
             loadAllPages();
 
@@ -463,7 +463,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<R> get(long timeout, TimeUnit unit) throws GridException {
+    @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteCheckedException {
         if (!isDone())
             loadAllPages();
 
@@ -498,7 +498,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /** {@inheritDoc} */
-    @Override public boolean cancel() throws GridException {
+    @Override public boolean cancel() throws IgniteCheckedException {
         if (onCancelled()) {
             cancelQuery();
 
@@ -509,9 +509,9 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /**
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    protected abstract void cancelQuery() throws GridException;
+    protected abstract void cancelQuery() throws IgniteCheckedException;
 
     /** {@inheritDoc} */
     @Override public IgniteUuid timeoutId() {
@@ -530,7 +530,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
 
             onDone(new IgniteFutureTimeoutException("Query timed out."));
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             onDone(e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
index dbb2665..25c0668 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
@@ -77,7 +77,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     private final GridSpinBusyLock busyLock = new GridSpinBusyLock();
 
     /** {@inheritDoc} */
-    @Override public void start0() throws GridException {
+    @Override public void start0() throws IgniteCheckedException {
         idxProc = cctx.kernalContext().query();
         space = cctx.name();
         maxIterCnt = cctx.config().getMaximumQueryIteratorCount();
@@ -93,7 +93,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         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 GridException {
+                            @Override public void applyx(IgniteFuture<QueryResult<K, V>> f) throws IgniteCheckedException {
                                 f.get().closeIfNotShared(recipient);
                             }
                         });
@@ -108,7 +108,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                         entry.getValue().listenAsync(new CIX1<IgniteFuture<FieldsResult>>() {
                             @Override public void applyx(IgniteFuture<FieldsResult> f)
-                                throws GridException {
+                                throws IgniteCheckedException {
                                 f.get().closeIfNotShared(recipient);
                             }
                         });
@@ -158,9 +158,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      *
      * @param valType Value type.
      * @return Number of objects or -1 if type was not indexed at all.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public long size(Class<?> valType) throws GridException {
+    public long size(Class<?> valType) throws IgniteCheckedException {
         if (!enterBusy())
             throw new IllegalStateException("Failed to get size (grid is stopping).");
 
@@ -255,9 +255,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      *
      * @param swapSpaceName Swap space name.
      * @param key Key.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void onSwap(String swapSpaceName, K key) throws GridException {
+    public void onSwap(String swapSpaceName, K key) throws IgniteCheckedException {
         if (!enterBusy())
             return; // Ignore index update when node is stopping.
 
@@ -275,9 +275,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param key Key.
      * @param val Value
      * @param valBytes Value bytes.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void onUnswap(K key, V val, byte[] valBytes) throws GridException {
+    public void onUnswap(K key, V val, byte[] valBytes) throws IgniteCheckedException {
         if (!enterBusy())
             return; // Ignore index update when node is stopping.
 
@@ -306,11 +306,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param valBytes Value bytes.
      * @param ver Cache entry version.
      * @param expirationTime Expiration time or 0 if never expires.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     public void store(K key, @Nullable byte[] keyBytes, @Nullable V val, @Nullable byte[] valBytes,
         GridCacheVersion ver, long expirationTime)
-        throws GridException {
+        throws IgniteCheckedException {
         assert key != null;
         assert val != null || valBytes != null;
 
@@ -336,10 +336,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * @param key Key.
      * @param keyBytes Byte array with key value.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @SuppressWarnings("SimplifiableIfStatement")
-    public void remove(K key, @Nullable byte[] keyBytes) throws GridException {
+    public void remove(K key, @Nullable byte[] keyBytes) throws IgniteCheckedException {
         assert key != null;
 
         if (!cctx.config().isQueryIndexEnabled() && !(key instanceof GridCacheInternal))
@@ -370,8 +370,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         try {
             idxProc.onUndeploy(space, ldr);
         }
-        catch (GridException e) {
-            throw new GridRuntimeException(e);
+        catch (IgniteCheckedException e) {
+            throw new IgniteException(e);
         }
         finally {
             invalidateResultCache();
@@ -434,15 +434,15 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param taskName Task name.
      * @param recipient ID of the recipient.
      * @return Collection of found keys.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     private QueryResult<K, V> executeQuery(GridCacheQueryAdapter<?> qry,
         @Nullable Object[] args, boolean loc, @Nullable UUID subjId, @Nullable String taskName, Object recipient)
-        throws GridException {
+        throws IgniteCheckedException {
         if (qry.type() == null) {
             assert !loc;
 
-            throw new GridException("Received next page request after iterator was removed. " +
+            throw new IgniteCheckedException("Received next page request after iterator was removed. " +
                 "Consider increasing maximum number of stored iterators (see " +
                 "GridCacheConfiguration.getMaximumQueryIteratorCount() configuration property).");
         }
@@ -544,7 +544,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                     assert false : "SQL fields query is incorrectly processed.";
 
                 default:
-                    throw new GridException("Unknown query type: " + qry.type());
+                    throw new IgniteCheckedException("Unknown query type: " + qry.type());
             }
 
             res.onDone(iter);
@@ -570,10 +570,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param taskName Task name.
      * @param recipient ID of the recipient.
      * @return Collection of found keys.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     private FieldsResult executeFieldsQuery(GridCacheQueryAdapter<?> qry, @Nullable Object[] args,
-        boolean loc, @Nullable UUID subjId, @Nullable String taskName, Object recipient) throws GridException {
+        boolean loc, @Nullable UUID subjId, @Nullable String taskName, Object recipient) throws IgniteCheckedException {
         assert qry != null;
 
         FieldsResult res;
@@ -584,7 +584,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             if (qry.clause() == null) {
                 assert !loc;
 
-                throw new GridException("Received next page request after iterator was removed. " +
+                throw new IgniteCheckedException("Received next page request after iterator was removed. " +
                     "Consider increasing maximum number of stored iterators (see " +
                     "GridCacheConfiguration.getMaximumQueryIteratorCount() configuration property).");
             }
@@ -720,11 +720,11 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * @param qry Query.
      * @return Full-scan row iterator.
-     * @throws GridException If failed to get iterator.
+     * @throws IgniteCheckedException If failed to get iterator.
      */
     @SuppressWarnings({"unchecked"})
     private GridCloseableIterator<IgniteBiTuple<K, V>> scanIterator(final GridCacheQueryAdapter<?> qry)
-        throws GridException {
+        throws IgniteCheckedException {
         IgnitePredicate<GridCacheEntry<K, V>> filter = null;
 
         if (qry.projectionFilter() != null) {
@@ -846,10 +846,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * @param qry Query.
      * @return Swap iterator.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     private GridIterator<IgniteBiTuple<K, V>> swapIterator(GridCacheQueryAdapter<?> qry)
-        throws GridException {
+        throws IgniteCheckedException {
         IgnitePredicate<GridCacheEntry<Object, Object>> prjPred = qry.projectionFilter();
 
         IgniteBiPredicate<K, V> filter = qry.scanFilter();
@@ -952,9 +952,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
     /**
      * @param o Object to inject resources to.
-     * @throws GridException If failure occurred while injecting resources.
+     * @throws IgniteCheckedException If failure occurred while injecting resources.
      */
-    private void injectResources(@Nullable Object o) throws GridException {
+    private void injectResources(@Nullable Object o) throws IgniteCheckedException {
         if (o != null) {
             GridKernalContext ctx = cctx.kernalContext();
 
@@ -1102,7 +1102,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         Collections.singletonList(rdc.reduce()), true, null);
                 }
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (log.isDebugEnabled() || !e.hasCause(SQLException.class))
                     U.error(log, "Failed to run fields query [qry=" + qryInfo + ", node=" + cctx.nodeId() + ']', e);
                 else {
@@ -1378,9 +1378,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * @param qryInfo Info.
      * @return Iterator.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    private QueryResult<K, V> queryResult(GridCacheQueryInfo qryInfo, String taskName) throws GridException {
+    private QueryResult<K, V> queryResult(GridCacheQueryInfo qryInfo, String taskName) throws IgniteCheckedException {
         final UUID sndId = qryInfo.senderId();
 
         assert sndId != null;
@@ -1397,7 +1397,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         try {
                             e.getValue().get().closeIfNotShared(recipient(sndId, e.getKey()));
                         }
-                        catch (GridException ex) {
+                        catch (IgniteCheckedException ex) {
                             U.error(log, "Failed to close query iterator.", ex);
                         }
                     }
@@ -1419,12 +1419,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param futs Futures map.
      * @param qryInfo Info.
      * @return Iterator.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings({"SynchronizationOnLocalVariableOrMethodParameter",
         "NonPrivateFieldAccessedInSynchronizedContext"})
     private QueryResult<K, V> queryResult(Map<Long, GridFutureAdapter<QueryResult<K, V>>> futs,
-        GridCacheQueryInfo qryInfo, String taskName) throws GridException {
+        GridCacheQueryInfo qryInfo, String taskName) throws IgniteCheckedException {
         assert futs != null;
         assert qryInfo != null;
 
@@ -1482,7 +1482,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 try {
                     fut.get().closeIfNotShared(recipient(sndId, reqId));
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(log, "Failed to close iterator.", e);
                 }
             }
@@ -1503,10 +1503,10 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
     /**
      * @param qryInfo Info.
      * @return Iterator.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     private FieldsResult fieldsQueryResult(GridCacheQueryInfo qryInfo, String taskName)
-        throws GridException {
+        throws IgniteCheckedException {
         final UUID sndId = qryInfo.senderId();
 
         assert sndId != null;
@@ -1523,7 +1523,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         try {
                             e.getValue().get().closeIfNotShared(recipient(sndId, e.getKey()));
                         }
-                        catch (GridException ex) {
+                        catch (IgniteCheckedException ex) {
                             U.error(log, "Failed to close fields query iterator.", ex);
                         }
                     }
@@ -1549,12 +1549,12 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * @param resMap Results map.
      * @param qryInfo Info.
      * @return Fields query result.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings({"SynchronizationOnLocalVariableOrMethodParameter",
         "NonPrivateFieldAccessedInSynchronizedContext"})
     private FieldsResult fieldsQueryResult(Map<Long, GridFutureAdapter<FieldsResult>> resMap,
-        GridCacheQueryInfo qryInfo, String taskName) throws GridException {
+        GridCacheQueryInfo qryInfo, String taskName) throws IgniteCheckedException {
         assert resMap != null;
         assert qryInfo != null;
 
@@ -1578,7 +1578,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 fut.onDone(executeFieldsQuery(qryInfo.query(), qryInfo.arguments(), false,
                     qryInfo.query().subjectId(), taskName, recipient(qryInfo.senderId(), qryInfo.requestId())));
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 fut.onDone(e);
             }
         }
@@ -1610,7 +1610,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 try {
                     fut.get().closeIfNotShared(recipient(sndId, reqId));
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(log, "Failed to close iterator.", e);
                 }
             }
@@ -1674,9 +1674,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
      * Gets SQL metadata.
      *
      * @return SQL metadata.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    public Collection<GridCacheSqlMetadata> sqlMetadata() throws GridException {
+    public Collection<GridCacheSqlMetadata> sqlMetadata() throws IgniteCheckedException {
         if (!enterBusy())
             throw new IllegalStateException("Failed to get metadata (grid is stopping).");
 
@@ -2224,7 +2224,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         /**
          * @return Metadata.
          */
-        public List<GridQueryFieldMetadata> metaData() throws GridException {
+        public List<GridQueryFieldMetadata> metaData() throws IgniteCheckedException {
             get(); // Ensure that result is ready.
 
             return meta;
@@ -2255,9 +2255,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         /**
          * @return Value.
-         * @throws GridException If failed.
+         * @throws IgniteCheckedException If failed.
          */
-        protected abstract V unmarshalValue() throws GridException;
+        protected abstract V unmarshalValue() throws IgniteCheckedException;
 
         /**
          * @return Key.
@@ -2271,8 +2271,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 return key;
             }
-            catch (GridException e) {
-                throw new GridRuntimeException(e);
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
             }
         }
 
@@ -2288,8 +2288,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
                 return val;
             }
-            catch (GridException e) {
-                throw new GridRuntimeException(e);
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
             }
         }
 
@@ -2330,7 +2330,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         /** {@inheritDoc} */
         @SuppressWarnings("IfMayBeConditional")
-        @Override protected V unmarshalValue() throws GridException {
+        @Override protected V unmarshalValue() throws IgniteCheckedException {
             byte[] bytes = e.getValue();
 
             byte[] val = GridCacheSwapEntryImpl.getValueIfByteArray(bytes);
@@ -2393,7 +2393,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override protected V unmarshalValue() throws GridException {
+        @Override protected V unmarshalValue() throws IgniteCheckedException {
             long ptr = GridCacheOffheapSwapEntry.valueAddress(valPtr.get1(), valPtr.get2());
 
             V val = (V)cctx.portable().unmarshal(ptr, false);
@@ -2455,7 +2455,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         /** {@inheritDoc} */
         @Nullable @Override public IgniteBiTuple<K, V> applyx(T2<Long, Integer> keyPtr,
             T2<Long, Integer> valPtr)
-            throws GridException {
+            throws IgniteCheckedException {
             LazyOffheapEntry e = new LazyOffheapEntry(keyPtr, valPtr);
 
             if (prjPred != null) {
@@ -2506,7 +2506,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public boolean hasNextX() throws GridException {
+        @Override public boolean hasNextX() throws IgniteCheckedException {
             if (iter.hasNextX())
                 return true;
 
@@ -2525,7 +2525,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public T nextX() throws GridException {
+        @Override public T nextX() throws IgniteCheckedException {
             if (!hasNextX())
                 throw new NoSuchElementException();
 
@@ -2533,7 +2533,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public void removeX() throws GridException {
+        @Override public void removeX() throws IgniteCheckedException {
             throw new UnsupportedOperationException();
         }
     }
@@ -2591,7 +2591,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public V peek(@Nullable Collection<GridCachePeekMode> modes) throws GridException {
+        @Nullable @Override public V peek(@Nullable Collection<GridCachePeekMode> modes) throws IgniteCheckedException {
             return null;
         }
 
@@ -2753,7 +2753,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override public boolean remove(V val) throws GridException {
+        @Override public boolean remove(V val) throws IgniteCheckedException {
             throw new UnsupportedOperationException();
         }
 
@@ -2902,9 +2902,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
          * Close if this result does not have any other recipients.
          *
          * @param recipient ID of the recipient.
-         * @throws GridException If failed.
+         * @throws IgniteCheckedException If failed.
          */
-        public void closeIfNotShared(Object recipient) throws GridException {
+        public void closeIfNotShared(Object recipient) throws IgniteCheckedException {
             assert isDone();
 
             synchronized (recipients) {
@@ -2969,9 +2969,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         /**
          * @param recipient ID of the recipient.
-         * @throws GridException If failed.
+         * @throws IgniteCheckedException If failed.
          */
-        public IgniteSpiCloseableIterator<R> iterator(Object recipient) throws GridException {
+        public IgniteSpiCloseableIterator<R> iterator(Object recipient) throws IgniteCheckedException {
             assert recipient != null;
 
             IgniteSpiCloseableIterator<R> it = get();
@@ -3019,7 +3019,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             }
 
             /** {@inheritDoc} */
-            @Override public void close() throws GridException {
+            @Override public void close() throws IgniteCheckedException {
                 closeIfNotShared(recipient);
             }
 
@@ -3045,8 +3045,8 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 try {
                     it = get();
                 }
-                catch (GridException e) {
-                    throw new GridRuntimeException(e);
+                catch (IgniteCheckedException e) {
+                    throw new IgniteException(e);
                 }
 
                 synchronized (recipients) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryRequest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryRequest.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryRequest.java
index 221eebb..2abaae2 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryRequest.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryRequest.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.cache.*;
@@ -230,7 +230,7 @@ public class GridCacheQueryRequest<K, V> extends GridCacheMessage<K, V> implemen
 
     /** {@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 (keyValFilter != null) {
@@ -272,7 +272,7 @@ public class GridCacheQueryRequest<K, V> extends GridCacheMessage<K, V> implemen
     }
 
     /** {@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);
 
         IgniteMarshaller mrsh = ctx.marshaller();
@@ -295,9 +295,9 @@ public class GridCacheQueryRequest<K, V> extends GridCacheMessage<K, V> implemen
 
     /**
      * @param ctx Context.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    void beforeLocalExecution(GridCacheContext<K, V> ctx) throws GridException {
+    void beforeLocalExecution(GridCacheContext<K, V> ctx) throws IgniteCheckedException {
         IgniteMarshaller marsh = ctx.marshaller();
 
         rdc = rdc != null ? marsh.<IgniteReducer<Object, Object>>unmarshal(marsh.marshal(rdc), null) : null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryResponse.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryResponse.java
index 479f2e8..caf783c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryResponse.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryResponse.java
@@ -9,14 +9,14 @@
 
 package org.gridgain.grid.kernal.processors.cache.query;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.query.*;
 import org.gridgain.grid.util.direct.*;
+import org.gridgain.grid.util.tostring.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.grid.util.tostring.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -97,7 +97,7 @@ public class GridCacheQueryResponse<K, V> extends GridCacheMessage<K, V> impleme
 
     /** {@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 (err != null)
@@ -119,7 +119,7 @@ public class GridCacheQueryResponse<K, V> extends GridCacheMessage<K, V> impleme
     }
 
     /** {@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 (errBytes != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
index 910f563..9b88858 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
@@ -212,12 +212,12 @@ public class GridCacheContinuousQueryAdapter<K, V> implements GridCacheContinuou
     }
 
     /** {@inheritDoc} */
-    @Override public void execute() throws GridException {
+    @Override public void execute() throws IgniteCheckedException {
         execute(null, false);
     }
 
     /** {@inheritDoc} */
-    @Override public void execute(@Nullable ClusterGroup prj) throws GridException {
+    @Override public void execute(@Nullable ClusterGroup prj) throws IgniteCheckedException {
         execute(prj, false);
     }
 
@@ -226,9 +226,9 @@ public class GridCacheContinuousQueryAdapter<K, V> implements GridCacheContinuou
      *
      * @param prj Grid projection.
      * @param internal If {@code true} then query notified about internal entries updates.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void execute(@Nullable ClusterGroup prj, boolean internal) throws GridException {
+    public void execute(@Nullable ClusterGroup prj, boolean internal) throws IgniteCheckedException {
         if (locCb == null)
             throw new IllegalStateException("Mandatory local callback is not set for the query: " + this);
 
@@ -285,7 +285,7 @@ public class GridCacheContinuousQueryAdapter<K, V> implements GridCacheContinuou
     }
 
     /** {@inheritDoc} */
-    @Override public void close() throws GridException {
+    @Override public void close() throws IgniteCheckedException {
         closeLock.lock();
 
         try {