http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java
index 97316e3..d09f126 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/interop/os/GridOsInteropProcessor.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.interop.os;
 
+import org.apache.ignite.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.interop.*;
@@ -36,12 +37,12 @@ public class GridOsInteropProcessor extends 
GridInteropProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void awaitStart() throws GridException {
+    @Override public void awaitStart() throws IgniteCheckedException {
         throw new UnsupportedOperationException(ERR_MSG);
     }
 
     /** {@inheritDoc} */
-    @Override public long environmentPointer() throws GridException {
+    @Override public long environmentPointer() throws IgniteCheckedException {
         throw new UnsupportedOperationException(ERR_MSG);
     }
 
@@ -56,12 +57,12 @@ public class GridOsInteropProcessor extends 
GridInteropProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public GridInteropTarget projection() throws GridException {
+    @Override public GridInteropTarget projection() throws 
IgniteCheckedException {
         throw new UnsupportedOperationException(ERR_MSG);
     }
 
     /** {@inheritDoc} */
-    @Override public GridInteropTarget cache(@Nullable String name) throws 
GridException {
+    @Override public GridInteropTarget cache(@Nullable String name) throws 
IgniteCheckedException {
         throw new UnsupportedOperationException(ERR_MSG);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java
index 9a4a21e..3f7d4e1 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobProcessor.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.job;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.events.*;
@@ -176,9 +177,9 @@ public class GridJobProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void start() throws GridException {
+    @Override public void start() throws IgniteCheckedException {
         if (metricsUpdateFreq < -1)
-            throw new GridException("Invalid value for 
'metricsUpdateFrequency' configuration property " +
+            throw new IgniteCheckedException("Invalid value for 
'metricsUpdateFrequency' configuration property " +
                 "(should be greater than or equals to -1): " + 
metricsUpdateFreq);
 
         if (metricsUpdateFreq == -1)
@@ -291,7 +292,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
      * @param sndReply {@code True} to send reply.
      */
     private void rejectJob(GridJobWorker job, boolean sndReply) {
-        GridException e = new ComputeExecutionRejectedException("Job was 
cancelled before execution [taskSesId=" +
+        IgniteCheckedException e = new ComputeExecutionRejectedException("Job 
was cancelled before execution [taskSesId=" +
             job.getSession().getId() + ", jobId=" + job.getJobId() + ", job=" 
+ job.getJob() + ']');
 
         job.finishJob(null, e, sndReply);
@@ -325,9 +326,9 @@ public class GridJobProcessor extends GridProcessorAdapter {
     /**
      * @param ses Session.
      * @param attrs Attributes.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void setAttributes(GridJobSessionImpl ses, Map<?, ?> attrs) throws 
GridException {
+    public void setAttributes(GridJobSessionImpl ses, Map<?, ?> attrs) throws 
IgniteCheckedException {
         assert ses.isFullSupport();
 
         long timeout = ses.getEndTime() - U.currentTimeMillis();
@@ -344,7 +345,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
         ClusterNode taskNode = ctx.discovery().node(ses.getTaskNodeId());
 
         if (taskNode == null)
-            throw new GridException("Node that originated task execution has 
left grid: " +
+            throw new IgniteCheckedException("Node that originated task 
execution has left grid: " +
                 ses.getTaskNodeId());
 
         boolean loc = ctx.localNodeId().equals(taskNode.id()) && 
!ctx.config().isMarshalLocalJobs();
@@ -368,9 +369,9 @@ public class GridJobProcessor extends GridProcessorAdapter {
     /**
      * @param ses Session.
      * @return Siblings.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public Collection<ComputeJobSibling> requestJobSiblings(final 
ComputeTaskSession ses) throws GridException {
+    public Collection<ComputeJobSibling> requestJobSiblings(final 
ComputeTaskSession ses) throws IgniteCheckedException {
         assert ses != null;
 
         final UUID taskNodeId = ses.getTaskNodeId();
@@ -378,7 +379,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
         ClusterNode taskNode = ctx.discovery().node(taskNodeId);
 
         if (taskNode == null)
-            throw new GridException("Node that originated task execution has 
left grid: " + taskNodeId);
+            throw new IgniteCheckedException("Node that originated task 
execution has left grid: " + taskNodeId);
 
         // Tuple: error message-response.
         final IgniteBiTuple<String, GridJobSiblingsResponse> t = F.t2();
@@ -404,7 +405,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
                     try {
                         res.unmarshalSiblings(marsh);
                     }
-                    catch (GridException e) {
+                    catch (IgniteCheckedException e) {
                         U.error(log, "Failed to unmarshal job siblings.", e);
 
                         err = e.getMessage();
@@ -473,7 +474,7 @@ public class GridJobProcessor extends GridProcessorAdapter {
             taskNode = ctx.discovery().node(taskNodeId);
 
             if (taskNode == null)
-                throw new GridException("Node that originated task execution 
has left grid: " + taskNodeId);
+                throw new IgniteCheckedException("Node that originated task 
execution has left grid: " + taskNodeId);
 
             // 6. Wait for result.
             lock.lock();
@@ -485,18 +486,18 @@ public class GridJobProcessor extends 
GridProcessorAdapter {
                     cond.await(netTimeout, MILLISECONDS);
 
                 if (t.isEmpty())
-                    throw new GridException("Timed out waiting for job 
siblings (consider increasing" +
+                    throw new IgniteCheckedException("Timed out waiting for 
job siblings (consider increasing" +
                         "'networkTimeout' configuration property) [ses=" + ses 
+ ", netTimeout=" + netTimeout + ']');
 
                 // Error is set?
                 if (t.get1() != null)
-                    throw new GridException(t.get1());
+                    throw new IgniteCheckedException(t.get1());
                 else
                     // Return result
                     return t.get2().jobSiblings();
             }
             catch (InterruptedException e) {
-                throw new GridException("Interrupted while waiting for job 
siblings response: " + ses, e);
+                throw new IgniteCheckedException("Interrupted while waiting 
for job siblings response: " + ses, e);
             }
             finally {
                 lock.unlock();
@@ -1009,8 +1010,8 @@ public class GridJobProcessor extends 
GridProcessorAdapter {
 
                         jobCtx = new GridJobContextImpl(ctx, req.getJobId(), 
jobAttrs);
                     }
-                    catch (GridException e) {
-                        GridException ex = new GridException("Failed to 
deserialize task attributes [taskName=" +
+                    catch (IgniteCheckedException e) {
+                        IgniteCheckedException ex = new 
IgniteCheckedException("Failed to deserialize task attributes [taskName=" +
                             req.getTaskName() + ", taskClsName=" + 
req.getTaskClassName() + ", codeVer=" +
                             req.getUserVersion() + ", taskClsLdr=" + 
dep.classLoader() + ']', e);
 
@@ -1090,7 +1091,7 @@ public class GridJobProcessor extends 
GridProcessorAdapter {
                 }
                 else {
                     // Deployment is null.
-                    GridException ex = new GridDeploymentException("Task was 
not deployed or was redeployed since " +
+                    IgniteCheckedException ex = new 
GridDeploymentException("Task was not deployed or was redeployed since " +
                         "task execution [taskName=" + req.getTaskName() + ", 
taskClsName=" + req.getTaskClassName() +
                         ", codeVer=" + req.getUserVersion() + ", clsLdrId=" + 
req.getClassLoaderId() +
                         ", seqNum=" + req.getClassLoaderId().localId() + ", 
depMode=" + req.getDeploymentMode() +
@@ -1173,7 +1174,7 @@ public class GridJobProcessor extends 
GridProcessorAdapter {
 
             // Even if job has been removed from another thread, we need to 
reject it
             // here since job has never been executed.
-            GridException e2 = new ComputeExecutionRejectedException(
+            IgniteCheckedException e2 = new ComputeExecutionRejectedException(
                 "Job was cancelled before execution [jobSes=" + jobWorker.
                     getSession() + ", job=" + jobWorker.getJob() + ']');
 
@@ -1221,7 +1222,7 @@ public class GridJobProcessor extends 
GridProcessorAdapter {
 
             // Even if job was removed from another thread, we need to reject 
it
             // here since job has never been executed.
-            GridException e2 = new ComputeExecutionRejectedException("Job has 
been rejected " +
+            IgniteCheckedException e2 = new 
ComputeExecutionRejectedException("Job has been rejected " +
                 "[jobSes=" + jobWorker.getSession() + ", job=" + 
jobWorker.getJob() + ']', e);
 
             if (metricsUpdateFreq > -1L)
@@ -1241,7 +1242,7 @@ public class GridJobProcessor extends 
GridProcessorAdapter {
      * @param ex Exception that happened.
      * @param endTime Job end time.
      */
-    private void handleException(ClusterNode node, GridJobExecuteRequest req, 
GridException ex, long endTime) {
+    private void handleException(ClusterNode node, GridJobExecuteRequest req, 
IgniteCheckedException ex, long endTime) {
         UUID locNodeId = ctx.localNodeId();
 
         ClusterNode sndNode = ctx.discovery().node(node.id());
@@ -1319,7 +1320,7 @@ public class GridJobProcessor extends 
GridProcessorAdapter {
                 // Send response to common topic as unordered message.
                 ctx.io().send(sndNode, TOPIC_TASK, jobRes, req.isInternal() ? 
MANAGEMENT_POOL : SYSTEM_POOL);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             // The only option here is to log, as we must assume that 
resending will fail too.
             if (isDeadNode(node.id()))
                 // Avoid stack trace for left nodes.
@@ -1399,7 +1400,7 @@ public class GridJobProcessor extends 
GridProcessorAdapter {
                 ses.setInternal(attrs);
             }
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to deserialize session attributes.", e);
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java
index abbd45e..9d100e3 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/job/GridJobWorker.java
@@ -365,7 +365,7 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
     boolean initialize(GridDeployment dep, Class<?> taskCls) {
         assert dep != null;
 
-        GridException ex = null;
+        IgniteCheckedException ex = null;
 
         try {
             if (job == null) {
@@ -381,7 +381,7 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
             if (!internal && ctx.event().isRecordable(EVT_JOB_QUEUED))
                 recordEvent(EVT_JOB_QUEUED, "Job got queued for computation.");
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to initialize job [jobId=" + ses.getJobId() + 
", ses=" + ses + ']', e);
 
             ex = e;
@@ -452,7 +452,7 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
 
         Object res = null;
 
-        GridException ex = null;
+        IgniteCheckedException ex = null;
 
         try {
             ctx.job().currentTaskSession(ses);
@@ -463,7 +463,7 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
                 sndRes = false;
             else {
                 res = U.wrapThreadLoader(dep.classLoader(), new 
Callable<Object>() {
-                    @Nullable @Override public Object call() throws 
GridException {
+                    @Nullable @Override public Object call() throws 
IgniteCheckedException {
                         try {
                             if (internal && 
ctx.config().isPeerClassLoadingEnabled())
                                 ctx.job().internal(true);
@@ -481,7 +481,7 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
                     log.debug("Job execution has successfully finished [job=" 
+ job + ", res=" + res + ']');
             }
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             if (sysStopping && e.hasCause(GridInterruptedException.class, 
InterruptedException.class)) {
                 ex = handleThrowable(e);
 
@@ -530,10 +530,10 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
      * @param e Exception.
      * @return Wrapped exception.
      */
-    private GridException handleThrowable(Throwable e) {
+    private IgniteCheckedException handleThrowable(Throwable e) {
         String msg = null;
 
-        GridException ex = null;
+        IgniteCheckedException ex = null;
 
         // Special handling for weird interrupted exception which
         // happens due to JDk 1.5 bug.
@@ -541,7 +541,7 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
             msg = "Failed to execute job due to interrupted exception.";
 
             // Turn interrupted exception into checked exception.
-            ex = new GridException(msg, e);
+            ex = new IgniteCheckedException(msg, e);
         }
         // Special 'NoClassDefFoundError' handling if P2P is on. We had many 
questions
         // about this exception and decided to change error message.
@@ -640,7 +640,7 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
      * @param ex Error.
      * @param sndReply If {@code true}, reply will be sent.
      */
-    void finishJob(@Nullable Object res, @Nullable GridException ex, boolean 
sndReply) {
+    void finishJob(@Nullable Object res, @Nullable IgniteCheckedException ex, 
boolean sndReply) {
         // Avoid finishing a job more than once from different threads.
         if (!finishing.compareAndSet(false, true))
             return;
@@ -735,7 +735,7 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
                                 // Send response to common topic as unordered 
message.
                                 ctx.io().send(sndNode, TOPIC_TASK, jobRes, 
internal ? MANAGEMENT_POOL : SYSTEM_POOL);
                         }
-                        catch (GridException e) {
+                        catch (IgniteCheckedException e) {
                             // Log and invoke the master-leave callback.
                             if (isDeadNode(taskNode.id())) {
                                 onMasterNodeLeft();
@@ -811,7 +811,7 @@ public class GridJobWorker extends GridWorker implements 
GridTimeoutObject {
                         log.debug("Successfully executed 
GridComputeJobMasterLeaveAware.onMasterNodeLeft() callback " +
                             "[nodeId=" + taskNode.id() + ", jobId=" + 
ses.getJobId() + ", job=" + job + ']');
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(log, "Failed to execute 
GridComputeJobMasterLeaveAware.onMasterNodeLeft() callback " +
                         "[nodeId=" + taskNode.id() + ", jobId=" + 
ses.getJobId() + ", job=" + job + ']', e);
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java
index 1edfd9b..857bf7c 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/jobmetrics/GridJobMetricsProcessor.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.jobmetrics;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.*;
 import org.gridgain.grid.util.typedef.*;
@@ -114,12 +114,12 @@ public class GridJobMetricsProcessor extends 
GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void start() throws GridException {
+    @Override public void start() throws IgniteCheckedException {
         assertParameter(histSize > 0, "metricsHistorySize > 0");
         assertParameter(expireTime > 0, "metricsExpireTime > 0");
 
         if (metrics.snapshotsQueues == null)
-            throw new GridException("Invalid concurrency level configured " +
+            throw new IgniteCheckedException("Invalid concurrency level 
configured " +
                 "(is 'GRIDGAIN_JOBS_METRICS_CONCURRENCY_LEVEL' system property 
properly set?).");
 
         if (log.isDebugEnabled())
@@ -129,7 +129,7 @@ public class GridJobMetricsProcessor extends 
GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) throws GridException {
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Job metrics processor stopped.");
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java
index 11d0ca8..10c699a 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/offheap/GridOffHeapProcessor.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.offheap;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.*;
 import org.gridgain.grid.util.*;
@@ -65,7 +65,7 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) throws GridException {
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
         super.stop(cancel);
 
         for (GridOffHeapPartitionedMap m : offheap.values())
@@ -89,9 +89,9 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
      * @param key Key.
      * @param keyBytes Optional key bytes.
      * @return Key bytes
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private byte[] keyBytes(Object key, @Nullable byte[] keyBytes) throws 
GridException {
+    private byte[] keyBytes(Object key, @Nullable byte[] keyBytes) throws 
IgniteCheckedException {
         assert key != null;
 
         return keyBytes != null ? keyBytes : marsh.marshal(key);
@@ -118,9 +118,9 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
      * @param key Key.
      * @param keyBytes Key bytes.
      * @return {@code true} If offheap space contains value for the given key.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public boolean contains(@Nullable String spaceName, int part, Object key, 
byte[] keyBytes) throws GridException {
+    public boolean contains(@Nullable String spaceName, int part, Object key, 
byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m != null && m.contains(part, U.hash(key), keyBytes(key, 
keyBytes));
@@ -134,9 +134,9 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
      * @param key Key.
      * @param keyBytes Key bytes.
      * @return Value bytes.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    @Nullable public byte[] get(@Nullable String spaceName, int part, Object 
key, byte[] keyBytes) throws GridException {
+    @Nullable public byte[] get(@Nullable String spaceName, int part, Object 
key, byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m == null ? null : m.get(part, U.hash(key), keyBytes(key, 
keyBytes));
@@ -152,10 +152,10 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
      * @param key Key.
      * @param keyBytes Key bytes.
      * @return Tuple where first value is pointer and second is value size.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @Nullable public IgniteBiTuple<Long, Integer> valuePointer(@Nullable 
String spaceName, int part, Object key,
-        byte[] keyBytes) throws GridException {
+        byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m == null ? null : m.valuePointer(part, U.hash(key), 
keyBytes(key, keyBytes));
@@ -168,9 +168,9 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
      * @param part Partition.
      * @param key Key.
      * @param keyBytes Key bytes.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void enableEviction(@Nullable String spaceName, int part, Object 
key, byte[] keyBytes) throws GridException {
+    public void enableEviction(@Nullable String spaceName, int part, Object 
key, byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         if (m != null)
@@ -186,10 +186,10 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
      * @param keyBytes Key bytes.
      * @param ldr Class loader.
      * @return Value bytes.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @Nullable public <T> T getValue(@Nullable String spaceName, int part, 
Object key, byte[] keyBytes,
-        @Nullable ClassLoader ldr) throws GridException {
+        @Nullable ClassLoader ldr) throws IgniteCheckedException {
         byte[] valBytes = get(spaceName, part, key, keyBytes);
 
         if (valBytes == null)
@@ -206,9 +206,9 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
      * @param key Key.
      * @param keyBytes Key bytes.
      * @return Value bytes.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    @Nullable public byte[] remove(@Nullable String spaceName, int part, 
Object key, byte[] keyBytes) throws GridException {
+    @Nullable public byte[] remove(@Nullable String spaceName, int part, 
Object key, byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m == null ? null : m.remove(part, U.hash(key), keyBytes(key, 
keyBytes));
@@ -222,14 +222,14 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
      * @param key Key.
      * @param keyBytes Key bytes.
      * @param valBytes Value bytes.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public void put(@Nullable String spaceName, int part, Object key, byte[] 
keyBytes, byte[] valBytes)
-        throws GridException {
+        throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         if (m == null)
-            throw new GridException("Failed to write data to off-heap space, 
no space registered for name: " +
+            throw new IgniteCheckedException("Failed to write data to off-heap 
space, no space registered for name: " +
                 spaceName);
 
         m.put(part, U.hash(key), keyBytes(key, keyBytes), valBytes);
@@ -243,9 +243,9 @@ public class GridOffHeapProcessor extends 
GridProcessorAdapter {
      * @param key Key.
      * @param keyBytes Key bytes.
      * @return {@code true} If succeeded.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public boolean removex(@Nullable String spaceName, int part, Object key, 
byte[] keyBytes) throws GridException {
+    public boolean removex(@Nullable String spaceName, int part, Object key, 
byte[] keyBytes) throws IgniteCheckedException {
         GridOffHeapPartitionedMap m = offheap(spaceName);
 
         return m != null && m.removex(part, U.hash(key), keyBytes(key, 
keyBytes));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java
index 4b80cd3..0a7146c 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/port/GridPortProcessor.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.port;
 
+import org.apache.ignite.*;
 import org.apache.ignite.spi.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
@@ -44,13 +45,13 @@ public class GridPortProcessor extends GridProcessorAdapter 
{
     }
 
     /** {@inheritDoc} */
-    @Override public void start() throws GridException {
+    @Override public void start() throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Started port processor.");
     }
 
     /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) throws GridException {
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Stopped port processor.");
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java
index e22bfd4..18b2832 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryIndexing.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.query;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.indexing.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.util.lang.*;
 import org.jetbrains.annotations.*;
@@ -26,16 +26,16 @@ public interface GridQueryIndexing {
      * Starts indexing.
      *
      * @param ctx Context.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void start(GridKernalContext ctx) throws GridException;
+    public void start(GridKernalContext ctx) throws IgniteCheckedException;
 
     /**
      * Stops indexing.
      *
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void stop() throws GridException;
+    public void stop() throws IgniteCheckedException;
 
     /**
      * Queries individual fields (generally used by JDBC drivers).
@@ -45,10 +45,10 @@ public interface GridQueryIndexing {
      * @param params Query parameters.
      * @param filters Space name and key filters.
      * @return Query result.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public <K, V> GridQueryFieldsResult queryFields(@Nullable String 
spaceName, String qry,
-        Collection<Object> params, GridIndexingQueryFilter filters) throws 
GridException;
+        Collection<Object> params, GridIndexingQueryFilter filters) throws 
IgniteCheckedException;
 
     /**
      * Executes regular query.
@@ -59,10 +59,10 @@ public interface GridQueryIndexing {
      * @param type Query return type.
      * @param filters Space name and key filters.
      * @return Queried rows.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> query(@Nullable 
String spaceName, String qry,
-        Collection<Object> params, GridQueryTypeDescriptor type, 
GridIndexingQueryFilter filters) throws GridException;
+        Collection<Object> params, GridQueryTypeDescriptor type, 
GridIndexingQueryFilter filters) throws IgniteCheckedException;
 
     /**
      * Executes text query.
@@ -72,10 +72,10 @@ public interface GridQueryIndexing {
      * @param type Query return type.
      * @param filters Space name and key filter.
      * @return Queried rows.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> 
queryText(@Nullable String spaceName, String qry,
-        GridQueryTypeDescriptor type, GridIndexingQueryFilter filters) throws 
GridException;
+        GridQueryTypeDescriptor type, GridIndexingQueryFilter filters) throws 
IgniteCheckedException;
 
     /**
      * Gets size of index for given type or -1 if it is a unknown type.
@@ -84,29 +84,29 @@ public interface GridQueryIndexing {
      * @param desc Type descriptor.
      * @param filters Filters.
      * @return Objects number.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public long size(@Nullable String spaceName, GridQueryTypeDescriptor desc, 
GridIndexingQueryFilter filters)
-        throws GridException;
+        throws IgniteCheckedException;
 
     /**
      * Registers type if it was not known before or updates it otherwise.
      *
      * @param spaceName Space name.
      * @param desc Type descriptor.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      * @return {@code True} if type was registered, {@code false} if for some 
reason it was rejected.
      */
-    public boolean registerType(@Nullable String spaceName, 
GridQueryTypeDescriptor desc) throws GridException;
+    public boolean registerType(@Nullable String spaceName, 
GridQueryTypeDescriptor desc) throws IgniteCheckedException;
 
     /**
      * Unregisters type and removes all corresponding data.
      *
      * @param spaceName Space name.
      * @param type Type descriptor.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void unregisterType(@Nullable String spaceName, 
GridQueryTypeDescriptor type) throws GridException;
+    public void unregisterType(@Nullable String spaceName, 
GridQueryTypeDescriptor type) throws IgniteCheckedException;
 
     /**
      * Updates index. Note that key is unique for space, so if space contains 
multiple indexes
@@ -118,28 +118,28 @@ public interface GridQueryIndexing {
      * @param val Value.
      * @param ver Version.
      * @param expirationTime Expiration time or 0 if never expires.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public void store(@Nullable String spaceName, GridQueryTypeDescriptor 
type, Object key, Object val, byte[] ver,
-        long expirationTime) throws GridException;
+        long expirationTime) throws IgniteCheckedException;
 
     /**
      * Removes index entry by key.
      *
      * @param spaceName Space name.
      * @param key Key.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void remove(@Nullable String spaceName, Object key) throws 
GridException;
+    public void remove(@Nullable String spaceName, Object key) throws 
IgniteCheckedException;
 
     /**
      * Will be called when entry with given key is swapped.
      *
      * @param spaceName Space name.
      * @param key Key.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void onSwap(@Nullable String spaceName, Object key) throws 
GridException;
+    public void onSwap(@Nullable String spaceName, Object key) throws 
IgniteCheckedException;
 
     /**
      * Will be called when entry with given key is unswapped.
@@ -148,9 +148,9 @@ public interface GridQueryIndexing {
      * @param key Key.
      * @param val Value.
      * @param valBytes Value bytes.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void onUnswap(@Nullable String spaceName, Object key, Object val, 
byte[] valBytes) throws GridException;
+    public void onUnswap(@Nullable String spaceName, Object key, Object val, 
byte[] valBytes) throws IgniteCheckedException;
 
     /**
      * Rebuilds all indexes of given type.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java
index 94db08d..6bc0235 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryProcessor.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.query;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.portables.*;
 import org.apache.ignite.spi.indexing.*;
@@ -71,7 +72,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     /**
      * @param ctx Kernal context.
      */
-    public GridQueryProcessor(GridKernalContext ctx) throws GridException {
+    public GridQueryProcessor(GridKernalContext ctx) throws 
IgniteCheckedException {
         super(ctx);
 
         if (idxCls != null) {
@@ -84,7 +85,7 @@ public class GridQueryProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void start() throws GridException {
+    @Override public void start() throws IgniteCheckedException {
         super.start();
 
         if (idx != null) {
@@ -118,7 +119,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void stop(boolean cancel) throws GridException {
+    @Override public void stop(boolean cancel) throws IgniteCheckedException {
         super.stop(cancel);
 
         if (idx != null)
@@ -131,9 +132,9 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param space Space.
      * @param valType Value type.
      * @return Objects number or -1 if this type is unknown for given SPI and 
space.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public long size(@Nullable String space, Class<?> valType) throws 
GridException {
+    public long size(@Nullable String space, Class<?> valType) throws 
IgniteCheckedException {
         checkEnabled();
 
         if (!busyLock.enterBusy())
@@ -178,7 +179,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      */
     private IgniteFuture<?> rebuildIndexes(@Nullable final String space, 
@Nullable final TypeDescriptor desc) {
         if (idx == null)
-            return new GridFinishedFuture<>(ctx, new GridException("Indexing 
is disabled."));
+            return new GridFinishedFuture<>(ctx, new 
IgniteCheckedException("Indexing is disabled."));
 
         if (desc == null || !desc.registered())
             return new GridFinishedFuture<Void>(ctx);
@@ -245,11 +246,11 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param valBytes Byte array with value data.
      * @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.
      */
     @SuppressWarnings("unchecked")
     public <K, V> void store(final String space, final K key, @Nullable byte[] 
keyBytes, final V val,
-        @Nullable byte[] valBytes, byte[] ver, long expirationTime) throws 
GridException {
+        @Nullable byte[] valBytes, byte[] ver, long expirationTime) throws 
IgniteCheckedException {
         assert key != null;
         assert val != null;
 
@@ -376,7 +377,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
                 return;
 
             if (!desc.valueClass().equals(valCls))
-                throw new GridException("Failed to update index due to class 
name conflict" +
+                throw new IgniteCheckedException("Failed to update index due 
to class name conflict" +
                     "(multiple classes with same simple name are stored in the 
same cache) " +
                     "[expCls=" + desc.valueClass().getName() + ", actualCls=" 
+ valCls.getName() + ']');
 
@@ -388,11 +389,11 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
     }
 
     /**
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private void checkEnabled() throws GridException {
+    private void checkEnabled() throws IgniteCheckedException {
         if (idx == null)
-            throw new GridException("Indexing is disabled.");
+            throw new IgniteCheckedException("Indexing is disabled.");
     }
 
     /**
@@ -402,12 +403,12 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param resType Result type.
      * @param filters Filters.
      * @return Key/value rows.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
     public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> query(String 
space, String clause,
         Collection<Object> params, String resType, GridIndexingQueryFilter 
filters)
-        throws GridException {
+        throws IgniteCheckedException {
         checkEnabled();
 
         if (!busyLock.enterBusy())
@@ -429,10 +430,10 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
     /**
      * @param space Space.
      * @param key Key.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
     @SuppressWarnings("unchecked")
-    public void remove(String space, Object key) throws GridException {
+    public void remove(String space, Object key) throws IgniteCheckedException 
{
         assert key != null;
 
         ctx.indexing().remove(space, key);
@@ -546,11 +547,11 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param <K> Key type.
      * @param <V> Value type.
      * @return Key/value rows.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
     public <K, V> GridCloseableIterator<IgniteBiTuple<K, V>> queryText(String 
space, String clause, String resType,
-        GridIndexingQueryFilter filters) throws GridException {
+        GridIndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
         if (!busyLock.enterBusy())
@@ -575,10 +576,10 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param params Parameters collection.
      * @param filters Key and value filters.
      * @return Field rows.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public <K, V> GridQueryFieldsResult queryFields(@Nullable String space, 
String clause, Collection<Object> params,
-        GridIndexingQueryFilter filters) throws GridException {
+        GridIndexingQueryFilter filters) throws IgniteCheckedException {
         checkEnabled();
 
         if (!busyLock.enterBusy())
@@ -597,9 +598,9 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      *
      * @param spaceName Space name.
      * @param key key.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void onSwap(String spaceName, Object key) throws GridException {
+    public void onSwap(String spaceName, Object key) throws 
IgniteCheckedException {
         ctx.indexing().onSwap(spaceName, key);
 
         if (idx == null)
@@ -623,10 +624,10 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param key Key.
      * @param val Value.
      * @param valBytes Value bytes.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public void onUnswap(String spaceName, Object key, Object val, byte[] 
valBytes)
-        throws GridException {
+        throws IgniteCheckedException {
         ctx.indexing().onUnswap(spaceName, key, val);
 
         if (idx == null)
@@ -648,9 +649,9 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      *
      * @param space Space name.
      * @param ldr Class loader to undeploy.
-     * @throws GridException If undeploy failed.
+     * @throws IgniteCheckedException If undeploy failed.
      */
-    public void onUndeploy(@Nullable String space, ClassLoader ldr) throws 
GridException {
+    public void onUndeploy(@Nullable String space, ClassLoader ldr) throws 
IgniteCheckedException {
         if (idx == null)
             return;
 
@@ -687,15 +688,15 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param cls Class.
      * @param type Type descriptor.
      * @param parent Parent in case of embeddable.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     static void processAnnotationsInClass(boolean key, Class<?> cls, 
TypeDescriptor type,
-        @Nullable ClassProperty parent) throws GridException {
+        @Nullable ClassProperty parent) throws IgniteCheckedException {
         if (U.isJdk(cls))
             return;
 
         if (parent != null && parent.knowsClass(cls))
-            throw new GridException("Recursive reference found in type: " + 
cls.getName());
+            throw new IgniteCheckedException("Recursive reference found in 
type: " + cls.getName());
 
         if (parent == null) { // Check class annotation at top level only.
             GridCacheQueryTextField txtAnnCls = 
cls.getAnnotation(GridCacheQueryTextField.class);
@@ -738,7 +739,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
 
                 if (sqlAnn != null || txtAnn != null) {
                     if (mtd.getParameterTypes().length != 0)
-                        throw new GridException("Getter with 
GridCacheQuerySqlField " +
+                        throw new IgniteCheckedException("Getter with 
GridCacheQuerySqlField " +
                             "annotation cannot have parameters: " + mtd);
 
                     ClassProperty prop = new ClassProperty(mtd);
@@ -762,10 +763,10 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param cls Class of field or return type for method.
      * @param prop Current property.
      * @param desc Class description.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     static void processAnnotation(boolean key, GridCacheQuerySqlField sqlAnn, 
GridCacheQueryTextField txtAnn,
-        Class<?> cls, ClassProperty prop, TypeDescriptor desc) throws 
GridException {
+        Class<?> cls, ClassProperty prop, TypeDescriptor desc) throws 
IgniteCheckedException {
         if (sqlAnn != null) {
             processAnnotationsInClass(key, cls, desc, prop);
 
@@ -802,10 +803,10 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param cls Class to process.
      * @param meta Type metadata.
      * @param d Type descriptor.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     static void processClassMeta(boolean key, Class<?> cls, 
GridCacheQueryTypeMetadata meta, TypeDescriptor d)
-        throws GridException {
+        throws IgniteCheckedException {
         for (Map.Entry<String, Class<?>> entry : 
meta.getAscendingFields().entrySet()) {
             ClassProperty prop = buildClassProperty(cls, entry.getKey(), 
entry.getValue());
 
@@ -875,10 +876,10 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param key Key or value flag.
      * @param meta Declared metadata.
      * @param d Type descriptor.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     static void processPortableMeta(boolean key, GridCacheQueryTypeMetadata 
meta, TypeDescriptor d)
-        throws GridException {
+        throws IgniteCheckedException {
         for (Map.Entry<String, Class<?>> entry : 
meta.getAscendingFields().entrySet()) {
             PortableProperty prop = buildPortableProperty(entry.getKey(), 
entry.getValue());
 
@@ -967,9 +968,9 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param pathStr String representing path to the property. May contains 
dots '.' to identify nested fields.
      * @param resType Expected result type.
      * @return Property instance corresponding to the given path.
-     * @throws GridException If property cannot be created.
+     * @throws IgniteCheckedException If property cannot be created.
      */
-    static ClassProperty buildClassProperty(Class<?> cls, String pathStr, 
Class<?> resType) throws GridException {
+    static ClassProperty buildClassProperty(Class<?> cls, String pathStr, 
Class<?> resType) throws IgniteCheckedException {
         String[] path = pathStr.split("\\.");
 
         ClassProperty res = null;
@@ -991,7 +992,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
                     tmp = new ClassProperty(cls.getDeclaredField(prop));
                 }
                 catch (NoSuchFieldException ignored) {
-                    throw new GridException("Failed to find getter method or 
field for property named " +
+                    throw new IgniteCheckedException("Failed to find getter 
method or field for property named " +
                         "'" + prop + "': " + cls.getName());
                 }
             }
@@ -1004,7 +1005,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
         }
 
         if (!U.box(resType).isAssignableFrom(U.box(res.type())))
-            throw new GridException("Failed to create property for given path 
(actual property type is not assignable" +
+            throw new IgniteCheckedException("Failed to create property for 
given path (actual property type is not assignable" +
                 " to declared type [path=" + pathStr + ", actualType=" + 
res.type().getName() +
                 ", declaredType=" + resType.getName() + ']');
 
@@ -1037,13 +1038,13 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
      * @param space Space name.
      * @param typeName Type name.
      * @return Type.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public GridQueryTypeDescriptor type(@Nullable String space, String 
typeName) throws GridException {
+    public GridQueryTypeDescriptor type(@Nullable String space, String 
typeName) throws IgniteCheckedException {
         TypeDescriptor type = typesByName.get(new TypeName(space, typeName));
 
         if (type == null || !type.registered())
-            throw new GridException("Failed to find type descriptor for type 
name: " + typeName);
+            throw new IgniteCheckedException("Failed to find type descriptor 
for type name: " + typeName);
 
         return type;
     }
@@ -1051,9 +1052,9 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
     /**
      * @param cls Field type.
      * @return {@code True} if given type is a spatial geometry type based on 
{@code com.vividsolutions.jts} library.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private static boolean isGeometryClass(Class<?> cls) throws GridException 
{ // TODO optimize
+    private static boolean isGeometryClass(Class<?> cls) throws 
IgniteCheckedException { // TODO optimize
         Class<?> dataTypeCls;
 
         try {
@@ -1069,7 +1070,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
             return (Boolean)method.invoke(null, cls);
         }
         catch (Exception e) {
-            throw new GridException("Failed to invoke 
'org.h2.value.DataType.isGeometryClass' method.", e);
+            throw new IgniteCheckedException("Failed to invoke 
'org.h2.value.DataType.isGeometryClass' method.", e);
         }
     }
 
@@ -1082,9 +1083,9 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
          *
          * @param x Object with this property.
          * @return Property value.
-         * @throws GridException If failed.
+         * @throws IgniteCheckedException If failed.
          */
-        public abstract Object value(Object x) throws GridException;
+        public abstract Object value(Object x) throws IgniteCheckedException;
 
         /**
          * @return Property name.
@@ -1130,7 +1131,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public Object value(Object x) throws GridException {
+        @Override public Object value(Object x) throws IgniteCheckedException {
             if (parent != null)
                 x = parent.value(x);
 
@@ -1150,7 +1151,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
                 }
             }
             catch (Exception e) {
-                throw new GridException(e);
+                throw new IgniteCheckedException(e);
             }
         }
 
@@ -1219,7 +1220,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public Object value(Object obj) throws GridException {
+        @Override public Object value(Object obj) throws 
IgniteCheckedException {
             if (parent != null)
                 obj = parent.value(obj);
 
@@ -1227,7 +1228,7 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
                 return null;
 
             if (!(obj instanceof PortableObject))
-                throw new GridException("Non-portable object received as a 
result of property extraction " +
+                throw new IgniteCheckedException("Non-portable object received 
as a result of property extraction " +
                     "[parent=" + parent + ", propName=" + propName + ", obj=" 
+ obj + ']');
 
             return ((PortableObject)obj).field(propName);
@@ -1287,9 +1288,9 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
 
         /**
          * @param c Initialization callable.
-         * @throws GridException In case of error.
+         * @throws IgniteCheckedException In case of error.
          */
-        void init(Callable<Void> c) throws GridException {
+        void init(Callable<Void> c) throws IgniteCheckedException {
             initializer.init(c);
         }
 
@@ -1347,14 +1348,14 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public <T> T value(Object obj, String field) throws 
GridException {
+        @Override public <T> T value(Object obj, String field) throws 
IgniteCheckedException {
             assert obj != null;
             assert field != null;
 
             Property prop = props.get(field);
 
             if (prop == null)
-                throw new GridException("Failed to find field '" + field + "' 
in type '" + name + "'.");
+                throw new IgniteCheckedException("Failed to find field '" + 
field + "' in type '" + name + "'.");
 
             return (T)prop.value(obj);
         }
@@ -1370,13 +1371,13 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
          * @param idxName Index name.
          * @param type Index type.
          * @return Index descriptor.
-         * @throws GridException In case of error.
+         * @throws IgniteCheckedException In case of error.
          */
-        public IndexDescriptor addIndex(String idxName, GridQueryIndexType 
type) throws GridException {
+        public IndexDescriptor addIndex(String idxName, GridQueryIndexType 
type) throws IgniteCheckedException {
             IndexDescriptor idx = new IndexDescriptor(type);
 
             if (indexes.put(idxName, idx) != null)
-                throw new GridException("Index with name '" + idxName + "' 
already exists.");
+                throw new IgniteCheckedException("Index with name '" + idxName 
+ "' already exists.");
 
             return idx;
         }
@@ -1388,10 +1389,10 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
          * @param field Field name.
          * @param orderNum Fields order number in index.
          * @param descending Sorting order.
-         * @throws GridException If failed.
+         * @throws IgniteCheckedException If failed.
          */
         public void addFieldToIndex(String idxName, String field, int orderNum,
-            boolean descending) throws GridException {
+            boolean descending) throws IgniteCheckedException {
             IndexDescriptor desc = indexes.get(idxName);
 
             if (desc == null)
@@ -1449,13 +1450,13 @@ public class GridQueryProcessor extends 
GridProcessorAdapter {
          * @param key If given property relates to key.
          * @param prop Property.
          * @param failOnDuplicate Fail on duplicate flag.
-         * @throws GridException In case of error.
+         * @throws IgniteCheckedException In case of error.
          */
-        public void addProperty(boolean key, Property prop, boolean 
failOnDuplicate) throws GridException {
+        public void addProperty(boolean key, Property prop, boolean 
failOnDuplicate) throws IgniteCheckedException {
             String name = prop.name();
 
             if (props.put(name, prop) != null && failOnDuplicate)
-                throw new GridException("Property with name '" + name + "' 
already exists.");
+                throw new IgniteCheckedException("Property with name '" + name 
+ "' already exists.");
 
             if (key)
                 keyFields.put(name, prop.type());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java
index d835349..ae2f01a 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/query/GridQueryTypeDescriptor.java
@@ -9,8 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.query;
 
-
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 import java.util.*;
 
@@ -45,9 +44,9 @@ public interface GridQueryTypeDescriptor {
      * @param obj Object to get field value from.
      * @param field Field name.
      * @return Value for given field.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public <T> T value(Object obj, String field) throws GridException;
+    public <T> T value(Object obj, String field) throws IgniteCheckedException;
 
     /**
      * Gets indexes for this type.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java
index 9ee4734..6c795bd 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceBasicInjector.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.resource;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.kernal.managers.deployment.*;
 import org.gridgain.grid.util.typedef.internal.*;
 
@@ -41,13 +41,13 @@ class GridResourceBasicInjector<T> implements 
GridResourceInjector {
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceField field, Object target, 
Class<?> depCls, GridDeployment dep)
-        throws GridException {
+        throws IgniteCheckedException {
         GridResourceUtils.inject(field.getField(), target, rsrc);
     }
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceMethod mtd, Object target, 
Class<?> depCls, GridDeployment dep)
-        throws GridException {
+        throws IgniteCheckedException {
         GridResourceUtils.inject(mtd.getMethod(), target, rsrc);
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java
index 2da1ceb..dfd1884 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceCustomInjector.java
@@ -256,7 +256,7 @@ class GridResourceCustomInjector implements 
GridResourceInjector {
                     }
                 }
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 U.error(log, "Failed to find finalizers for resource: " + 
rsrc, e);
             }
 
@@ -283,14 +283,14 @@ class GridResourceCustomInjector implements 
GridResourceInjector {
         try {
             ioc.inject(rsrc.getResource(), annCls, nullInjector, null, null);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to clean up resource [ann=" + annCls + ", 
rsrc=" + rsrc + ']', e);
         }
     }
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceField field, Object target, 
Class<?> depCls,
-        GridDeployment dep) throws GridException {
+        GridDeployment dep) throws IgniteCheckedException {
         assert dep != null;
 
         IgniteUserResource ann = (IgniteUserResource)field.getAnnotation();
@@ -298,7 +298,7 @@ class GridResourceCustomInjector implements 
GridResourceInjector {
         assert ann != null;
 
         if (!Modifier.isTransient(field.getField().getModifiers())) {
-            throw new GridException("@GridUserResource must only be used with 
'transient' fields: " +
+            throw new IgniteCheckedException("@GridUserResource must only be 
used with 'transient' fields: " +
                 field.getField());
         }
 
@@ -312,13 +312,13 @@ class GridResourceCustomInjector implements 
GridResourceInjector {
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceMethod mtd, Object target, 
Class<?> depCls, GridDeployment dep)
-        throws GridException {
+        throws IgniteCheckedException {
         assert dep != null;
 
         IgniteUserResource ann = (IgniteUserResource)mtd.getAnnotation();
 
         if (mtd.getMethod().getParameterTypes().length != 1)
-            throw new GridException("Method injection setter must have only 
one parameter: " + mtd.getMethod());
+            throw new IgniteCheckedException("Method injection setter must 
have only one parameter: " + mtd.getMethod());
 
         Class<?> rsrcCls = !ann.resourceClass().equals(Void.class) ? 
ann.resourceClass() :
             mtd.getMethod().getParameterTypes()[0];
@@ -338,10 +338,10 @@ class GridResourceCustomInjector implements 
GridResourceInjector {
      * @param rsrcCls Resource class.
      * @param rsrcName Resource name.
      * @return Created resource.
-     * @throws GridException If resource creation failed.
+     * @throws IgniteCheckedException If resource creation failed.
      */
     private Object getResource(GridDeployment dep, Class<?> depCls, Class<?> 
rsrcCls, String rsrcName)
-        throws GridException {
+        throws IgniteCheckedException {
         assert dep != null;
 
         // For performance reasons we first try to acquire read lock and
@@ -434,9 +434,9 @@ class GridResourceCustomInjector implements 
GridResourceInjector {
      * @param dep Deployment.
      * @param depCls Deployed class.
      * @return Created object with injected resources.
-     * @throws GridException Thrown in case of any errors during injection.
+     * @throws IgniteCheckedException Thrown in case of any errors during 
injection.
      */
-    private CachedResource createResource(Class<?> rsrcCls, GridDeployment 
dep, Class<?> depCls) throws GridException {
+    private CachedResource createResource(Class<?> rsrcCls, GridDeployment 
dep, Class<?> depCls) throws IgniteCheckedException {
         assert dep != null;
 
         try {
@@ -466,13 +466,13 @@ class GridResourceCustomInjector implements 
GridResourceInjector {
             return new CachedResource(rsrc, dep.classLoader());
         }
         catch (InstantiationException e) {
-            throw new GridException("Failed to instantiate task shared 
resource: " + rsrcCls, e);
+            throw new IgniteCheckedException("Failed to instantiate task 
shared resource: " + rsrcCls, e);
         }
         catch (IllegalAccessException e) {
-            throw new GridException("Failed to access task shared resource (is 
class public?): " + rsrcCls, e);
+            throw new IgniteCheckedException("Failed to access task shared 
resource (is class public?): " + rsrcCls, e);
         }
         catch (InvocationTargetException e) {
-            throw new GridException("Failed to initialize task shared 
resource: " + rsrcCls, e);
+            throw new IgniteCheckedException("Failed to initialize task shared 
resource: " + rsrcCls, e);
         }
     }
 
@@ -482,17 +482,17 @@ class GridResourceCustomInjector implements 
GridResourceInjector {
      * @param cls Class in which search for methods.
      * @param annCls Annotation.
      * @return Set of methods with given annotations.
-     * @throws GridException Thrown in case when method contains parameters.
+     * @throws IgniteCheckedException Thrown in case when method contains 
parameters.
      */
     private List<Method> getMethodsWithAnnotation(Class<?> cls, Class<? 
extends Annotation> annCls)
-        throws GridException {
+        throws IgniteCheckedException {
         List<Method> mtds = new ArrayList<>();
 
         for (Class<?> c = cls; !c.equals(Object.class); c = c.getSuperclass()) 
{
             for (Method mtd : c.getDeclaredMethods()) {
                 if (mtd.getAnnotation(annCls) != null) {
                     if (mtd.getParameterTypes().length > 0) {
-                        throw new GridException("Task shared resource 
initialization or finalization method should " +
+                        throw new IgniteCheckedException("Task shared resource 
initialization or finalization method should " +
                             "not have parameters: " + mtd);
                     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java
index b522be7..2a895d2 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceInjector.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.resource;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.kernal.managers.deployment.*;
 
 /**
@@ -24,9 +24,9 @@ interface GridResourceInjector {
      * @param target Target object the field belongs to.
      * @param depCls Deployed class.
      * @param dep Deployment.
-     * @throws GridException If injection failed.
+     * @throws IgniteCheckedException If injection failed.
      */
-    public void inject(GridResourceField field, Object target, Class<?> 
depCls, GridDeployment dep) throws GridException;
+    public void inject(GridResourceField field, Object target, Class<?> 
depCls, GridDeployment dep) throws IgniteCheckedException;
 
     /**
      * Injects resource with a setter method. Caches injected resource with 
the given key if needed.
@@ -35,9 +35,9 @@ interface GridResourceInjector {
      * @param target Target object the field belongs to.
      * @param depCls Deployed class.
      * @param dep Deployment.
-     * @throws GridException If injection failed.
+     * @throws IgniteCheckedException If injection failed.
      */
-    public void inject(GridResourceMethod mtd, Object target, Class<?> depCls, 
GridDeployment dep) throws GridException;
+    public void inject(GridResourceMethod mtd, Object target, Class<?> depCls, 
GridDeployment dep) throws IgniteCheckedException;
 
     /**
      * Gracefully cleans all resources associated with deployment.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java
index f1a134a..ef7c508 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceIoc.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.resource;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.kernal.managers.deployment.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.typedef.*;
@@ -90,10 +90,10 @@ class GridResourceIoc {
      * @param injector Resource to inject.
      * @param dep Deployment.
      * @param depCls Deployment class.
-     * @throws GridException Thrown in case of any errors during injection.
+     * @throws IgniteCheckedException Thrown in case of any errors during 
injection.
      */
     void inject(Object target, Class<? extends Annotation> annCls, 
GridResourceInjector injector,
-        @Nullable GridDeployment dep, @Nullable Class<?> depCls) throws 
GridException {
+        @Nullable GridDeployment dep, @Nullable Class<?> depCls) throws 
IgniteCheckedException {
         assert target != null;
         assert annCls != null;
         assert injector != null;
@@ -110,10 +110,10 @@ class GridResourceIoc {
      * @param dep Deployment.
      * @param depCls Deployment class.
      * @param checkedObjs Set of already inspected objects to avoid indefinite 
recursion.
-     * @throws GridException Thrown in case of any errors during injection.
+     * @throws IgniteCheckedException Thrown in case of any errors during 
injection.
      */
     private void injectInternal(Object target, Class<? extends Annotation> 
annCls, GridResourceInjector injector,
-        @Nullable GridDeployment dep, @Nullable Class<?> depCls, Set<Object> 
checkedObjs) throws GridException {
+        @Nullable GridDeployment dep, @Nullable Class<?> depCls, Set<Object> 
checkedObjs) throws IgniteCheckedException {
         assert target != null;
         assert annCls != null;
         assert injector != null;
@@ -149,7 +149,7 @@ class GridResourceIoc {
                         injectInternal(obj, annCls, injector, dep, depCls, 
checkedObjs);
                 }
                 catch (IllegalAccessException e) {
-                    throw new GridException("Failed to inject resource 
[field=" + f.getName() +
+                    throw new IgniteCheckedException("Failed to inject 
resource [field=" + f.getName() +
                         ", target=" + target + ']', e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java
index abc2861..bdd89d8 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceJobContextInjector.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.resource;
 
+import org.apache.ignite.*;
 import org.apache.ignite.compute.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.managers.deployment.*;
 
 /**
@@ -28,7 +28,7 @@ public class GridResourceJobContextInjector extends 
GridResourceBasicInjector<Co
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceField field, Object target, 
Class<?> depCls, GridDeployment dep)
-        throws GridException {
+        throws IgniteCheckedException {
         assert target != null;
 
         if (!(target instanceof ComputeTask))
@@ -37,7 +37,7 @@ public class GridResourceJobContextInjector extends 
GridResourceBasicInjector<Co
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceMethod mtd, Object target, 
Class<?> depCls, GridDeployment dep)
-        throws GridException {
+        throws IgniteCheckedException {
         assert target != null;
 
         if (!(target instanceof ComputeTask))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java
index 7034257..b4ba0c7 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceLoggerInjector.java
@@ -27,13 +27,13 @@ public class GridResourceLoggerInjector extends 
GridResourceBasicInjector<Ignite
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceField field, Object target, 
Class<?> depCls, GridDeployment dep)
-        throws GridException {
+        throws IgniteCheckedException {
         GridResourceUtils.inject(field.getField(), target, 
resource((IgniteLoggerResource)field.getAnnotation(), target));
     }
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceMethod mtd, Object target, 
Class<?> depCls, GridDeployment dep)
-        throws GridException {
+        throws IgniteCheckedException {
         GridResourceUtils.inject(mtd.getMethod(), target, 
resource((IgniteLoggerResource)mtd.getAnnotation(), target));
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java
----------------------------------------------------------------------
diff --git 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java
 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java
index 89548f1..be23993 100644
--- 
a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java
+++ 
b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceProcessor.java
@@ -145,7 +145,7 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void start() throws GridException {
+    @Override public void start() throws IgniteCheckedException {
         customInjector = new GridResourceCustomInjector(log, ioc);
 
         customInjector.setExecutorInjector(execInjector);
@@ -199,10 +199,10 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * @param dep Deployment.
      * @param target Target object.
      * @param annCls Annotation class.
-     * @throws GridException If failed to execute annotated methods.
+     * @throws IgniteCheckedException If failed to execute annotated methods.
      */
     public void invokeAnnotated(GridDeployment dep, Object target, Class<? 
extends Annotation> annCls)
-        throws GridException {
+        throws IgniteCheckedException {
         if (target != null) {
             Collection<Method> mtds = getMethodsWithAnnotation(dep, 
target.getClass(), annCls);
 
@@ -214,7 +214,7 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
                         mtd.invoke(target);
                     }
                     catch (IllegalArgumentException | 
InvocationTargetException | IllegalAccessException e) {
-                        throw new GridException("Failed to invoke annotated 
method [job=" + target + ", mtd=" + mtd +
+                        throw new IgniteCheckedException("Failed to invoke 
annotated method [job=" + target + ", mtd=" + mtd +
                             ", ann=" + annCls + ']', e);
                     }
                 }
@@ -228,9 +228,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * @param dep Deployment.
      * @param depCls Deployed class.
      * @param target Target instance to inject into.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void inject(GridDeployment dep, Class<?> depCls, Object target) 
throws GridException {
+    public void inject(GridDeployment dep, Class<?> depCls, Object target) 
throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Injecting resources: " + target);
 
@@ -259,9 +259,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      *
      * @param obj Object.
      * @param cacheName Cache name to inject.
-     * @throws GridException If failed to inject.
+     * @throws IgniteCheckedException If failed to inject.
      */
-    public void injectCacheName(Object obj, String cacheName) throws 
GridException {
+    public void injectCacheName(Object obj, String cacheName) throws 
IgniteCheckedException {
         assert obj != null;
 
         if (log.isDebugEnabled())
@@ -275,9 +275,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
 
     /**
      * @param obj Object to inject.
-     * @throws GridException If failed to inject.
+     * @throws IgniteCheckedException If failed to inject.
      */
-    public void injectGeneric(Object obj) throws GridException {
+    public void injectGeneric(Object obj) throws IgniteCheckedException {
         assert obj != null;
 
         if (log.isDebugEnabled())
@@ -303,9 +303,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
 
     /**
      * @param obj Object.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void cleanupGeneric(Object obj) throws GridException {
+    public void cleanupGeneric(Object obj) throws IgniteCheckedException {
         if (obj != null) {
             if (log.isDebugEnabled())
                 log.debug("Cleaning up resources: " + obj);
@@ -337,10 +337,10 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * @param job Grid job to inject resources to.
      * @param ses Current task session.
      * @param jobCtx Job context.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
     public void inject(GridDeployment dep, Class<?> taskCls, ComputeJob job, 
ComputeTaskSession ses,
-        GridJobContextImpl jobCtx) throws GridException {
+        GridJobContextImpl jobCtx) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Injecting resources: " + job);
 
@@ -365,10 +365,10 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * @param job Job.
      * @param ses Session.
      * @param jobCtx Job context.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     private void injectToJob(GridDeployment dep, Class<?> taskCls, Object job, 
ComputeTaskSession ses,
-        GridJobContextImpl jobCtx) throws GridException {
+        GridJobContextImpl jobCtx) throws IgniteCheckedException {
         Class<? extends Annotation>[] filtered = ioc.filter(dep, job, 
JOB_INJECTIONS);
 
         if (filtered.length > 0) {
@@ -432,10 +432,10 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * @param ses Grid task session.
      * @param balancer Load balancer.
      * @param mapper Continuous task mapper.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
     public void inject(GridDeployment dep, ComputeTask<?, ?> task, 
GridTaskSessionImpl ses,
-        ComputeLoadBalancer balancer, ComputeTaskContinuousMapper mapper) 
throws GridException {
+        ComputeLoadBalancer balancer, ComputeTaskContinuousMapper mapper) 
throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Injecting resources: " + task);
 
@@ -504,9 +504,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * Injects held resources into given SPI implementation.
      *
      * @param spi SPI implementation.
-     * @throws GridException Throw in case of any errors.
+     * @throws IgniteCheckedException Throw in case of any errors.
      */
-    public void inject(IgniteSpi spi) throws GridException {
+    public void inject(IgniteSpi spi) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Injecting resources: " + spi);
 
@@ -533,9 +533,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * method injects {@code null}s into SPI implementation.
      *
      * @param spi SPI implementation.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void cleanup(IgniteSpi spi) throws GridException {
+    public void cleanup(IgniteSpi spi) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Cleaning up resources: " + spi);
 
@@ -560,9 +560,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * Injects held resources into given lifecycle bean.
      *
      * @param lifecycleBean Lifecycle bean.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void inject(LifecycleBean lifecycleBean) throws GridException {
+    public void inject(LifecycleBean lifecycleBean) throws 
IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Injecting resources: " + lifecycleBean);
 
@@ -589,9 +589,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * method injects {@code null}s into lifecycle bean.
      *
      * @param lifecycleBean Lifecycle bean.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void cleanup(LifecycleBean lifecycleBean) throws GridException {
+    public void cleanup(LifecycleBean lifecycleBean) throws 
IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Cleaning up resources: " + lifecycleBean);
 
@@ -617,9 +617,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * Injects resources into service.
      *
      * @param svc Service to inject.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void inject(ManagedService svc) throws GridException {
+    public void inject(ManagedService svc) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Injecting resources: " + svc);
 
@@ -646,9 +646,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * method injects {@code null}s into service bean.
      *
      * @param svc Service.
-     * @throws GridException Thrown in case of any errors.
+     * @throws IgniteCheckedException Thrown in case of any errors.
      */
-    public void cleanup(ManagedService svc) throws GridException {
+    public void cleanup(ManagedService svc) throws IgniteCheckedException {
         if (log.isDebugEnabled())
             log.debug("Cleaning up resources: " + svc);
 
@@ -680,10 +680,10 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * @param rsrc Resource to inject.
      * @param dep Deployment.
      * @param depCls Deployed class.
-     * @throws GridException If injection failed.
+     * @throws IgniteCheckedException If injection failed.
      */
     public void injectBasicResource(Object target, Class<? extends Annotation> 
annCls, Object rsrc,
-        GridDeployment dep, Class<?> depCls) throws GridException {
+        GridDeployment dep, Class<?> depCls) throws IgniteCheckedException {
         // Safety.
         assert !(rsrc instanceof GridResourceInjector) : "Invalid injection.";
 
@@ -699,10 +699,10 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      * @param target Target object.
      * @param annCls Setter annotation.
      * @param rsrc Resource to inject.
-     * @throws GridException If injection failed.
+     * @throws IgniteCheckedException If injection failed.
      */
     public void injectBasicResource(Object target, Class<? extends Annotation> 
annCls, Object rsrc)
-        throws GridException {
+        throws IgniteCheckedException {
         // Safety.
         assert !(rsrc instanceof GridResourceInjector) : "Invalid injection.";
 
@@ -762,9 +762,9 @@ public class GridResourceProcessor extends 
GridProcessorAdapter {
      *
      * @param target Target object.
      * @return Original object wrapped by proxy.
-     * @throws GridException If unwrap failed.
+     * @throws IgniteCheckedException If unwrap failed.
      */
-    private Object unwrapTarget(Object target) throws GridException {
+    private Object unwrapTarget(Object target) throws IgniteCheckedException {
         return rsrcCtx != null ? rsrcCtx.unwrapTarget(target) : target;
     }
 

Reply via email to