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

[22/51] [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/resource/GridResourceServiceInjector.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceServiceInjector.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceServiceInjector.java
index dd4daa0..c5a16db 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceServiceInjector.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceServiceInjector.java
@@ -35,7 +35,7 @@ public class GridResourceServiceInjector extends GridResourceBasicInjector<Colle
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceField field, Object target, Class<?> depCls, GridDeployment dep)
-        throws GridException {
+        throws IgniteCheckedException {
         IgniteServiceResource ann = (IgniteServiceResource)field.getAnnotation();
 
         Class svcItf = ann.proxyInterface();
@@ -53,7 +53,7 @@ public class GridResourceServiceInjector extends GridResourceBasicInjector<Colle
 
     /** {@inheritDoc} */
     @Override public void inject(GridResourceMethod mtd, Object target, Class<?> depCls, GridDeployment dep)
-        throws GridException {
+        throws IgniteCheckedException {
         IgniteServiceResource ann = (IgniteServiceResource)mtd.getAnnotation();
 
         Class svcItf = ann.proxyInterface();
@@ -68,7 +68,7 @@ public class GridResourceServiceInjector extends GridResourceBasicInjector<Colle
         Class<?>[] types = mtd.getMethod().getParameterTypes();
 
         if (types.length != 1)
-            throw new GridException("Setter does not have single parameter of required type [type=" +
+            throw new IgniteCheckedException("Setter does not have single parameter of required type [type=" +
                 svc.getClass().getName() + ", setter=" + mtd + ']');
 
         if (svc != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceUtils.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceUtils.java
index 6f0bfda..420bbc8 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceUtils.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridResourceUtils.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.resource;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 import java.lang.reflect.*;
 import java.util.concurrent.*;
@@ -32,12 +32,12 @@ final class GridResourceUtils {
      * @param field Field where resource should be injected.
      * @param target Target object.
      * @param rsrc Resource object which should be injected in target object field.
-     * @throws GridException Thrown if unable to inject resource.
+     * @throws IgniteCheckedException Thrown if unable to inject resource.
      */
     @SuppressWarnings({"ErrorNotRethrown"})
-    static void inject(Field field, Object target, Object rsrc) throws GridException {
+    static void inject(Field field, Object target, Object rsrc) throws IgniteCheckedException {
         if (rsrc != null && !field.getType().isAssignableFrom(rsrc.getClass()))
-            throw new GridException("Resource field is not assignable from the resource: " + rsrc.getClass());
+            throw new IgniteCheckedException("Resource field is not assignable from the resource: " + rsrc.getClass());
 
         try {
             // Override default Java access check.
@@ -46,7 +46,7 @@ final class GridResourceUtils {
             field.set(target, rsrc);
         }
         catch (SecurityException | ExceptionInInitializerError | IllegalAccessException e) {
-            throw new GridException("Failed to inject resource [field=" + field.getName() +
+            throw new IgniteCheckedException("Failed to inject resource [field=" + field.getName() +
                 ", target=" + target + ", rsrc=" + rsrc + ']', e);
         }
     }
@@ -59,13 +59,13 @@ final class GridResourceUtils {
      * @param mtd Method which should be invoked to inject resource.
      * @param target Target object.
      * @param rsrc Resource object which should be injected.
-     * @throws GridException Thrown if unable to inject resource.
+     * @throws IgniteCheckedException Thrown if unable to inject resource.
      */
     @SuppressWarnings({"ErrorNotRethrown"})
-    static void inject(Method mtd, Object target, Object rsrc) throws GridException {
+    static void inject(Method mtd, Object target, Object rsrc) throws IgniteCheckedException {
         if (mtd.getParameterTypes().length != 1 ||
             (rsrc != null && !mtd.getParameterTypes()[0].isAssignableFrom(rsrc.getClass()))) {
-            throw new GridException("Setter does not have single parameter of required type [type=" +
+            throw new IgniteCheckedException("Setter does not have single parameter of required type [type=" +
                 rsrc.getClass().getName() + ", setter=" + mtd + ']');
         }
 
@@ -75,7 +75,7 @@ final class GridResourceUtils {
             mtd.invoke(target, rsrc);
         }
         catch (IllegalAccessException | ExceptionInInitializerError | InvocationTargetException e) {
-            throw new GridException("Failed to inject resource [method=" + mtd.getName() +
+            throw new IgniteCheckedException("Failed to inject resource [method=" + mtd.getName() +
                 ", target=" + target + ", rsrc=" + rsrc + ']', e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridSpringResourceContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridSpringResourceContext.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridSpringResourceContext.java
index 1c1110f..bee16fb 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridSpringResourceContext.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/resource/GridSpringResourceContext.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.resource;
 
-import org.gridgain.grid.*;
-import org.gridgain.grid.kernal.GridGainEx;
+import org.apache.ignite.*;
+import org.gridgain.grid.kernal.*;
 
 /**
  * Interface was introduced to avoid compile-time dependency on spring framework. Spring resource context
@@ -33,8 +33,8 @@ public interface GridSpringResourceContext {
      *
      * @param target Target object.
      * @return Original object wrapped by proxy.
-     * @throws org.gridgain.grid.GridException If unwrap failed.
+     * @throws IgniteCheckedException If unwrap failed.
      */
-    public Object unwrapTarget(Object target) throws GridException;
+    public Object unwrapTarget(Object target) throws IgniteCheckedException;
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java
index 1f696ba..ae20c07 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProcessor.java
@@ -9,10 +9,11 @@
 
 package org.gridgain.grid.kernal.processors.rest;
 
+import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
+import org.apache.ignite.plugin.security.*;
 import org.apache.ignite.spi.authentication.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.securesession.*;
 import org.gridgain.grid.kernal.managers.security.*;
@@ -27,7 +28,6 @@ import org.gridgain.grid.kernal.processors.rest.handlers.top.*;
 import org.gridgain.grid.kernal.processors.rest.handlers.version.*;
 import org.gridgain.grid.kernal.processors.rest.protocols.tcp.*;
 import org.gridgain.grid.kernal.processors.rest.request.*;
-import org.apache.ignite.plugin.security.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.typedef.*;
@@ -39,8 +39,8 @@ import java.lang.reflect.*;
 import java.util.*;
 import java.util.concurrent.*;
 
-import static org.gridgain.grid.kernal.processors.rest.GridRestResponse.*;
 import static org.apache.ignite.plugin.security.GridSecuritySubjectType.*;
+import static org.gridgain.grid.kernal.processors.rest.GridRestResponse.*;
 
 /**
  * Rest processor implementation.
@@ -67,7 +67,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
     /** Protocol handler. */
     private final GridRestProtocolHandler protoHnd = new GridRestProtocolHandler() {
-        @Override public GridRestResponse handle(GridRestRequest req) throws GridException {
+        @Override public GridRestResponse handle(GridRestRequest req) throws IgniteCheckedException {
             return handleAsync(req).get();
         }
 
@@ -83,7 +83,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
     private IgniteFuture<GridRestResponse> handleAsync0(final GridRestRequest req) {
         if (!busyLock.tryReadLock())
             return new GridFinishedFuture<>(ctx,
-                new GridException("Failed to handle request (received request while stopping grid)."));
+                new IgniteCheckedException("Failed to handle request (received request while stopping grid)."));
 
         try {
             final GridWorkerFuture<GridRestResponse> fut = new GridWorkerFuture<>(ctx);
@@ -100,7 +100,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
                                 try {
                                     fut.onDone(f.get());
                                 }
-                                catch (GridException e) {
+                                catch (IgniteCheckedException e) {
                                     fut.onDone(e);
                                 }
                             }
@@ -148,7 +148,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
                 startLatch.await();
             }
             catch (InterruptedException e) {
-                return new GridFinishedFuture<>(ctx, new GridException("Failed to handle request " +
+                return new GridFinishedFuture<>(ctx, new IgniteCheckedException("Failed to handle request " +
                     "(protocol handler was interrupted when awaiting grid start).", e));
             }
         }
@@ -172,14 +172,14 @@ public class GridRestProcessor extends GridProcessorAdapter {
                 try {
                     res.sessionTokenBytes(updateSessionToken(req, subjCtx));
                 }
-                catch (GridException e1) {
+                catch (IgniteCheckedException e1) {
                     U.warn(log, "Cannot update response session token: " + e1.getMessage());
                 }
             }
 
             return new GridFinishedFuture<>(ctx, res);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(ctx, new GridRestResponse(STATUS_AUTH_FAILED, e.getMessage()));
         }
 
@@ -191,7 +191,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
         if (res == null)
             return new GridFinishedFuture<>(ctx,
-                new GridException("Failed to find registered handler for command: " + req.command()));
+                new IgniteCheckedException("Failed to find registered handler for command: " + req.command()));
 
         final GridSecurityContext subjCtx0 = subjCtx;
 
@@ -217,7 +217,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
                     try {
                         res.sessionTokenBytes(updateSessionToken(req, subjCtx0));
                     }
-                    catch (GridException e) {
+                    catch (IgniteCheckedException e) {
                         U.warn(log, "Cannot update response session token: " + e.getMessage());
                     }
                 }
@@ -237,7 +237,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void start() throws GridException {
+    @Override public void start() throws IgniteCheckedException {
         if (isRestEnabled()) {
             // Register handlers.
             addHandler(new GridCacheCommandHandler(ctx));
@@ -255,7 +255,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void onKernalStart() throws GridException {
+    @Override public void onKernalStart() throws IgniteCheckedException {
         if (isRestEnabled()) {
             for (GridRestProtocol proto : protos)
                 proto.onKernalStart();
@@ -299,7 +299,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
     }
 
     /** {@inheritDoc} */
-    @Override public void addAttributes(Map<String, Object> attrs)  throws GridException {
+    @Override public void addAttributes(Map<String, Object> attrs)  throws IgniteCheckedException {
         for (GridRestProtocol proto : protos) {
             Collection<IgniteBiTuple<String, Object>> props = proto.getProperties();
 
@@ -311,7 +311,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
                         continue;
 
                     if (attrs.containsKey(key))
-                        throw new GridException(
+                        throw new IgniteCheckedException(
                             "Node attribute collision for attribute [processor=GridRestProcessor, attr=" + key + ']');
 
                     attrs.put(key, p.getValue());
@@ -450,9 +450,9 @@ public class GridRestProcessor extends GridProcessorAdapter {
      *
      * @param req Request to authenticate.
      * @return Authentication subject context.
-     * @throws GridException If authentication failed.
+     * @throws IgniteCheckedException If authentication failed.
      */
-    private GridSecurityContext authenticate(GridRestRequest req) throws GridException {
+    private GridSecurityContext authenticate(GridRestRequest req) throws IgniteCheckedException {
         UUID clientId = req.clientId();
 
         byte[] sesTok = req.sessionToken();
@@ -500,9 +500,9 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
         if (subjCtx == null) {
             if (req.credentials() == null)
-                throw new GridException("Failed to authenticate remote client (secure session SPI not set?): " + req);
+                throw new IgniteCheckedException("Failed to authenticate remote client (secure session SPI not set?): " + req);
             else
-                throw new GridException("Failed to authenticate remote client (invalid credentials?): " + req);
+                throw new IgniteCheckedException("Failed to authenticate remote client (invalid credentials?): " + req);
         }
 
         return subjCtx;
@@ -514,15 +514,15 @@ public class GridRestProcessor extends GridProcessorAdapter {
      * @param req Grid est request.
      * @param subjCtx Authentication subject context.
      * @return Valid session token.
-     * @throws GridException If session token update process failed.
+     * @throws IgniteCheckedException If session token update process failed.
      */
-    private byte[] updateSessionToken(GridRestRequest req, GridSecurityContext subjCtx) throws GridException {
+    private byte[] updateSessionToken(GridRestRequest req, GridSecurityContext subjCtx) throws IgniteCheckedException {
         // Update token from request to actual state.
         byte[] sesTok = ctx.secureSession().updateSession(REMOTE_CLIENT, req.clientId(), subjCtx, null);
 
         // Validate token has been created.
         if (sesTok == null)
-            throw new GridException("Cannot create session token (is secure session SPI set?).");
+            throw new IgniteCheckedException("Cannot create session token (is secure session SPI set?).");
 
         return sesTok;
     }
@@ -626,18 +626,18 @@ public class GridRestProcessor extends GridProcessorAdapter {
     /**
      * Starts TCP protocol.
      *
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    private void startTcpProtocol() throws GridException {
+    private void startTcpProtocol() throws IgniteCheckedException {
         startProtocol(new GridTcpRestProtocol(ctx));
     }
 
     /**
      * Starts HTTP protocol if it exists on classpath.
      *
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    private void startHttpProtocol() throws GridException {
+    private void startHttpProtocol() throws IgniteCheckedException {
         try {
             Class<?> cls = Class.forName(HTTP_PROTO_CLS);
 
@@ -652,7 +652,7 @@ public class GridRestProcessor extends GridProcessorAdapter {
                 "module to classpath).");
         }
         catch (NoSuchMethodException | InvocationTargetException | InstantiationException | IllegalAccessException e) {
-            throw new GridException("Failed to initialize HTTP REST protocol.", e);
+            throw new IgniteCheckedException("Failed to initialize HTTP REST protocol.", e);
         }
     }
 
@@ -665,9 +665,9 @@ public class GridRestProcessor extends GridProcessorAdapter {
 
     /**
      * @param proto Protocol.
-     * @throws GridException If protocol initialization failed.
+     * @throws IgniteCheckedException If protocol initialization failed.
      */
-    private void startProtocol(GridRestProtocol proto) throws GridException {
+    private void startProtocol(GridRestProtocol proto) throws IgniteCheckedException {
         assert proto != null;
         assert !protos.contains(proto);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocol.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocol.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocol.java
index 3299178..45c6df3 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocol.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocol.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.rest;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 
 import java.util.*;
 
@@ -35,9 +35,9 @@ public interface GridRestProtocol {
      * Starts protocol.
      *
      * @param hnd Command handler.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public abstract void start(GridRestProtocolHandler hnd) throws GridException;
+    public abstract void start(GridRestProtocolHandler hnd) throws IgniteCheckedException;
 
     /**
      * Grid start callback.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocolHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocolHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocolHandler.java
index f53de29..e05a6b8 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocolHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/GridRestProtocolHandler.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.rest;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.processors.rest.request.*;
 
 /**
@@ -20,9 +20,9 @@ public interface GridRestProtocolHandler {
     /**
      * @param req Request.
      * @return Response.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    public GridRestResponse handle(GridRestRequest req) throws GridException;
+    public GridRestResponse handle(GridRestRequest req) throws IgniteCheckedException;
 
     /**
      * @param req Request.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandler.java
index b4caacb..1bab28a 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandler.java
@@ -139,7 +139,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
             GridRestCommand cmd = req0.command();
 
             if (key == null && KEY_REQUIRED_REQUESTS.contains(cmd))
-                throw new GridException(GridRestCommandHandlerAdapter.missingParameter("key"));
+                throw new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("key"));
 
             final Long ttl = req0.ttl();
 
@@ -157,7 +157,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                     Set<Object> keys = req0.values().keySet();
 
                     if (F.isEmpty(keys))
-                        throw new GridException(GridRestCommandHandlerAdapter.missingParameter("keys"));
+                        throw new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("keys"));
 
                     // HashSet wrapping for correct serialization
                     keys = new HashSet<>(keys);
@@ -172,7 +172,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                     final Object val = req0.value();
 
                     if (val == null)
-                        throw new GridException(GridRestCommandHandlerAdapter.missingParameter("val"));
+                        throw new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("val"));
 
                     fut = executeCommand(req.destinationId(), req.clientId(), cacheName, flags, key, new
                         PutCommand(key, ttl, val), req.portableMode());
@@ -184,7 +184,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                     final Object val = req0.value();
 
                     if (val == null)
-                        throw new GridException(GridRestCommandHandlerAdapter.missingParameter("val"));
+                        throw new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("val"));
 
                     fut = executeCommand(req.destinationId(), req.clientId(), cacheName, flags, key,
                         new AddCommand(key, ttl, val), req.portableMode());
@@ -196,14 +196,14 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                     Map<Object, Object> map = req0.values();
 
                     if (F.isEmpty(map))
-                        throw new GridException(GridRestCommandHandlerAdapter.missingParameter("values"));
+                        throw new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("values"));
 
                     for (Map.Entry<Object, Object> e : map.entrySet()) {
                         if (e.getKey() == null)
-                            throw new GridException("Failing putAll operation (null keys are not allowed).");
+                            throw new IgniteCheckedException("Failing putAll operation (null keys are not allowed).");
 
                         if (e.getValue() == null)
-                            throw new GridException("Failing putAll operation (null values are not allowed).");
+                            throw new IgniteCheckedException("Failing putAll operation (null values are not allowed).");
                     }
 
                     // HashMap wrapping for correct serialization
@@ -238,7 +238,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
                     final Object val = req0.value();
 
                     if (val == null)
-                        throw new GridException(GridRestCommandHandlerAdapter.missingParameter("val"));
+                        throw new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("val"));
 
                     fut = executeCommand(req.destinationId(), req.clientId(), cacheName, flags, key,
                         new ReplaceCommand(key, ttl, val), req.portableMode());
@@ -296,12 +296,12 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
             return fut;
         }
-        catch (GridRuntimeException e) {
+        catch (IgniteException e) {
             U.error(log, "Failed to execute cache command: " + req, e);
 
             return new GridFinishedFuture<>(ctx, e);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to execute cache command: " + req, e);
 
             return new GridFinishedFuture<>(ctx, e);
@@ -325,7 +325,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @param op Operation to perform.
      * @param keepPortable Keep portable flag.
      * @return Operation result in future.
-     * @throws GridException If failed
+     * @throws IgniteCheckedException If failed
      */
     private IgniteFuture<GridRestResponse> executeCommand(
         @Nullable UUID destId,
@@ -334,7 +334,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         final GridCacheFlag[] flags,
         final Object key,
         final CacheProjectionCommand op,
-        final boolean keepPortable) throws GridException {
+        final boolean keepPortable) throws IgniteCheckedException {
 
         final boolean locExec =
             destId == null || destId.equals(ctx.localNodeId()) || replicatedCacheAvailable(cacheName);
@@ -370,14 +370,14 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @param key Key to set affinity mapping in the response.
      * @param op Operation to perform.
      * @return Operation result in future.
-     * @throws GridException If failed
+     * @throws IgniteCheckedException If failed
      */
     private IgniteFuture<GridRestResponse> executeCommand(
         @Nullable UUID destId,
         UUID clientId,
         final String cacheName,
         final Object key,
-        final CacheCommand op) throws GridException {
+        final CacheCommand op) throws IgniteCheckedException {
         final boolean locExec = destId == null || destId.equals(ctx.localNodeId()) ||
             ctx.cache().cache(cacheName) != null;
 
@@ -405,10 +405,10 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @param req Request.
      * @param decr Whether to decrement (increment otherwise).
      * @return Future of operation result.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     private static IgniteFuture<?> incrementOrDecrement(GridCacheProjection<Object, Object> cache, String key,
-        GridRestCacheRequest req, final boolean decr) throws GridException {
+        GridRestCacheRequest req, final boolean decr) throws IgniteCheckedException {
         assert cache != null;
         assert key != null;
         assert req != null;
@@ -417,7 +417,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         Long delta = req.delta();
 
         if (delta == null)
-            throw new GridException(GridRestCommandHandlerAdapter.missingParameter("delta"));
+            throw new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("delta"));
 
         final GridCacheAtomicLong l = cache.cache().dataStructures().atomicLong(key, init != null ? init : 0, true);
 
@@ -439,12 +439,12 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @param req Request.
      * @param prepend Whether to prepend.
      * @return Future of operation result.
-     * @throws GridException In case of any exception.
+     * @throws IgniteCheckedException In case of any exception.
      */
     private static IgniteFuture<?> appendOrPrepend(
         final GridKernalContext ctx,
         final GridCacheProjection<Object, Object> cache,
-        final Object key, GridRestCacheRequest req, final boolean prepend) throws GridException {
+        final Object key, GridRestCacheRequest req, final boolean prepend) throws IgniteCheckedException {
         assert cache != null;
         assert key != null;
         assert req != null;
@@ -452,7 +452,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         final Object val = req.value();
 
         if (val == null)
-            throw new GridException(GridRestCommandHandlerAdapter.missingParameter("val"));
+            throw new IgniteCheckedException(GridRestCommandHandlerAdapter.missingParameter("val"));
 
         return ctx.closure().callLocalSafe(new Callable<Object>() {
             @Override public Object call() throws Exception {
@@ -483,9 +483,9 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @param appendVal Appendix value to add to the original one.
      * @param appendPlc Append or prepend policy flag.
      * @return Resulting value.
-     * @throws GridException In case of grid exceptions.
+     * @throws IgniteCheckedException In case of grid exceptions.
      */
-    private static Object appendOrPrepend(Object origVal, Object appendVal, boolean appendPlc) throws GridException {
+    private static Object appendOrPrepend(Object origVal, Object appendVal, boolean appendPlc) throws IgniteCheckedException {
         // Strings.
         if (appendVal instanceof String && origVal instanceof String)
             return appendPlc ? origVal + (String)appendVal : (String)appendVal + origVal;
@@ -530,7 +530,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
             return col;
         }
 
-        throw new GridException("Incompatible types [appendVal=" + appendVal + ", old=" + origVal + ']');
+        throw new IgniteCheckedException("Incompatible types [appendVal=" + appendVal + ", old=" + origVal + ']');
     }
 
     /** {@inheritDoc} */
@@ -548,7 +548,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
     private static IgniteClosure<IgniteFuture<?>, GridRestResponse> resultWrapper(
         final GridCacheProjection<Object, Object> c, @Nullable final Object key) {
         return new CX1<IgniteFuture<?>, GridRestResponse>() {
-            @Override public GridRestResponse applyx(IgniteFuture<?> f) throws GridException {
+            @Override public GridRestResponse applyx(IgniteFuture<?> f) throws IgniteCheckedException {
                 GridCacheRestResponse resp = new GridCacheRestResponse();
 
                 resp.setResponse(f.get());
@@ -576,13 +576,13 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      *
      * @param cacheName Name of the cache.
      * @return Instance on the named cache.
-     * @throws GridException If cache not found.
+     * @throws IgniteCheckedException If cache not found.
      */
-    protected GridCacheProjectionEx<Object, Object> localCache(String cacheName) throws GridException {
+    protected GridCacheProjectionEx<Object, Object> localCache(String cacheName) throws IgniteCheckedException {
         GridCacheProjectionEx<Object, Object> cache = (GridCacheProjectionEx<Object, Object>)ctx.cache().cache(cacheName);
 
         if (cache == null)
-            throw new GridException(
+            throw new IgniteCheckedException(
                 "Failed to find cache for given cache name (null for default cache): " + cacheName);
 
         return cache;
@@ -592,13 +592,13 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
      * @param ignite Grid instance.
      * @param cacheName Name of the cache.
      * @return Instance on the named cache.
-     * @throws GridException If cache not found.
+     * @throws IgniteCheckedException If cache not found.
      */
-    private static GridCacheProjectionEx<Object, Object> cache(Ignite ignite, String cacheName) throws GridException {
+    private static GridCacheProjectionEx<Object, Object> cache(Ignite ignite, String cacheName) throws IgniteCheckedException {
         GridCache<Object, Object> cache = ignite.cache(cacheName);
 
         if (cache == null)
-            throw new GridException(
+            throw new IgniteCheckedException(
                 "Failed to find cache for given cache name (null for default cache): " + cacheName);
 
         return (GridCacheProjectionEx<Object, Object>)cache;
@@ -622,7 +622,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public Object applyx(IgniteFuture<?> f) throws GridException {
+        @Override public Object applyx(IgniteFuture<?> f) throws IgniteCheckedException {
             f.get();
 
             return res;
@@ -1047,7 +1047,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         /** {@inheritDoc} */
         @Override public IgniteFuture<?> applyx(GridCacheProjection<Object, Object> c, GridKernalContext ctx)
-            throws GridException {
+            throws IgniteCheckedException {
             return incrementOrDecrement(c, (String)key, req, false);
         }
     }
@@ -1073,7 +1073,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public IgniteFuture<?> applyx(GridCacheProjection<Object, Object> c, GridKernalContext ctx) throws GridException {
+        @Override public IgniteFuture<?> applyx(GridCacheProjection<Object, Object> c, GridKernalContext ctx) throws IgniteCheckedException {
             return incrementOrDecrement(c, (String)key, req, true);
         }
     }
@@ -1100,7 +1100,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         /** {@inheritDoc} */
         @Override public IgniteFuture<?> applyx(GridCacheProjection<Object, Object> c, GridKernalContext ctx)
-            throws GridException {
+            throws IgniteCheckedException {
             return appendOrPrepend(ctx, c, key, req, false);
         }
     }
@@ -1127,7 +1127,7 @@ public class GridCacheCommandHandler extends GridRestCommandHandlerAdapter {
 
         /** {@inheritDoc} */
         @Override public IgniteFuture<?> applyx(GridCacheProjection<Object, Object> c, GridKernalContext ctx)
-            throws GridException {
+            throws IgniteCheckedException {
             return appendOrPrepend(ctx, c, key, req, true);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
index 9bf8cdf..3303b41 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
@@ -85,7 +85,7 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
             }
 
             default:
-                return new GridFinishedFutureEx<>(new GridException("Unsupported query command: " + req.command()));
+                return new GridFinishedFutureEx<>(new IgniteCheckedException("Unsupported query command: " + req.command()));
         }
     }
 
@@ -114,7 +114,7 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
             return ctx.closure().callLocalSafe(c, false);
         else {
             if (ctx.discovery().node(destId) == null)
-                return new GridFinishedFutureEx<>(new GridException("Destination node ID has left the grid (retry " +
+                return new GridFinishedFutureEx<>(new IgniteCheckedException("Destination node ID has left the grid (retry " +
                     "the query): " + destId));
 
             try {
@@ -124,7 +124,7 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
 
                 return comp.future();
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 // Should not be thrown since uses asynchronous execution.
                 return new GridFinishedFutureEx<>(e);
             }
@@ -151,13 +151,13 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
 
                         return new GridRestResponse();
                     }
-                    catch (GridException e) {
+                    catch (IgniteCheckedException e) {
                         throw new GridClosureException(e);
                     }
                 }
             });
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             // Should not be thrown since uses asynchronous execution.
             return new GridFinishedFutureEx<>(e);
         }
@@ -169,16 +169,16 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
      * @param locMap Queries map.
      * @param locNodeId Local node ID.
      * @return Rest response.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     private static GridRestResponse fetchQueryResults(
         long qryId,
         QueryFutureWrapper wrapper,
         ConcurrentMap<QueryExecutionKey, QueryFutureWrapper> locMap,
         UUID locNodeId
-    ) throws GridException {
+    ) throws IgniteCheckedException {
         if (wrapper == null)
-            throw new GridException("Failed to find query future (query has been expired).");
+            throw new IgniteCheckedException("Failed to find query future (query has been expired).");
 
         GridCacheQueryFutureAdapter<?, ?, ?> fut = wrapper.future();
 
@@ -211,14 +211,14 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
      * @param cls Target class.
      * @param clsName Implementing class name.
      * @return Class instance.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private static <T> T instance(Class<? extends T> cls, String clsName) throws GridException {
+    private static <T> T instance(Class<? extends T> cls, String clsName) throws IgniteCheckedException {
         try {
             Class<?> implCls = Class.forName(clsName);
 
             if (!cls.isAssignableFrom(implCls))
-                throw new GridException("Failed to create instance (target class does not extend or implement " +
+                throw new IgniteCheckedException("Failed to create instance (target class does not extend or implement " +
                     "required class or interface) [cls=" + cls.getName() + ", clsName=" + clsName + ']');
 
             Constructor<?> ctor = implCls.getConstructor();
@@ -226,22 +226,22 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
             return (T)ctor.newInstance();
         }
         catch (ClassNotFoundException e) {
-            throw new GridException("Failed to find target class: " + clsName, e);
+            throw new IgniteCheckedException("Failed to find target class: " + clsName, e);
         }
         catch (NoSuchMethodException e) {
-            throw new GridException("Failed to find constructor for provided arguments " +
+            throw new IgniteCheckedException("Failed to find constructor for provided arguments " +
                 "[clsName=" + clsName + ']', e);
         }
         catch (InstantiationException e) {
-            throw new GridException("Failed to instantiate target class " +
+            throw new IgniteCheckedException("Failed to instantiate target class " +
                 "[clsName=" + clsName + ']', e);
         }
         catch (IllegalAccessException e) {
-            throw new GridException("Failed to instantiate class (constructor is not available) " +
+            throw new IgniteCheckedException("Failed to instantiate class (constructor is not available) " +
                 "[clsName=" + clsName + ']', e);
         }
         catch (InvocationTargetException e) {
-            throw new GridException("Failed to instantiate class (constructor threw an exception) " +
+            throw new IgniteCheckedException("Failed to instantiate class (constructor threw an exception) " +
                 "[clsName=" + clsName + ']', e.getCause());
         }
     }
@@ -298,7 +298,7 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
                     break;
 
                 default:
-                    throw new GridException("Unsupported query type: " + req.type());
+                    throw new IgniteCheckedException("Unsupported query type: " + req.type());
             }
 
             boolean keepPortable = req.keepPortable();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/log/GridLogCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/log/GridLogCommandHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/log/GridLogCommandHandler.java
index 545f283..e8ecf93 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/log/GridLogCommandHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/log/GridLogCommandHandler.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.rest.handlers.log;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.rest.*;
 import org.gridgain.grid.kernal.processors.rest.handlers.*;
@@ -92,7 +92,7 @@ public class GridLogCommandHandler extends GridRestCommandHandlerAdapter {
         try {
             return new GridFinishedFuture<>(ctx, new GridRestResponse(readLog(path, from, to)));
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             return new GridFinishedFuture<>(ctx, e);
         }
         catch (IOException e) {
@@ -107,17 +107,17 @@ public class GridLogCommandHandler extends GridRestCommandHandlerAdapter {
      * @param from Number of line to start from.
      * @param to Number tof line to finish on.
      * @return List of read lines.
-     * @throws GridException If argumets are illegal.
+     * @throws IgniteCheckedException If argumets are illegal.
      * @throws IOException If file couldn't be accessed or read failed.
      */
-    private List<String> readLog(String path, int from, int to) throws GridException, IOException {
+    private List<String> readLog(String path, int from, int to) throws IgniteCheckedException, IOException {
         URL url = U.resolveGridGainUrl(path);
 
         if (url == null)
-            throw new GridException("Log file not found: " + path);
+            throw new IgniteCheckedException("Log file not found: " + path);
 
         if (!isAccessible(url))
-            throw new GridException("File is not accessible through REST" +
+            throw new IgniteCheckedException("File is not accessible through REST" +
                 " (check restAccessibleFolders configuration property): " + path);
 
         if (from >= 0 && to >= 0)
@@ -125,7 +125,7 @@ public class GridLogCommandHandler extends GridRestCommandHandlerAdapter {
         else if (from < 0 && to < 0)
             return readLinesBackward(url, from, to);
         else
-            throw new GridException(
+            throw new IgniteCheckedException(
                 "Illegal arguments (both should be positive or negative) [from=" + from + ", to=" + to + ']');
     }
 
@@ -136,15 +136,15 @@ public class GridLogCommandHandler extends GridRestCommandHandlerAdapter {
      * @param from Number of line to start from. Should be negative, representing number of line from the end.
      * @param to Number tof line to finish on. Should be negative, representing number of line from the end.
      * @return List of read lines.
-     * @throws GridException If arguments are illegal.
+     * @throws IgniteCheckedException If arguments are illegal.
      * @throws IOException If file couldn't be accessed or read failed.
      */
     @SuppressWarnings("TooBroadScope")
-    private List<String> readLinesBackward(URL url, final int from, final int to) throws GridException, IOException {
+    private List<String> readLinesBackward(URL url, final int from, final int to) throws IgniteCheckedException, IOException {
         File file = new File(url.getFile());
 
         if (!file.exists() || !file.isFile())
-            throw new GridException("File doesn't exists: " + url);
+            throw new IgniteCheckedException("File doesn't exists: " + url);
 
         int linesToRead = to - from + 1;
         int linesRead = 0;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java
index f1825dd..8e547b1 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/metadata/GridPortableMetadataHandler.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.rest.handlers.metadata;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.portables.*;
 import org.gridgain.grid.*;
@@ -84,7 +85,7 @@ public class GridPortableMetadataHandler extends GridRestCommandHandlerAdapter {
                 return new GridFinishedFuture<>(ctx, res);
             }
         }
-        catch (GridRuntimeException e) {
+        catch (IgniteException e) {
             return new GridFinishedFuture<>(ctx, e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
index bd54401..77cf114 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
@@ -108,7 +108,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
 
                     ctx.io().send(nodeId, topic, res, SYSTEM_POOL);
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(log, "Failed to send job task result response.", e);
                 }
             }
@@ -125,7 +125,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
         try {
             return handleAsyncUnsafe(req);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to execute task command: " + req, e);
 
             return new GridFinishedFuture<>(ctx, e);
@@ -139,9 +139,9 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
     /**
      * @param req Request.
      * @return Future.
-     * @throws GridException On any handling exception.
+     * @throws IgniteCheckedException On any handling exception.
      */
-    private IgniteFuture<GridRestResponse> handleAsyncUnsafe(final GridRestRequest req) throws GridException {
+    private IgniteFuture<GridRestResponse> handleAsyncUnsafe(final GridRestRequest req) throws IgniteCheckedException {
         assert req instanceof GridRestTaskRequest : "Invalid command for topology handler: " + req;
 
         assert SUPPORTED_COMMANDS.contains(req.command());
@@ -170,7 +170,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                 final String name = req0.taskName();
 
                 if (F.isEmpty(name))
-                    throw new GridException(missingParameter("name"));
+                    throw new IgniteCheckedException(missingParameter("name"));
 
                 final List<Object> params = req0.params();
 
@@ -227,7 +227,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                             try {
                                 desc = new TaskDescriptor(true, f.get(), null);
                             }
-                            catch (GridException e) {
+                            catch (IgniteCheckedException e) {
                                 if (e.hasCause(ClusterTopologyException.class, ClusterGroupEmptyException.class))
                                     U.warn(log, "Failed to execute task due to topology issues (are all mapped " +
                                         "nodes alive?) [name=" + name + ", clientId=" + req.clientId() +
@@ -258,7 +258,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                                         fut.onDone(res);
                                     }
                                     catch (PortableException e) {
-                                        fut.onDone(new GridException("Failed to marshal task result: " +
+                                        fut.onDone(new IgniteCheckedException("Failed to marshal task result: " +
                                             desc.result(), e));
                                     }
                                 }
@@ -268,7 +268,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                         }
                         finally {
                             if (!async && !fut.isDone())
-                                fut.onDone(new GridException("Failed to execute task (see server logs for details)."));
+                                fut.onDone(new IgniteCheckedException("Failed to execute task (see server logs for details)."));
                         }
                     }
                 });
@@ -280,12 +280,12 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                 String id = req0.taskId();
 
                 if (F.isEmpty(id))
-                    throw new GridException(missingParameter("id"));
+                    throw new IgniteCheckedException(missingParameter("id"));
 
                 StringTokenizer st = new StringTokenizer(id, "~");
 
                 if (st.countTokens() != 2)
-                    throw new GridException("Failed to parse id parameter: " + id);
+                    throw new IgniteCheckedException("Failed to parse id parameter: " + id);
 
                 String tidParam = st.nextToken();
                 String resHolderIdParam = st.nextToken();
@@ -298,19 +298,19 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                     UUID resHolderId = !F.isEmpty(resHolderIdParam) ? UUID.fromString(resHolderIdParam) : null;
 
                     if (tid == null || resHolderId == null)
-                        throw new GridException("Failed to parse id parameter: " + id);
+                        throw new IgniteCheckedException("Failed to parse id parameter: " + id);
 
                     if (ctx.localNodeId().equals(resHolderId)) {
                         TaskDescriptor desc = taskDescs.get(tid);
 
                         if (desc == null)
-                            throw new GridException("Task with provided id has never been started on provided node" +
+                            throw new IgniteCheckedException("Task with provided id has never been started on provided node" +
                                 " [taskId=" + tidParam + ", taskResHolderId=" + resHolderIdParam + ']');
 
                         taskRestRes.setFinished(desc.finished());
 
                         if (desc.error() != null)
-                            throw new GridException(desc.error().getMessage());
+                            throw new IgniteCheckedException(desc.error().getMessage());
 
                         taskRestRes.setResult(desc.result());
 
@@ -320,20 +320,20 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                         IgniteBiTuple<String, GridTaskResultResponse> t = requestTaskResult(resHolderId, tid);
 
                         if (t.get1() != null)
-                            throw new GridException(t.get1());
+                            throw new IgniteCheckedException(t.get1());
 
                         GridTaskResultResponse taskRes = t.get2();
 
                         assert taskRes != null;
 
                         if (!taskRes.found())
-                            throw new GridException("Task with provided id has never been started on provided node " +
+                            throw new IgniteCheckedException("Task with provided id has never been started on provided node " +
                                 "[taskId=" + tidParam + ", taskResHolderId=" + resHolderIdParam + ']');
 
                         taskRestRes.setFinished(taskRes.finished());
 
                         if (taskRes.error() != null)
-                            throw new GridException(taskRes.error());
+                            throw new IgniteCheckedException(taskRes.error());
 
                         taskRestRes.setResult(taskRes.result());
 
@@ -347,7 +347,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                     if (log.isDebugEnabled())
                         log.debug(msg);
 
-                    throw new GridException(msg, e);
+                    throw new IgniteCheckedException(msg, e);
                 }
 
                 fut.onDone(res);
@@ -405,7 +405,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                 try {
                     res.result(ctx.config().getMarshaller().unmarshal(res.resultBytes(), null));
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(log, "Failed to unmarshal task result: " + res, e);
                 }
 
@@ -460,7 +460,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
 
                 ctx.io().send(taskNode, TOPIC_REST, new GridTaskResultRequest(taskId, topic, topicBytes), SYSTEM_POOL);
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 String errMsg = "Failed to send task result request [resHolderId=" + resHolderId +
                     ", err=" + e.getMessage() + ']';
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/top/GridTopologyCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/top/GridTopologyCommandHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/top/GridTopologyCommandHandler.java
index e93ac90..4116368 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/top/GridTopologyCommandHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/top/GridTopologyCommandHandler.java
@@ -9,10 +9,10 @@
 
 package org.gridgain.grid.kernal.processors.rest.handlers.top;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.affinity.consistenthash.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.cache.*;
@@ -88,7 +88,7 @@ public class GridTopologyCommandHandler extends GridRestCommandHandlerAdapter {
                 final String ip = req0.nodeIp();
 
                 if (id == null && ip == null)
-                    return new GridFinishedFuture<>(ctx, new GridException(
+                    return new GridFinishedFuture<>(ctx, new IgniteCheckedException(
                         "Failed to handle request (either id or ip should be specified)."));
 
                 ClusterNode node;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/GridRestProtocolAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/GridRestProtocolAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/GridRestProtocolAdapter.java
index 48ff406..d624f2f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/GridRestProtocolAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/GridRestProtocolAdapter.java
@@ -127,11 +127,11 @@ public abstract class GridRestProtocolAdapter implements GridRestProtocol {
     /**
      * @param cond Condition to check.
      * @param condDesc Error message.
-     * @throws GridException If check failed.
+     * @throws IgniteCheckedException If check failed.
      */
-    protected final void assertParameter(boolean cond, String condDesc) throws GridException {
+    protected final void assertParameter(boolean cond, String condDesc) throws IgniteCheckedException {
         if (!cond)
-            throw new GridException("REST protocol parameter failed condition check: " + condDesc);
+            throw new IgniteCheckedException("REST protocol parameter failed condition check: " + condDesc);
     }
 
     /**
@@ -156,7 +156,7 @@ public abstract class GridRestProtocolAdapter implements GridRestProtocol {
                 ) :
                 Collections.<IgniteBiTuple<String, Object>>emptyList();
         }
-        catch (GridException | IOException ignored) {
+        catch (IgniteCheckedException | IOException ignored) {
             return null;
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMemcachedMessageWrapper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMemcachedMessageWrapper.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMemcachedMessageWrapper.java
index 32d00a3..40c0331 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMemcachedMessageWrapper.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridMemcachedMessageWrapper.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.rest.protocols.tcp;
 
+import org.apache.ignite.*;
 import org.apache.ignite.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -47,9 +47,9 @@ public class GridMemcachedMessageWrapper extends GridTcpCommunicationMessageAdap
     /**
      * @param msg Message.
      * @param jdkMarshaller JDK marshaller.
-     * @throws GridException If failed to marshal.
+     * @throws IgniteCheckedException If failed to marshal.
      */
-    public GridMemcachedMessageWrapper(GridMemcachedMessage msg, IgniteMarshaller jdkMarshaller) throws GridException {
+    public GridMemcachedMessageWrapper(GridMemcachedMessage msg, IgniteMarshaller jdkMarshaller) throws IgniteCheckedException {
         bytes = encodeMemcache(msg, jdkMarshaller);
     }
 
@@ -109,9 +109,9 @@ public class GridMemcachedMessageWrapper extends GridTcpCommunicationMessageAdap
      * @param msg Message being serialized.
      * @param jdkMarshaller JDK marshaller.
      * @return Serialized message.
-     * @throws org.gridgain.grid.GridException If serialization failed.
+     * @throws IgniteCheckedException If serialization failed.
      */
-    private byte[] encodeMemcache(GridMemcachedMessage msg, IgniteMarshaller jdkMarshaller) throws GridException {
+    private byte[] encodeMemcache(GridMemcachedMessage msg, IgniteMarshaller jdkMarshaller) throws IgniteCheckedException {
         GridByteArrayList res = new GridByteArrayList(HDR_LEN - 1);
 
         int keyLen = 0;
@@ -193,9 +193,9 @@ public class GridMemcachedMessageWrapper extends GridTcpCommunicationMessageAdap
      * @param out Output stream to which object should be written.
      * @param jdkMarshaller JDK marshaller.
      * @return Serialization flags.
-     * @throws GridException If JDK serialization failed.
+     * @throws IgniteCheckedException If JDK serialization failed.
      */
-    private int encodeObj(Object obj, ByteArrayOutputStream out, IgniteMarshaller jdkMarshaller) throws GridException {
+    private int encodeObj(Object obj, ByteArrayOutputStream out, IgniteMarshaller jdkMarshaller) throws IgniteCheckedException {
         int flags = 0;
 
         byte[] data = null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
index 3c2c8a0..7b28af5 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpMemcachedNioListener.java
@@ -161,7 +161,7 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
         IgniteFuture<GridRestResponse> f = hnd.handleAsync(createRestRequest(req, cmd.get1()));
 
         f.listenAsync(new CIX1<IgniteFuture<GridRestResponse>>() {
-            @Override public void applyx(IgniteFuture<GridRestResponse> f) throws GridException {
+            @Override public void applyx(IgniteFuture<GridRestResponse> f) throws IgniteCheckedException {
                 GridRestResponse restRes = f.get();
 
                 // Handle 'Stat' command (special case because several packets are included in response).
@@ -243,7 +243,7 @@ public class GridTcpMemcachedNioListener extends GridNioServerListenerAdapter<Gr
 
             return ses.send(wrapper);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to marshal response: " + res, e);
 
             ses.close();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestDirectParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestDirectParser.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestDirectParser.java
index 841759b..d42c762 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestDirectParser.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestDirectParser.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.rest.protocols.tcp;
 
+import org.apache.ignite.*;
 import org.gridgain.client.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.processors.rest.client.message.*;
 import org.gridgain.grid.util.direct.*;
 import org.gridgain.grid.util.nio.*;
@@ -51,7 +51,7 @@ public class GridTcpRestDirectParser implements GridNioParser {
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public Object decode(GridNioSession ses, ByteBuffer buf) throws IOException, GridException {
+    @Nullable @Override public Object decode(GridNioSession ses, ByteBuffer buf) throws IOException, IgniteCheckedException {
         ParserState state = ses.removeMeta(PARSER_STATE.ordinal());
 
         if (state != null) {
@@ -139,7 +139,7 @@ public class GridTcpRestDirectParser implements GridNioParser {
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer encode(GridNioSession ses, Object msg) throws IOException, GridException {
+    @Override public ByteBuffer encode(GridNioSession ses, Object msg) throws IOException, IgniteCheckedException {
         // No encoding needed for direct messages.
         throw new UnsupportedEncodingException();
     }
@@ -152,10 +152,10 @@ public class GridTcpRestDirectParser implements GridNioParser {
      * @param state Current parser state.
      * @return Parsed packet.s
      * @throws IOException If packet cannot be parsed.
-     * @throws GridException If deserialization error occurred.
+     * @throws IgniteCheckedException If deserialization error occurred.
      */
     @Nullable private GridClientMessage parseMemcachePacket(GridNioSession ses, ByteBuffer buf, ParserState state)
-        throws IOException, GridException {
+        throws IOException, IgniteCheckedException {
         assert state.packetType() == GridClientPacketType.MEMCACHE;
         assert state.packet() != null;
         assert state.packet() instanceof GridMemcachedMessage;
@@ -249,9 +249,9 @@ public class GridTcpRestDirectParser implements GridNioParser {
      * @param req Raw packet.
      * @return Same packet with fields deserialized.
      * @throws IOException If parsing failed.
-     * @throws GridException If deserialization failed.
+     * @throws IgniteCheckedException If deserialization failed.
      */
-    private GridClientMessage assemble(GridNioSession ses, GridMemcachedMessage req) throws IOException, GridException {
+    private GridClientMessage assemble(GridNioSession ses, GridMemcachedMessage req) throws IOException, IgniteCheckedException {
         byte[] extras = req.extras();
 
         // First, decode key and value, if any
@@ -341,9 +341,9 @@ public class GridTcpRestDirectParser implements GridNioParser {
      * @param flags Flags.
      * @param bytes Byte array to decode.
      * @return Decoded value.
-     * @throws GridException If deserialization failed.
+     * @throws IgniteCheckedException If deserialization failed.
      */
-    private Object decodeObj(short flags, byte[] bytes) throws GridException {
+    private Object decodeObj(short flags, byte[] bytes) throws IgniteCheckedException {
         assert bytes != null;
 
         if ((flags & SERIALIZED_FLAG) != 0)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestNioListener.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
index 987a4cb..f80ab41 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestNioListener.java
@@ -192,7 +192,7 @@ public class GridTcpRestNioListener extends GridNioServerListenerAdapter<GridCli
 
                                 res.result(o);
                             }
-                            catch (GridException e) {
+                            catch (IgniteCheckedException e) {
                                 U.error(log, "Failed to process client request: " + msg, e);
 
                                 res.successStatus(GridClientResponse.STATUS_FAILED);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestParser.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestParser.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestParser.java
index 3f3719c..1364923 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestParser.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestParser.java
@@ -8,10 +8,10 @@
  */
 package org.gridgain.grid.kernal.processors.rest.protocols.tcp;
 
+import org.apache.ignite.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
 import org.gridgain.client.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.processors.rest.client.message.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.nio.*;
@@ -38,7 +38,7 @@ public class GridTcpRestParser implements GridNioParser {
 
     /** {@inheritDoc} */
     @Nullable @Override public GridClientMessage decode(GridNioSession ses, ByteBuffer buf) throws IOException,
-        GridException {
+        IgniteCheckedException {
         ParserState state = ses.removeMeta(PARSER_STATE.ordinal());
 
         if (state == null)
@@ -105,7 +105,7 @@ public class GridTcpRestParser implements GridNioParser {
     }
 
     /** {@inheritDoc} */
-    @Override public ByteBuffer encode(GridNioSession ses, Object msg0) throws IOException, GridException {
+    @Override public ByteBuffer encode(GridNioSession ses, Object msg0) throws IOException, IgniteCheckedException {
         assert msg0 != null;
 
         GridClientMessage msg = (GridClientMessage)msg0;
@@ -143,10 +143,10 @@ public class GridTcpRestParser implements GridNioParser {
      * @param state Current parser state.
      * @return Parsed packet.s
      * @throws IOException If packet cannot be parsed.
-     * @throws GridException If deserialization error occurred.
+     * @throws IgniteCheckedException If deserialization error occurred.
      */
     @Nullable private GridClientMessage parseMemcachePacket(GridNioSession ses, ByteBuffer buf, ParserState state)
-        throws IOException, GridException {
+        throws IOException, IgniteCheckedException {
         assert state.packetType() == GridClientPacketType.MEMCACHE;
         assert state.packet() != null;
         assert state.packet() instanceof GridMemcachedMessage;
@@ -293,10 +293,10 @@ public class GridTcpRestParser implements GridNioParser {
      * @param state Parser state.
      * @return Parsed message.
      * @throws IOException If packet parsing or deserialization failed.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @Nullable private GridClientMessage parseCustomPacket(GridNioSession ses, ByteBuffer buf, ParserState state)
-        throws IOException, GridException {
+        throws IOException, IgniteCheckedException {
         assert state.packetType() == GridClientPacketType.GRIDGAIN;
         assert state.packet() == null;
 
@@ -417,9 +417,9 @@ public class GridTcpRestParser implements GridNioParser {
      * @param state Parser state.
      * @return A parsed client message.
      * @throws IOException On marshaller error.
-     * @throws GridException If no marshaller was defined for the session.
+     * @throws IgniteCheckedException If no marshaller was defined for the session.
      */
-    protected GridClientMessage parseClientMessage(GridNioSession ses, ParserState state) throws IOException, GridException {
+    protected GridClientMessage parseClientMessage(GridNioSession ses, ParserState state) throws IOException, IgniteCheckedException {
         GridClientMarshaller marsh = marshaller(ses);
 
         GridClientMessage msg = marsh.unmarshal(state.buffer().toByteArray());
@@ -436,9 +436,9 @@ public class GridTcpRestParser implements GridNioParser {
      *
      * @param msg Message being serialized.
      * @return Serialized message.
-     * @throws GridException If serialization failed.
+     * @throws IgniteCheckedException If serialization failed.
      */
-    private ByteBuffer encodeMemcache(GridMemcachedMessage msg) throws GridException {
+    private ByteBuffer encodeMemcache(GridMemcachedMessage msg) throws IgniteCheckedException {
         GridByteArrayList res = new GridByteArrayList(HDR_LEN);
 
         int keyLen = 0;
@@ -522,9 +522,9 @@ public class GridTcpRestParser implements GridNioParser {
      * @param req Raw packet.
      * @return Same packet with fields deserialized.
      * @throws IOException If parsing failed.
-     * @throws GridException If deserialization failed.
+     * @throws IgniteCheckedException If deserialization failed.
      */
-    private GridClientMessage assemble(GridNioSession ses, GridMemcachedMessage req) throws IOException, GridException {
+    private GridClientMessage assemble(GridNioSession ses, GridMemcachedMessage req) throws IOException, IgniteCheckedException {
         byte[] extras = req.extras();
 
         // First, decode key and value, if any
@@ -614,9 +614,9 @@ public class GridTcpRestParser implements GridNioParser {
      * @param flags Flags.
      * @param bytes Byte array to decode.
      * @return Decoded value.
-     * @throws GridException If deserialization failed.
+     * @throws IgniteCheckedException If deserialization failed.
      */
-    private Object decodeObj(short flags, byte[] bytes) throws GridException {
+    private Object decodeObj(short flags, byte[] bytes) throws IgniteCheckedException {
         assert bytes != null;
 
         if ((flags & SERIALIZED_FLAG) != 0)
@@ -652,9 +652,9 @@ public class GridTcpRestParser implements GridNioParser {
      * @param obj Object to serialize.
      * @param out Output stream to which object should be written.
      * @return Serialization flags.
-     * @throws GridException If JDK serialization failed.
+     * @throws IgniteCheckedException If JDK serialization failed.
      */
-    private int encodeObj(Object obj, ByteArrayOutputStream out) throws GridException {
+    private int encodeObj(Object obj, ByteArrayOutputStream out) throws IgniteCheckedException {
         int flags = 0;
 
         byte[] data = null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestProtocol.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
index e6c3812..24e259f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/protocols/tcp/GridTcpRestProtocol.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.rest.protocols.tcp;
 
+import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.marshaller.jdk.*;
@@ -17,7 +18,6 @@ import org.gridgain.client.marshaller.*;
 import org.gridgain.client.marshaller.jdk.*;
 import org.gridgain.client.marshaller.optimized.*;
 import org.gridgain.client.ssl.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.processors.rest.*;
 import org.gridgain.grid.kernal.processors.rest.client.message.*;
@@ -143,7 +143,7 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
 
     /** {@inheritDoc} */
     @SuppressWarnings("BusyWait")
-    @Override public void start(final GridRestProtocolHandler hnd) throws GridException {
+    @Override public void start(final GridRestProtocolHandler hnd) throws IgniteCheckedException {
         assert hnd != null;
 
         ClientConnectionConfiguration cfg = ctx.config().getClientConnectionConfiguration();
@@ -163,7 +163,7 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
                 GridSslContextFactory factory = cfg.getRestTcpSslContextFactory();
 
                 if (factory == null)
-                    // Thrown SSL exception instead of GridException for writing correct warning message into log.
+                    // Thrown SSL exception instead of IgniteCheckedException for writing correct warning message into log.
                     throw new SSLException("SSL is enabled, but SSL context factory is not specified.");
 
                 sslCtx = factory.createSslContext();
@@ -302,7 +302,7 @@ public class GridTcpRestProtocol extends GridRestProtocolAdapter {
 
             return true;
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             if (log.isDebugEnabled())
                 log.debug("Failed to start " + name() + " protocol on port " + port + ": " + e.getMessage());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/schedule/GridNoopScheduleProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/schedule/GridNoopScheduleProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/schedule/GridNoopScheduleProcessor.java
index c6663c2..8e2f41b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/schedule/GridNoopScheduleProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/schedule/GridNoopScheduleProcessor.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.schedule;
 
+import org.apache.ignite.*;
 import org.apache.ignite.scheduler.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 
 import java.util.concurrent.*;
@@ -39,8 +39,8 @@ public class GridNoopScheduleProcessor extends GridScheduleProcessorAdapter {
     /**
      * @return No-op processor usage exception;
      */
-    private GridRuntimeException processorException() {
-        return new GridRuntimeException("Current GridGain configuration does not support schedule functionality " +
+    private IgniteException processorException() {
+        return new IgniteException("Current GridGain configuration does not support schedule functionality " +
             "(consider adding gridgain-schedule module to classpath).");
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceMethodNotFoundException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceMethodNotFoundException.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceMethodNotFoundException.java
index 3d13613..341daf5 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceMethodNotFoundException.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceMethodNotFoundException.java
@@ -9,14 +9,14 @@
 
 package org.gridgain.grid.kernal.processors.service;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 import java.util.*;
 
 /**
  * Exception thrown if service is not found.
  */
-public class GridServiceMethodNotFoundException extends GridRuntimeException {
+public class GridServiceMethodNotFoundException extends IgniteException {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java
index 070e950..0a7a688 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java
@@ -9,12 +9,12 @@
 
 package org.gridgain.grid.kernal.processors.service;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 /**
  * Exception thrown if service is not found.
  */
-public class GridServiceNotFoundException extends GridException {
+public class GridServiceNotFoundException extends IgniteCheckedException {
     /** */
     private static final long serialVersionUID = 0L;