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

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java
index 37e011a..e6ce07e 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/IgniteDataLoaderImpl.java
@@ -270,14 +270,14 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     }
 
     /** {@inheritDoc} */
-    @Override public void isolated(boolean isolated) throws GridException {
+    @Override public void isolated(boolean isolated) throws IgniteCheckedException {
         if (isolated())
             return;
 
         ClusterNode node = F.first(ctx.grid().forCache(cacheName).nodes());
 
         if (node == null)
-            throw new GridException("Failed to get node for cache: " + cacheName);
+            throw new IgniteCheckedException("Failed to get node for cache: " + cacheName);
 
         GridCacheAttributes a = U.cacheAttributes(node, cacheName);
 
@@ -365,7 +365,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
             return resFut;
         }
-        catch (GridRuntimeException e) {
+        catch (IgniteException e) {
             return new GridFinishedFuture<>(ctx, e);
         }
         finally {
@@ -374,21 +374,21 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws GridException, IllegalStateException {
+    @Override public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteCheckedException, IllegalStateException {
         A.notNull(entry, "entry");
 
         return addData(F.asList(entry));
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> addData(K key, V val) throws GridException, IllegalStateException {
+    @Override public IgniteFuture<?> addData(K key, V val) throws IgniteCheckedException, IllegalStateException {
         A.notNull(key, "key");
 
         return addData(new Entry0<>(key, val));
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeData(K key) throws GridException, IllegalStateException {
+    @Override public IgniteFuture<?> removeData(K key) throws IgniteCheckedException, IllegalStateException {
         return addData(key, null);
     }
 
@@ -407,7 +407,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
         assert entries != null;
 
         if (remaps >= MAX_REMAP_CNT) {
-            resFut.onDone(new GridException("Failed to finish operation (too many remaps): " + remaps));
+            resFut.onDone(new IgniteCheckedException("Failed to finish operation (too many remaps): " + remaps));
 
             return;
         }
@@ -432,7 +432,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
                 node = ctx.affinity().mapKeyToNode(cacheName, key);
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 resFut.onDone(e);
 
                 return;
@@ -479,12 +479,12 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                         if (activeKeys.isEmpty())
                             resFut.onDone();
                     }
-                    catch (GridException e1) {
+                    catch (IgniteCheckedException e1) {
                         if (log.isDebugEnabled())
                             log.debug("Future finished with error [nodeId=" + nodeId + ", err=" + e1 + ']');
 
                         if (cancelled) {
-                            resFut.onDone(new GridException("Data loader has been cancelled: " +
+                            resFut.onDone(new IgniteCheckedException("Data loader has been cancelled: " +
                                 IgniteDataLoaderImpl.this, e1));
                         }
                         else
@@ -518,9 +518,9 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     /**
      * Performs flush.
      *
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private void doFlush() throws GridException {
+    private void doFlush() throws IgniteCheckedException {
         lastFlushTime = U.currentTimeMillis();
 
         List<IgniteFuture> activeFuts0 = null;
@@ -567,7 +567,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                     try {
                         fut.get();
                     }
-                    catch (GridException e) {
+                    catch (IgniteCheckedException e) {
                         if (log.isDebugEnabled())
                             log.debug("Failed to flush buffer: " + e);
 
@@ -605,7 +605,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
     /** {@inheritDoc} */
     @SuppressWarnings("ForLoopReplaceableByForEach")
-    @Override public void flush() throws GridException {
+    @Override public void flush() throws IgniteCheckedException {
         enterBusy();
 
         try {
@@ -640,9 +640,9 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
     /**
      * @param cancel {@code True} to close with cancellation.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    @Override public void close(boolean cancel) throws GridException {
+    @Override public void close(boolean cancel) throws IgniteCheckedException {
         if (!closed.compareAndSet(false, true))
             return;
 
@@ -651,7 +651,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
         if (log.isDebugEnabled())
             log.debug("Closing data loader [ldr=" + this + ", cancel=" + cancel + ']');
 
-        GridException e = null;
+        IgniteCheckedException e = null;
 
         try {
             // Assuming that no methods are called on this loader after this method is called.
@@ -668,7 +668,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
             ctx.io().removeMessageListener(topic);
         }
-        catch (GridException e0) {
+        catch (IgniteCheckedException e0) {
             e = e0;
         }
 
@@ -686,7 +686,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     }
 
     /** {@inheritDoc} */
-    @Override public void close() throws GridException {
+    @Override public void close() throws IgniteCheckedException {
         close(false);
     }
 
@@ -802,7 +802,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                 submit(entries0, curFut0);
 
                 if (cancelled)
-                    curFut0.onDone(new GridException("Data loader has been cancelled: " + IgniteDataLoaderImpl.this));
+                    curFut0.onDone(new IgniteCheckedException("Data loader has been cancelled: " + IgniteDataLoaderImpl.this));
             }
 
             return curFut0;
@@ -910,7 +910,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
                             curFut.onDone(t.get());
                         }
-                        catch (GridException e) {
+                        catch (IgniteCheckedException e) {
                             curFut.onDone(e);
                         }
                     }
@@ -949,7 +949,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                     if (topicBytes == null)
                         topicBytes = ctx.config().getMarshaller().marshal(topic);
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(log, "Failed to marshal (request will not be sent).", e);
 
                     return;
@@ -971,7 +971,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                         if (cache != null)
                             cache.context().deploy().onEnter();
                     }
-                    catch (GridException e) {
+                    catch (IgniteCheckedException e) {
                         U.error(log, "Failed to deploy class (request will not be sent): " + jobPda0.deployClass(), e);
 
                         return;
@@ -1007,7 +1007,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                     if (log.isDebugEnabled())
                         log.debug("Sent request to node [nodeId=" + node.id() + ", req=" + req + ']');
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     if (ctx.discovery().alive(node) && ctx.discovery().pingNode(node.id()))
                         ((GridFutureAdapter<Object>)fut).onDone(e);
                     else
@@ -1071,8 +1071,8 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                         errBytes,
                         jobPda0 != null ? jobPda0.classLoader() : U.gridClassLoader());
                 }
-                catch (GridException e) {
-                    f.onDone(null, new GridException("Failed to unmarshal response.", e));
+                catch (IgniteCheckedException e) {
+                    f.onDone(null, new IgniteCheckedException("Failed to unmarshal response.", e));
 
                     return;
                 }
@@ -1088,13 +1088,13 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
          *
          */
         void cancelAll() {
-            GridException err = new GridException("Data loader has been cancelled: " + IgniteDataLoaderImpl.this);
+            IgniteCheckedException err = new IgniteCheckedException("Data loader has been cancelled: " + IgniteDataLoaderImpl.this);
 
             for (IgniteFuture<?> f : locFuts) {
                 try {
                     f.cancel();
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(log, "Failed to cancel mini-future.", e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrDataLoadCacheUpdater.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrDataLoadCacheUpdater.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrDataLoadCacheUpdater.java
index 90989a2..3522cf8 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrDataLoadCacheUpdater.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridDrDataLoadCacheUpdater.java
@@ -30,7 +30,7 @@ public class GridDrDataLoadCacheUpdater<K, V> implements IgniteDataLoadCacheUpda
 
     /** {@inheritDoc} */
     @Override public void update(GridCache<K, V> cache0, Collection<Map.Entry<K, V>> col)
-        throws GridException {
+        throws IgniteCheckedException {
         String cacheName = cache0.name();
 
         GridKernalContext ctx = ((GridKernal)cache0.gridProjection().ignite()).context();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridRawVersionedEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridRawVersionedEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridRawVersionedEntry.java
index 3f0d7ba..919a958 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridRawVersionedEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridRawVersionedEntry.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.dr;
 
+import org.apache.ignite.*;
 import org.apache.ignite.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
@@ -121,7 +121,7 @@ public class GridRawVersionedEntry<K, V> implements GridVersionedEntry<K, V>, Ex
     }
 
     /** {@inheritDoc} */
-    @Override public void unmarshal(IgniteMarshaller marsh) throws GridException {
+    @Override public void unmarshal(IgniteMarshaller marsh) throws IgniteCheckedException {
         unmarshalKey(marsh);
 
         if (valBytes != null && val == null)
@@ -133,15 +133,15 @@ public class GridRawVersionedEntry<K, V> implements GridVersionedEntry<K, V>, Ex
      * its restored key/value are needed.
      *
      * @param marsh Marshaller.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private void unmarshalKey(IgniteMarshaller marsh) throws GridException {
+    private void unmarshalKey(IgniteMarshaller marsh) throws IgniteCheckedException {
         if (key == null)
             key = marsh.unmarshal(keyBytes, null);
     }
 
     /** {@inheritDoc} */
-    @Override public void marshal(IgniteMarshaller marsh) throws GridException {
+    @Override public void marshal(IgniteMarshaller marsh) throws IgniteCheckedException {
         if (keyBytes == null)
             keyBytes = marsh.marshal(key);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridVersionedEntry.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridVersionedEntry.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridVersionedEntry.java
index 90a7fe5..89cdadb 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridVersionedEntry.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dr/GridVersionedEntry.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.dr;
 
+import org.apache.ignite.*;
 import org.apache.ignite.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.jetbrains.annotations.*;
 
@@ -57,16 +57,16 @@ public interface GridVersionedEntry<K, V> extends Map.Entry<K, V> {
      * Perform internal marshal of this entry before it will be serialized.
      *
      * @param marsh Marshaller.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void marshal(IgniteMarshaller marsh) throws GridException;
+    public void marshal(IgniteMarshaller marsh) throws IgniteCheckedException;
 
     /**
      * Perform internal unmarshal of this entry. It must be performed after entry is deserialized and before
      * its restored key/value are needed.
      *
      * @param marsh Marshaller.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void unmarshal(IgniteMarshaller marsh) throws GridException;
+    public void unmarshal(IgniteMarshaller marsh) throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/email/GridEmailProcessorAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/email/GridEmailProcessorAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/email/GridEmailProcessorAdapter.java
index 6b842bd..ab942f0 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/email/GridEmailProcessorAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/email/GridEmailProcessorAdapter.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.email;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
@@ -34,9 +35,9 @@ public abstract class GridEmailProcessorAdapter extends GridProcessorAdapter {
      * @param subj Email subject.
      * @param body Email body.
      * @param html HTML format flag.
-     * @throws org.gridgain.grid.GridException Thrown in case of any failure on sending.
+     * @throws IgniteCheckedException Thrown in case of any failure on sending.
      */
-    public abstract void sendNow(String subj, String body, boolean html) throws GridException;
+    public abstract void sendNow(String subj, String body, boolean html) throws IgniteCheckedException;
 
     /**
      * Sends given email in the current thread blocking until it's either successfully sent or failed.
@@ -46,9 +47,9 @@ public abstract class GridEmailProcessorAdapter extends GridProcessorAdapter {
      * @param body Email body.
      * @param html HTML format flag.
      * @param addrs Addresses.
-     * @throws GridException Thrown in case of any failure on sending.
+     * @throws IgniteCheckedException Thrown in case of any failure on sending.
      */
-    public abstract void sendNow(String subj, String body, boolean html, Collection<String> addrs) throws GridException;
+    public abstract void sendNow(String subj, String body, boolean html, Collection<String> addrs) throws IgniteCheckedException;
 
     /**
      * Schedules sending of given email to all admin emails, if any. If SMTP is disabled or admin emails

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java
index 29b6e3d..d47c661 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAckMessage.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
+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.util.direct.*;
 import org.jetbrains.annotations.*;
@@ -34,7 +34,7 @@ public class GridGgfsAckMessage extends GridGgfsCommunicationMessage {
 
     /** Write exception. */
     @GridDirectTransient
-    private GridException err;
+    private IgniteCheckedException err;
 
     /** */
     private byte[] errBytes;
@@ -51,7 +51,7 @@ public class GridGgfsAckMessage extends GridGgfsCommunicationMessage {
      * @param id Request ID.
      * @param err Error.
      */
-    public GridGgfsAckMessage(IgniteUuid fileId, long id, @Nullable GridException err) {
+    public GridGgfsAckMessage(IgniteUuid fileId, long id, @Nullable IgniteCheckedException err) {
         this.fileId = fileId;
         this.id = id;
         this.err = err;
@@ -74,12 +74,12 @@ public class GridGgfsAckMessage extends GridGgfsCommunicationMessage {
     /**
      * @return Error occurred when writing this batch, if any.
      */
-    public GridException error() {
+    public IgniteCheckedException error() {
         return err;
     }
 
     /** {@inheritDoc} */
-    @Override public void prepareMarshal(IgniteMarshaller marsh) throws GridException {
+    @Override public void prepareMarshal(IgniteMarshaller marsh) throws IgniteCheckedException {
         super.prepareMarshal(marsh);
 
         if (err != null)
@@ -87,7 +87,7 @@ public class GridGgfsAckMessage extends GridGgfsCommunicationMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public void finishUnmarshal(IgniteMarshaller marsh, @Nullable ClassLoader ldr) throws GridException {
+    @Override public void finishUnmarshal(IgniteMarshaller marsh, @Nullable ClassLoader ldr) throws IgniteCheckedException {
         super.finishUnmarshal(marsh, ldr);
 
         if (errBytes != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java
index 24f0a5c..e22c0b7 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAsyncImpl.java
@@ -41,33 +41,33 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid
     }
 
     /** {@inheritDoc} */
-    @Override public void format() throws GridException {
+    @Override public void format() throws IgniteCheckedException {
         saveOrGet(ggfs.formatAsync());
     }
 
     /** {@inheritDoc} */
     @Override public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
-        Collection<IgniteFsPath> paths, @Nullable T arg) throws GridException {
+        Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException {
         return saveOrGet(ggfs.executeAsync(task, rslvr, paths, arg));
     }
 
     /** {@inheritDoc} */
     @Override public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
         Collection<IgniteFsPath> paths, boolean skipNonExistentFiles, long maxRangeLen, @Nullable T arg)
-        throws GridException {
+        throws IgniteCheckedException {
         return saveOrGet(ggfs.executeAsync(task, rslvr, paths, skipNonExistentFiles, maxRangeLen, arg));
     }
 
     /** {@inheritDoc} */
     @Override public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls,
-        @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws GridException {
+        @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException {
         return saveOrGet(ggfs.executeAsync(taskCls, rslvr, paths, arg));
     }
 
     /** {@inheritDoc} */
     @Override public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls,
         @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, boolean skipNonExistentFiles,
-        long maxRangeLen, @Nullable T arg) throws GridException {
+        long maxRangeLen, @Nullable T arg) throws IgniteCheckedException {
         return saveOrGet(ggfs.executeAsync(taskCls, rslvr, paths, skipNonExistentFiles, maxRangeLen, arg));
     }
 
@@ -88,27 +88,27 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid
 
     /** {@inheritDoc} */
     @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize,
-        int seqReadsBeforePrefetch) throws GridException {
+        int seqReadsBeforePrefetch) throws IgniteCheckedException {
         return ggfs.open(path, bufSize, seqReadsBeforePrefetch);
     }
 
     /** {@inheritDoc} */
-    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws GridException {
+    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteCheckedException {
         return ggfs.open(path);
     }
 
     /** {@inheritDoc} */
-    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws GridException {
+    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteCheckedException {
         return ggfs.open(path, bufSize);
     }
 
     /** {@inheritDoc} */
-    @Override public GridGgfsStatus globalSpace() throws GridException {
+    @Override public GridGgfsStatus globalSpace() throws IgniteCheckedException {
         return ggfs.globalSpace();
     }
 
     /** {@inheritDoc} */
-    @Override public void globalSampling(@Nullable Boolean val) throws GridException {
+    @Override public void globalSampling(@Nullable Boolean val) throws IgniteCheckedException {
         ggfs.globalSampling(val);
     }
 
@@ -128,7 +128,7 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> awaitDeletesAsync() throws GridException {
+    @Override public IgniteFuture<?> awaitDeletesAsync() throws IgniteCheckedException {
         return ggfs.awaitDeletesAsync();
     }
 
@@ -168,118 +168,118 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws GridException {
+    @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteCheckedException {
         return ggfs.summary(path);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws GridException {
+    @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteCheckedException {
         return ggfs.create(path, overwrite);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite, int replication,
-        long blockSize, @Nullable Map<String, String> props) throws GridException {
+        long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
         return ggfs.create(path, bufSize, overwrite, replication, blockSize, props);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite,
         @Nullable IgniteUuid affKey, int replication, long blockSize, @Nullable Map<String, String> props)
-        throws GridException {
+        throws IgniteCheckedException {
         return ggfs.create(path, bufSize, overwrite, affKey, replication, blockSize, props);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws GridException {
+    @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteCheckedException {
         return ggfs.append(path, create);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFsOutputStream append(IgniteFsPath path, int bufSize, boolean create,
-        @Nullable Map<String, String> props) throws GridException {
+        @Nullable Map<String, String> props) throws IgniteCheckedException {
         return ggfs.append(path, bufSize, create, props);
     }
 
     /** {@inheritDoc} */
-    @Override public void setTimes(IgniteFsPath path, long accessTime, long modificationTime) throws GridException {
+    @Override public void setTimes(IgniteFsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
         ggfs.setTimes(path, accessTime, modificationTime);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len)
-        throws GridException {
+        throws IgniteCheckedException {
         return ggfs.affinity(path, start, len);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len, long maxLen)
-        throws GridException {
+        throws IgniteCheckedException {
         return ggfs.affinity(path, start, len, maxLen);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsMetrics metrics() throws GridException {
+    @Override public IgniteFsMetrics metrics() throws IgniteCheckedException {
         return ggfs.metrics();
     }
 
     /** {@inheritDoc} */
-    @Override public void resetMetrics() throws GridException {
+    @Override public void resetMetrics() throws IgniteCheckedException {
         ggfs.resetMetrics();
     }
 
     /** {@inheritDoc} */
-    @Override public long size(IgniteFsPath path) throws GridException {
+    @Override public long size(IgniteFsPath path) throws IgniteCheckedException {
         return ggfs.size(path);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean exists(IgniteFsPath path) throws GridException {
+    @Override public boolean exists(IgniteFsPath path) throws IgniteCheckedException {
         return ggfs.exists(path);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws GridException {
+    @Nullable @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws IgniteCheckedException {
         return ggfs.update(path, props);
     }
 
     /** {@inheritDoc} */
-    @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws GridException {
+    @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws IgniteCheckedException {
         ggfs.rename(src, dest);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean delete(IgniteFsPath path, boolean recursive) throws GridException {
+    @Override public boolean delete(IgniteFsPath path, boolean recursive) throws IgniteCheckedException {
         return ggfs.delete(path, recursive);
     }
 
     /** {@inheritDoc} */
-    @Override public void mkdirs(IgniteFsPath path) throws GridException {
+    @Override public void mkdirs(IgniteFsPath path) throws IgniteCheckedException {
         ggfs.mkdirs(path);
     }
 
     /** {@inheritDoc} */
-    @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws GridException {
+    @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws IgniteCheckedException {
         ggfs.mkdirs(path, props);
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgniteFsPath> listPaths(IgniteFsPath path) throws GridException {
+    @Override public Collection<IgniteFsPath> listPaths(IgniteFsPath path) throws IgniteCheckedException {
         return ggfs.listPaths(path);
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgniteFsFile> listFiles(IgniteFsPath path) throws GridException {
+    @Override public Collection<IgniteFsFile> listFiles(IgniteFsPath path) throws IgniteCheckedException {
         return ggfs.listFiles(path);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFsFile info(IgniteFsPath path) throws GridException {
+    @Nullable @Override public IgniteFsFile info(IgniteFsPath path) throws IgniteCheckedException {
         return ggfs.info(path);
     }
 
     /** {@inheritDoc} */
-    @Override public long usedSpaceSize() throws GridException {
+    @Override public long usedSpaceSize() throws IgniteCheckedException {
         return ggfs.usedSpaceSize();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlockLocationImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlockLocationImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlockLocationImpl.java
index 6fdf889..6833123 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlockLocationImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsBlockLocationImpl.java
@@ -9,12 +9,12 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.fs.*;
-import org.gridgain.grid.*;
+import org.gridgain.grid.util.tostring.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.grid.util.tostring.*;
 
 import java.io.*;
 import java.net.*;
@@ -236,7 +236,7 @@ public class GridGgfsBlockLocationImpl implements IgniteFsBlockLocation, Externa
                     }
                 }
             }
-            catch (GridException ignored) {
+            catch (IgniteCheckedException ignored) {
                 names.addAll(node.addresses());
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsCommunicationMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsCommunicationMessage.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsCommunicationMessage.java
index ce88052..a905c52 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsCommunicationMessage.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsCommunicationMessage.java
@@ -9,8 +9,8 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
+import org.apache.ignite.*;
 import org.apache.ignite.marshaller.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.util.direct.*;
 import org.jetbrains.annotations.*;
 
@@ -29,18 +29,18 @@ public abstract class GridGgfsCommunicationMessage extends GridTcpCommunicationM
 
     /**
      * @param marsh Marshaller.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    public void prepareMarshal(IgniteMarshaller marsh) throws GridException {
+    public void prepareMarshal(IgniteMarshaller marsh) throws IgniteCheckedException {
         // No-op.
     }
 
     /**
      * @param marsh Marshaller.
      * @param ldr Class loader.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    public void finishUnmarshal(IgniteMarshaller marsh, @Nullable ClassLoader ldr) throws GridException {
+    public void finishUnmarshal(IgniteMarshaller marsh, @Nullable ClassLoader ldr) throws IgniteCheckedException {
         // No-op.
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsContext.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsContext.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsContext.java
index c06fafe..3ecbdcf 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsContext.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsContext.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.fs.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.util.typedef.*;
@@ -56,7 +56,7 @@ public class GridGgfsContext {
      * @param dataMgr Data manager.
      * @param srvMgr Server manager.
      * @param fragmentizerMgr Fragmentizer manager.
-     * @throws GridException If GGFs context instantiation is failed.
+     * @throws IgniteCheckedException If GGFs context instantiation is failed.
      */
     public GridGgfsContext(
         GridKernalContext ctx,
@@ -65,7 +65,7 @@ public class GridGgfsContext {
         GridGgfsDataManager dataMgr,
         GridGgfsServerManager srvMgr,
         GridGgfsFragmentizerManager fragmentizerMgr
-    ) throws GridException {
+    ) throws IgniteCheckedException {
         this.ctx = ctx;
         this.cfg = cfg;
 
@@ -138,10 +138,10 @@ public class GridGgfsContext {
      * @param topic Topic.
      * @param msg Message.
      * @param plc Policy.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     public void send(UUID nodeId, Object topic, GridGgfsCommunicationMessage msg, GridIoPolicy plc)
-        throws GridException {
+        throws IgniteCheckedException {
         if (!kernalContext().localNodeId().equals(nodeId))
             msg.prepareMarshal(kernalContext().config().getMarshaller());
 
@@ -153,10 +153,10 @@ public class GridGgfsContext {
      * @param topic Topic.
      * @param msg Message.
      * @param plc Policy.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     public void send(ClusterNode node, Object topic, GridGgfsCommunicationMessage msg, GridIoPolicy plc)
-        throws GridException {
+        throws IgniteCheckedException {
         if (!kernalContext().localNodeId().equals(node.id()))
             msg.prepareMarshal(kernalContext().config().getMarshaller());
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java
index d495693..a42f530 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDataManager.java
@@ -130,14 +130,14 @@ public class GridGgfsDataManager extends GridGgfsManager {
             try {
                 dataCacheStartFut.get();
             }
-            catch (GridException e) {
-                throw new GridRuntimeException(e);
+            catch (IgniteCheckedException e) {
+                throw new IgniteException(e);
             }
         }
     }
 
     /** {@inheritDoc} */
-    @Override protected void start0() throws GridException {
+    @Override protected void start0() throws IgniteCheckedException {
         ggfs = ggfsCtx.ggfs();
 
         dataCachePrj = ggfsCtx.kernalContext().cache().internalCache(ggfsCtx.configuration().getDataCacheName());
@@ -147,7 +147,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
             .preloader().startFuture();
 
         if (dataCache.configuration().getAtomicityMode() != TRANSACTIONAL)
-            throw new GridException("Data cache should be transactional: " +
+            throw new IgniteCheckedException("Data cache should be transactional: " +
                 ggfsCtx.configuration().getDataCacheName());
 
         metrics = ggfsCtx.ggfs().localMetrics();
@@ -214,7 +214,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
     }
 
     /** {@inheritDoc} */
-    @Override protected void onKernalStart0() throws GridException {
+    @Override protected void onKernalStart0() throws IgniteCheckedException {
         new Thread(delWorker).start();
     }
 
@@ -313,10 +313,10 @@ public class GridGgfsDataManager extends GridGgfsManager {
      *
      * @param fileInfo File info.
      * @return List of local data block indices.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public List<Long> listLocalDataBlocks(GridGgfsFileInfo fileInfo)
-        throws GridException {
+        throws IgniteCheckedException {
         assert fileInfo != null;
 
         int prevGrpIdx = 0; // Block index within affinity group.
@@ -368,11 +368,11 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param blockIdx Block index.
      * @param secReader Optional secondary file system reader.
      * @return Requested data block or {@code null} if nothing found.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @Nullable public IgniteFuture<byte[]> dataBlock(final GridGgfsFileInfo fileInfo, final IgniteFsPath path,
         final long blockIdx, @Nullable final IgniteFsReader secReader)
-        throws GridException {
+        throws IgniteCheckedException {
         //assert validTxState(any); // Allow this method call for any transaction state.
 
         assert fileInfo != null;
@@ -395,7 +395,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
 
         if (secReader != null) {
             fut = fut.chain(new CX1<IgniteFuture<byte[]>, byte[]>() {
-                @Override public byte[] applyx(IgniteFuture<byte[]> fut) throws GridException {
+                @Override public byte[] applyx(IgniteFuture<byte[]> fut) throws IgniteCheckedException {
                     byte[] res = fut.get();
 
                     if (res == null) {
@@ -430,7 +430,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                                         }
                                     }
                                     catch (IOException e) {
-                                        throw new GridException("Failed to read data due to secondary file system " +
+                                        throw new IgniteCheckedException("Failed to read data due to secondary file system " +
                                             "exception: " + e.getMessage(), e);
                                     }
                                 }
@@ -445,7 +445,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
 
                                 metrics.addReadBlocks(1, 1);
                             }
-                            catch (GridException e) {
+                            catch (IgniteCheckedException e) {
                                 rmtReadFut.onDone(e);
 
                                 throw e;
@@ -527,7 +527,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param batch Optional secondary file system worker batch.
      *
      * @return Remainder if data did not fill full block.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     @Nullable public byte[] storeDataBlocks(
         GridGgfsFileInfo fileInfo,
@@ -538,7 +538,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
         boolean flush,
         GridGgfsFileAffinityRange affinityRange,
         @Nullable GridGgfsFileWorkerBatch batch
-    ) throws GridException {
+    ) throws IgniteCheckedException {
         //assert validTxState(any); // Allow this method call for any transaction state.
 
         return byteBufWriter.storeDataBlocks(fileInfo, reservedLen, remainder, remainderLen, data, data.remaining(),
@@ -558,7 +558,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param flush Flush flag.
      * @param affinityRange File affinity range to update if file cal be colocated.
      * @param batch Optional secondary file system worker batch.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      * @return Remainder of data that did not fit the block if {@code flush} flag is {@code false}.
      * @throws IOException If store failed.
      */
@@ -572,7 +572,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
         boolean flush,
         GridGgfsFileAffinityRange affinityRange,
         @Nullable GridGgfsFileWorkerBatch batch
-    ) throws GridException, IOException {
+    ) throws IgniteCheckedException, IOException {
         //assert validTxState(any); // Allow this method call for any transaction state.
 
         return dataInputWriter.storeDataBlocks(fileInfo, reservedLen, remainder, remainderLen, in, len, flush,
@@ -644,7 +644,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                 }
             }
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             log.error("Failed to clean up file range [fileInfo=" + fileInfo + ", range=" + range + ']', e);
         }
     }
@@ -715,7 +715,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                 }
             }
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             log.error("Failed to clean up file range [fileInfo=" + fileInfo + ", range=" + range + ']', e);
         }
     }
@@ -727,10 +727,10 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param start Start position in the file.
      * @param len File part length to get affinity for.
      * @return Affinity blocks locations.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public Collection<IgniteFsBlockLocation> affinity(GridGgfsFileInfo info, long start, long len)
-        throws GridException {
+        throws IgniteCheckedException {
         return affinity(info, start, len, 0);
     }
 
@@ -742,10 +742,10 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param len File part length to get affinity for.
      * @param maxLen Maximum block length.
      * @return Affinity blocks locations.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     public Collection<IgniteFsBlockLocation> affinity(GridGgfsFileInfo info, long start, long len, long maxLen)
-        throws GridException {
+        throws IgniteCheckedException {
         assert validTxState(false);
         assert info.isFile() : "Failed to get affinity (not a file): " + info;
         assert start >= 0 : "Start position should not be negative: " + start;
@@ -852,10 +852,10 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param len Length.
      * @param maxLen Maximum allowed split length.
      * @param res Result collection to add regions to.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
     private void affinity0(GridGgfsFileInfo info, long start, long len, long maxLen, Deque<IgniteFsBlockLocation> res)
-        throws GridException {
+        throws IgniteCheckedException {
         long firstGrpIdx = start / grpBlockSize;
         long limitGrpIdx = (start + len + grpBlockSize - 1) / grpBlockSize;
 
@@ -996,10 +996,10 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param fileId File ID.
      * @param node Node to process blocks on.
      * @param blocks Blocks to put in cache.
-     * @throws GridException If batch processing failed.
+     * @throws IgniteCheckedException If batch processing failed.
      */
     private void processBatch(IgniteUuid fileId, final ClusterNode node,
-        final Map<GridGgfsBlockKey, byte[]> blocks) throws GridException {
+        final Map<GridGgfsBlockKey, byte[]> blocks) throws IgniteCheckedException {
         final long batchId = reqIdCtr.getAndIncrement();
 
         final WriteCompletionFuture completionFut = pendingWrites.get(fileId);
@@ -1028,7 +1028,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                     try {
                         ggfsCtx.send(nodeId, topic, msg, SYSTEM_POOL);
                     }
-                    catch (GridException e) {
+                    catch (IgniteCheckedException e) {
                         completionFut.onError(nodeId, e);
                     }
 
@@ -1046,7 +1046,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
 
                                 completionFut.onWriteAck(nodeId, batchId);
                             }
-                            catch (GridException e) {
+                            catch (IgniteCheckedException e) {
                                 completionFut.onError(nodeId, e);
                             }
                         }
@@ -1066,10 +1066,10 @@ public class GridGgfsDataManager extends GridGgfsManager {
      * @param colocatedKey Block key.
      * @param startOff Data start offset within block.
      * @param data Data to write.
-     * @throws GridException If update failed.
+     * @throws IgniteCheckedException If update failed.
      */
     private void processPartialBlockWrite(IgniteUuid fileId, GridGgfsBlockKey colocatedKey, int startOff,
-        byte[] data) throws GridException {
+        byte[] data) throws IgniteCheckedException {
         if (dataCachePrj.ggfsDataSpaceUsed() >= dataCachePrj.ggfsDataSpaceMax()) {
             try {
                 ggfs.awaitDeletesAsync().get(trashPurgeTimeout);
@@ -1139,7 +1139,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
             }
 
             if (!hasVal)
-                throw new GridException("Failed to write partial block (no previous data was found in cache) " +
+                throw new IgniteCheckedException("Failed to write partial block (no previous data was found in cache) " +
                     "[key=" + colocatedKey + ", relaxedKey=" + key + ", startOff=" + startOff +
                     ", dataLen=" + data.length + ']');
 
@@ -1177,9 +1177,9 @@ public class GridGgfsDataManager extends GridGgfsManager {
      *
      * @param key Key.
      * @param data Data.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private void putSafe(final GridGgfsBlockKey key, final byte[] data) throws GridException {
+    private void putSafe(final GridGgfsBlockKey key, final byte[] data) throws IgniteCheckedException {
         assert key != null;
         assert data != null;
 
@@ -1193,7 +1193,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                 curPendingPuts += data.length;
             }
             catch (InterruptedException ignore) {
-                throw new GridException("Failed to put GGFS data block into cache due to interruption: " + key);
+                throw new IgniteCheckedException("Failed to put GGFS data block into cache due to interruption: " + key);
             }
             finally {
                 pendingPutsLock.unlock();
@@ -1205,7 +1205,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                 try {
                     dataCachePrj.putx(key, data);
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.warn(log, "Failed to put GGFS data block into cache [key=" + key + ", err=" + e + ']');
                 }
                 finally {
@@ -1258,8 +1258,8 @@ public class GridGgfsDataManager extends GridGgfsManager {
                             ", allowed=" + dataCachePrj.ggfsDataSpaceMax() + ']'));
 
             }
-            catch (GridException e) {
-                return new GridFinishedFuture<>(ggfsCtx.kernalContext(), new GridException("Failed to store data " +
+            catch (IgniteCheckedException e) {
+                return new GridFinishedFuture<>(ggfsCtx.kernalContext(), new IgniteCheckedException("Failed to store data " +
                     "block due to unexpected exception.", e));
             }
         }
@@ -1274,12 +1274,12 @@ public class GridGgfsDataManager extends GridGgfsManager {
     private void processBlocksMessage(final UUID nodeId, final GridGgfsBlocksMessage blocksMsg) {
         storeBlocksAsync(blocksMsg.blocks()).listenAsync(new CI1<IgniteFuture<?>>() {
             @Override public void apply(IgniteFuture<?> fut) {
-                GridException err = null;
+                IgniteCheckedException err = null;
 
                 try {
                     fut.get();
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     err = e;
                 }
 
@@ -1288,7 +1288,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                     ggfsCtx.send(nodeId, topic, new GridGgfsAckMessage(blocksMsg.fileId(), blocksMsg.id(), err),
                         SYSTEM_POOL);
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.warn(log, "Failed to send batch acknowledgement (did node leave the grid?) [nodeId=" + nodeId +
                         ", fileId=" + blocksMsg.fileId() + ", batchId=" + blocksMsg.id() + ']', e);
                 }
@@ -1304,7 +1304,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
         try {
             ackMsg.finishUnmarshal(ggfsCtx.kernalContext().config().getMarshaller(), null);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Failed to unmarshal message (will ignore): " + ackMsg, e);
 
             return;
@@ -1388,7 +1388,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
          * @param flush Flush flag.
          * @param affinityRange Affinity range to update if file write can be colocated.
          * @param batch Optional secondary file system worker batch.
-         * @throws GridException If failed.
+         * @throws IgniteCheckedException If failed.
          * @return Data remainder if {@code flush} flag is {@code false}.
          */
         @Nullable public byte[] storeDataBlocks(
@@ -1401,7 +1401,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
             boolean flush,
             GridGgfsFileAffinityRange affinityRange,
             @Nullable GridGgfsFileWorkerBatch batch
-        ) throws GridException {
+        ) throws IgniteCheckedException {
             IgniteUuid id = fileInfo.id();
             int blockSize = fileInfo.blockSize();
 
@@ -1480,7 +1480,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
 
                 if (batch != null) {
                     if (!batch.write(portion))
-                        throw new GridException("Cannot write more data to the secondary file system output " +
+                        throw new IgniteCheckedException("Cannot write more data to the secondary file system output " +
                             "stream because it was marked as closed: " + batch.path());
                     else
                         writtenSecondary = 1;
@@ -1534,9 +1534,9 @@ public class GridGgfsDataManager extends GridGgfsManager {
          * @param src Data source.
          * @param dst Destination.
          * @param dstOff Destination buffer offset.
-         * @throws GridException If read failed.
+         * @throws IgniteCheckedException If read failed.
          */
-        protected abstract void readData(T src, byte[] dst, int dstOff) throws GridException;
+        protected abstract void readData(T src, byte[] dst, int dstOff) throws IgniteCheckedException;
     }
 
     /**
@@ -1555,12 +1555,12 @@ public class GridGgfsDataManager extends GridGgfsManager {
     private class DataInputBlocksWriter extends BlocksWriter<DataInput> {
         /** {@inheritDoc} */
         @Override protected void readData(DataInput src, byte[] dst, int dstOff)
-            throws GridException {
+            throws IgniteCheckedException {
             try {
                 src.readFully(dst, dstOff, dst.length - dstOff);
             }
             catch (IOException e) {
-                throw new GridException(e);
+                throw new IgniteCheckedException(e);
             }
         }
     }
@@ -1715,7 +1715,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                     catch (GridInterruptedException ignored) {
                         // Ignore interruption during shutdown.
                     }
-                    catch (GridException e) {
+                    catch (IgniteCheckedException e) {
                         log.error("Failed to remove file contents: " + fileInfo, e);
                     }
                     finally {
@@ -1726,14 +1726,14 @@ public class GridGgfsDataManager extends GridGgfsManager {
                                 ldr.removeData(new GridGgfsBlockKey(fileId, fileInfo.affinityKey(),
                                     fileInfo.evictExclude(), block));
                         }
-                        catch (GridException e) {
+                        catch (IgniteCheckedException e) {
                             log.error("Failed to remove file contents: " + fileInfo, e);
                         }
                         finally {
                             try {
                                 ldr.close(isCancelled());
                             }
-                            catch (GridException e) {
+                            catch (IgniteCheckedException e) {
                                 log.error("Failed to stop data loader while shutting down ggfs async delete thread.", e);
                             }
                             finally {
@@ -1828,15 +1828,15 @@ public class GridGgfsDataManager extends GridGgfsManager {
          * @param nodeId Node ID.
          * @param e Caught exception.
          */
-        private void onError(UUID nodeId, GridException e) {
+        private void onError(UUID nodeId, IgniteCheckedException e) {
             Set<Long> reqIds = pendingAcks.get(nodeId);
 
             // If waiting for ack from this node.
             if (reqIds != null && !reqIds.isEmpty()) {
                 if (e instanceof IgniteFsOutOfSpaceException)
-                    onDone(new GridException("Failed to write data (not enough space on node): " + nodeId, e));
+                    onDone(new IgniteCheckedException("Failed to write data (not enough space on node): " + nodeId, e));
                 else
-                    onDone(new GridException(
+                    onDone(new IgniteCheckedException(
                         "Failed to wait for write completion (write failed on node): " + nodeId, e));
             }
         }
@@ -1844,12 +1844,12 @@ public class GridGgfsDataManager extends GridGgfsManager {
         /**
          * @param e Error.
          */
-        private void onLocalError(GridException e) {
+        private void onLocalError(IgniteCheckedException e) {
             if (e instanceof IgniteFsOutOfSpaceException)
-                onDone(new GridException("Failed to write data (not enough space on node): " +
+                onDone(new IgniteCheckedException("Failed to write data (not enough space on node): " +
                     ggfsCtx.kernalContext().localNodeId(), e));
             else
-                onDone(new GridException(
+                onDone(new IgniteCheckedException(
                     "Failed to wait for write completion (write failed on node): " +
                         ggfsCtx.kernalContext().localNodeId(), e));
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteMessage.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteMessage.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteMessage.java
index 82150f3..c74c5e6 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteMessage.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteMessage.java
@@ -9,9 +9,9 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
+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.util.direct.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -32,7 +32,7 @@ public class GridGgfsDeleteMessage extends GridGgfsCommunicationMessage {
 
     /** Optional error. */
     @GridDirectTransient
-    private GridException err;
+    private IgniteCheckedException err;
 
     /** */
     private byte[] errBytes;
@@ -61,7 +61,7 @@ public class GridGgfsDeleteMessage extends GridGgfsCommunicationMessage {
      * @param id Entry ID.
      * @param err Error.
      */
-    public GridGgfsDeleteMessage(IgniteUuid id, GridException err) {
+    public GridGgfsDeleteMessage(IgniteUuid id, IgniteCheckedException err) {
         assert err != null;
 
         this.id = id;
@@ -78,12 +78,12 @@ public class GridGgfsDeleteMessage extends GridGgfsCommunicationMessage {
     /**
      * @return Error.
      */
-    public GridException error() {
+    public IgniteCheckedException error() {
         return err;
     }
 
     /** {@inheritDoc} */
-    @Override public void prepareMarshal(IgniteMarshaller marsh) throws GridException {
+    @Override public void prepareMarshal(IgniteMarshaller marsh) throws IgniteCheckedException {
         super.prepareMarshal(marsh);
 
         if (err != null)
@@ -91,7 +91,7 @@ public class GridGgfsDeleteMessage extends GridGgfsCommunicationMessage {
     }
 
     /** {@inheritDoc} */
-    @Override public void finishUnmarshal(IgniteMarshaller marsh, @Nullable ClassLoader ldr) throws GridException {
+    @Override public void finishUnmarshal(IgniteMarshaller marsh, @Nullable ClassLoader ldr) throws IgniteCheckedException {
         super.finishUnmarshal(marsh, ldr);
 
         if (errBytes != null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteWorker.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteWorker.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteWorker.java
index 5d4eca2..21214b7 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteWorker.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDeleteWorker.java
@@ -147,7 +147,7 @@ public class GridGgfsDeleteWorker extends GridGgfsThread {
         try {
             info = meta.info(TRASH_ID);
         }
-        catch (GridException e) {
+        catch (IgniteCheckedException e) {
             U.error(log, "Cannot obtain trash directory info.", e);
         }
 
@@ -174,7 +174,7 @@ public class GridGgfsDeleteWorker extends GridGgfsThread {
                 catch (GridInterruptedException ignored) {
                     // Ignore this exception while stopping.
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     U.error(log, "Failed to delete entry from the trash directory: " + entry.getKey(), e);
 
                     sendDeleteMessage(new GridGgfsDeleteMessage(fileId, e));
@@ -189,9 +189,9 @@ public class GridGgfsDeleteWorker extends GridGgfsThread {
      * @param name Entry name.
      * @param id Entry ID.
      * @return {@code True} in case the entry really was deleted form the file system by this call.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    private boolean delete(String name, IgniteUuid id) throws GridException {
+    private boolean delete(String name, IgniteUuid id) throws IgniteCheckedException {
         assert name != null;
         assert id != null;
 
@@ -235,9 +235,9 @@ public class GridGgfsDeleteWorker extends GridGgfsThread {
      *
      * @param parentId Parent ID.
      * @param id Entry id.
-     * @throws GridException If delete failed for some reason.
+     * @throws IgniteCheckedException If delete failed for some reason.
      */
-    private void deleteDirectory(IgniteUuid parentId, IgniteUuid id) throws GridException {
+    private void deleteDirectory(IgniteUuid parentId, IgniteUuid id) throws IgniteCheckedException {
         assert parentId != null;
         assert id != null;
 
@@ -334,7 +334,7 @@ public class GridGgfsDeleteWorker extends GridGgfsThread {
             try {
                 ggfsCtx.send(node, topic, msg0, GridIoPolicy.SYSTEM_POOL);
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 U.warn(log, "Failed to send GGFS delete message to node [nodeId=" + node.id() +
                     ", msg=" + msg + ", 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/ggfs/GridGgfsEx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsEx.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsEx.java
index 7ee45f3..2e78702 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsEx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsEx.java
@@ -40,30 +40,30 @@ public interface GridGgfsEx extends IgniteFs {
 
     /** {@inheritDoc} */
     @Override GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize, int seqReadsBeforePrefetch)
-        throws GridException;
+        throws IgniteCheckedException;
 
     /** {@inheritDoc} */
-    @Override GridGgfsInputStreamAdapter open(IgniteFsPath path) throws GridException;
+    @Override GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteCheckedException;
 
     /** {@inheritDoc} */
-    @Override GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws GridException;
+    @Override GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteCheckedException;
 
     /**
      * Gets global space counters.
      *
      * @return Tuple in which first component is used space on all nodes,
      *      second is available space on all nodes.
-     * @throws GridException If task execution failed.
+     * @throws IgniteCheckedException If task execution failed.
      */
-    public GridGgfsStatus globalSpace() throws GridException;
+    public GridGgfsStatus globalSpace() throws IgniteCheckedException;
 
     /**
      * Enables, disables or clears sampling flag.
      *
      * @param val {@code True} to turn on sampling, {@code false} to turn it off, {@code null} to clear sampling state.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void globalSampling(@Nullable Boolean val) throws GridException;
+    public void globalSampling(@Nullable Boolean val) throws IgniteCheckedException;
 
     /**
      * Get sampling state.
@@ -91,9 +91,9 @@ public interface GridGgfsEx extends IgniteFs {
      * Asynchronously await for all entries existing in trash to be removed.
      *
      * @return Future which will be completed when all entries existed in trash by the time of invocation are removed.
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public IgniteFuture<?> awaitDeletesAsync() throws GridException;
+    public IgniteFuture<?> awaitDeletesAsync() throws IgniteCheckedException;
 
     /**
      * Gets client file system log directory.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileMap.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileMap.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileMap.java
index 362a8a7..5824a7f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileMap.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileMap.java
@@ -9,11 +9,11 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
+import org.apache.ignite.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
+import org.gridgain.grid.util.tostring.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.grid.util.tostring.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -116,9 +116,9 @@ public class GridGgfsFileMap implements Externalizable {
      *
      * @param range Range to update status.
      * @param status New range status.
-     * @throws GridException If range was not found.
+     * @throws IgniteCheckedException If range was not found.
      */
-    public void updateRangeStatus(GridGgfsFileAffinityRange range, int status) throws GridException {
+    public void updateRangeStatus(GridGgfsFileAffinityRange range, int status) throws IgniteCheckedException {
         if (ranges == null)
             throw new GridGgfsInvalidRangeException("Failed to update range status (file map is empty) " +
                 "[range=" + range + ", ranges=" + ranges + ']');
@@ -178,7 +178,7 @@ public class GridGgfsFileMap implements Externalizable {
      *
      * @param range Range to delete.
      */
-    public void deleteRange(GridGgfsFileAffinityRange range) throws GridException {
+    public void deleteRange(GridGgfsFileAffinityRange range) throws IgniteCheckedException {
         if (ranges == null)
             throw new GridGgfsInvalidRangeException("Failed to remove range (file map is empty) " +
                 "[range=" + range + ", ranges=" + ranges + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerBatch.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerBatch.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerBatch.java
index ba9ecc8..a0a82fc 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerBatch.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerBatch.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
+import org.apache.ignite.*;
 import org.apache.ignite.fs.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -41,7 +42,7 @@ public class GridGgfsFileWorkerBatch {
     private final OutputStream out;
 
     /** Caught exception. */
-    private volatile GridException err;
+    private volatile IgniteCheckedException err;
 
     /** Last task marker. */
     private boolean lastTask;
@@ -68,12 +69,12 @@ public class GridGgfsFileWorkerBatch {
      */
     boolean write(final byte[] data) {
         return addTask(new GridGgfsFileWorkerTask() {
-            @Override public void execute() throws GridException {
+            @Override public void execute() throws IgniteCheckedException {
                 try {
                     out.write(data);
                 }
                 catch (IOException e) {
-                    throw new GridException("Failed to write data to the file due to secondary file system " +
+                    throw new IgniteCheckedException("Failed to write data to the file due to secondary file system " +
                         "exception: " + path, e);
                 }
             }
@@ -99,7 +100,7 @@ public class GridGgfsFileWorkerBatch {
                     if (lastTask)
                         cancelled = true;
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     err = e;
 
                     cancelled = true;
@@ -149,9 +150,9 @@ public class GridGgfsFileWorkerBatch {
     /**
      * Await for that worker batch to complete.
      *
-     * @throws GridException In case any exception has occurred during batch tasks processing.
+     * @throws IgniteCheckedException In case any exception has occurred during batch tasks processing.
      */
-    void await() throws GridException {
+    void await() throws IgniteCheckedException {
         try {
             completeLatch.await();
         }
@@ -161,7 +162,7 @@ public class GridGgfsFileWorkerBatch {
             throw new GridInterruptedException(e);
         }
 
-        GridException err0 = err;
+        IgniteCheckedException err0 = err;
 
         if (err0 != null)
             throw err0;
@@ -170,9 +171,9 @@ public class GridGgfsFileWorkerBatch {
     /**
      * Await for that worker batch to complete in case it was marked as finished.
      *
-     * @throws GridException In case any exception has occurred during batch tasks processing.
+     * @throws IgniteCheckedException In case any exception has occurred during batch tasks processing.
      */
-    void awaitIfFinished() throws GridException {
+    void awaitIfFinished() throws IgniteCheckedException {
         if (finishGuard.get())
             await();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerTask.java
index 7665e6f..ed3df89 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerTask.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileWorkerTask.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 
 /**
  * Generic GGFS worker task which could potentially throw an exception.
@@ -18,7 +18,7 @@ public interface GridGgfsFileWorkerTask {
     /**
      * Execute task logic.
      *
-     * @throws GridException If failed.
+     * @throws IgniteCheckedException If failed.
      */
-    public void execute() throws GridException;
+    public void execute() throws IgniteCheckedException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java
index 08b922a..5295f80 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFragmentizerManager.java
@@ -9,6 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
+import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.events.*;
 import org.apache.ignite.lang.*;
@@ -16,9 +17,9 @@ import org.apache.ignite.thread.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.kernal.managers.communication.*;
 import org.gridgain.grid.kernal.managers.eventstorage.*;
+import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
-import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.worker.*;
 import org.jetbrains.annotations.*;
 
@@ -68,7 +69,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
     private Object topic;
 
     /** {@inheritDoc} */
-    @Override protected void start0() throws GridException {
+    @Override protected void start0() throws IgniteCheckedException {
         if (!ggfsCtx.configuration().isFragmentizerEnabled())
             return;
 
@@ -95,7 +96,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
     }
 
     /** {@inheritDoc} */
-    @Override protected void onKernalStart0() throws GridException {
+    @Override protected void onKernalStart0() throws IgniteCheckedException {
         if (ggfsCtx.configuration().isFragmentizerEnabled()) {
             // Check at startup if this node is a fragmentizer coordinator.
             IgniteDiscoveryEvent locJoinEvt = ggfsCtx.kernalContext().discovery().localJoinEvent();
@@ -149,16 +150,16 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
     /**
      * @param nodeId Node ID to send message to.
      * @param msg Message to send.
-     * @throws GridException If send failed.
+     * @throws IgniteCheckedException If send failed.
      */
-    private void sendWithRetries(UUID nodeId, GridGgfsCommunicationMessage msg) throws GridException {
+    private void sendWithRetries(UUID nodeId, GridGgfsCommunicationMessage msg) throws IgniteCheckedException {
         for (int i = 0; i < MESSAGE_SEND_RETRY_COUNT; i++) {
             try {
                 ggfsCtx.send(nodeId, topic, msg, SYSTEM_POOL);
 
                 return;
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (!ggfsCtx.kernalContext().discovery().alive(nodeId))
                     throw new ClusterTopologyException("Failed to send message (node left the grid) " +
                         "[nodeId=" + nodeId + ", msg=" + msg + ']');
@@ -227,10 +228,10 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
      * </ul>
      *
      * @param req Request.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
     @SuppressWarnings("fallthrough")
-    private void processFragmentizerRequest(GridGgfsFragmentizerRequest req) throws GridException {
+    private void processFragmentizerRequest(GridGgfsFragmentizerRequest req) throws IgniteCheckedException {
         req.finishUnmarshal(ggfsCtx.kernalContext().config().getMarshaller(), null);
 
         Collection<GridGgfsFileAffinityRange> ranges = req.fragmentRanges();
@@ -312,7 +313,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
     private IgniteClosure<GridGgfsFileInfo, GridGgfsFileInfo> updateRange(final GridGgfsFileAffinityRange range,
         final int status) {
         return new CX1<GridGgfsFileInfo, GridGgfsFileInfo>() {
-            @Override public GridGgfsFileInfo applyx(GridGgfsFileInfo info) throws GridException {
+            @Override public GridGgfsFileInfo applyx(GridGgfsFileInfo info) throws IgniteCheckedException {
                 GridGgfsFileMap map = new GridGgfsFileMap(info.fileMap());
 
                 map.updateRangeStatus(range, status);
@@ -338,7 +339,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
      */
     private IgniteClosure<GridGgfsFileInfo, GridGgfsFileInfo> deleteRange(final GridGgfsFileAffinityRange range) {
         return new CX1<GridGgfsFileInfo, GridGgfsFileInfo>() {
-            @Override public GridGgfsFileInfo applyx(GridGgfsFileInfo info) throws GridException {
+            @Override public GridGgfsFileInfo applyx(GridGgfsFileInfo info) throws IgniteCheckedException {
                 GridGgfsFileMap map = new GridGgfsFileMap(info.fileMap());
 
                 map.deleteRange(range);
@@ -400,7 +401,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
                         requestFragmenting(fileInfo);
                     }
                 }
-                catch (GridException | GridRuntimeException e) {
+                catch (IgniteCheckedException | IgniteException e) {
                     if (!X.hasCause(e, InterruptedException.class) && !X.hasCause(e, GridInterruptedException.class))
                         LT.warn(log, e, "Failed to get fragmentizer file info (will retry).");
                     else {
@@ -567,7 +568,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
                         if (!ggfsCtx.kernalContext().discovery().alive(nodeId))
                             startSync0.remove(nodeId);
                     }
-                    catch (GridException e) {
+                    catch (IgniteCheckedException e) {
                         if (e.hasCause(ClusterTopologyException.class)) {
                             if (log.isDebugEnabled())
                                 log.debug("Failed to send sync message to remote node (node has left the grid): " +
@@ -643,7 +644,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
 
                     sendWithRetries(nodeId, msg);
                 }
-                catch (GridException e) {
+                catch (IgniteCheckedException e) {
                     if (e.hasCause(ClusterTopologyException.class)) {
                         if (log.isDebugEnabled())
                             log.debug("Failed to send fragmentizer request to remote node (node left grid): " +
@@ -671,9 +672,9 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
      *
      * @param exclude File IDs to exclude (the ones that are currently being processed).
      * @return File ID to process or {@code null} if there are no such files.
-     * @throws GridException In case of error.
+     * @throws IgniteCheckedException In case of error.
      */
-    @Nullable private GridGgfsFileInfo fileForFragmentizer(Collection<IgniteUuid> exclude) throws GridException {
+    @Nullable private GridGgfsFileInfo fileForFragmentizer(Collection<IgniteUuid> exclude) throws IgniteCheckedException {
         return fragmentizerEnabled ? ggfsCtx.meta().fileForFragmentizer(exclude) : null;
     }
 
@@ -738,7 +739,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
                         try {
                             processFragmentizerRequest(fragmentizerReq);
                         }
-                        catch (GridException e) {
+                        catch (IgniteCheckedException e) {
                             if (e.hasCause(ClusterTopologyException.class)) {
                                 if (log.isDebugEnabled())
                                     log.debug("Failed to process fragmentizer request (remote node left the grid) " +
@@ -784,7 +785,7 @@ public class GridGgfsFragmentizerManager extends GridGgfsManager {
             try {
                 sendWithRetries(nodeId, msg);
             }
-            catch (GridException e) {
+            catch (IgniteCheckedException e) {
                 if (e.hasCause(ClusterTopologyException.class)) {
                     if (log.isDebugEnabled())
                         log.debug("Failed to send sync response to GGFS fragmentizer coordinator " +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelper.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelper.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelper.java
index d0c83ed..9286557 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelper.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelper.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.cache.*;
 
 /**
@@ -27,9 +27,9 @@ public interface GridGgfsHelper {
      * Validate cache configuration for GGFS.
      *
      * @param cfg Cache configuration.
-     * @throws org.gridgain.grid.GridException If validation failed.
+     * @throws IgniteCheckedException If validation failed.
      */
-    public abstract void validateCacheConfiguration(GridCacheConfiguration cfg) throws GridException;
+    public abstract void validateCacheConfiguration(GridCacheConfiguration cfg) throws IgniteCheckedException;
 
     /**
      * Check whether object is of type {@code GridGgfsBlockKey}

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/06931b4b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelperImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelperImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelperImpl.java
index 7de4a0e..1749f87 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelperImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsHelperImpl.java
@@ -9,7 +9,7 @@
 
 package org.gridgain.grid.kernal.processors.ggfs;
 
-import org.gridgain.grid.*;
+import org.apache.ignite.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.eviction.*;
 import org.gridgain.grid.cache.eviction.ggfs.*;
@@ -27,14 +27,14 @@ public class GridGgfsHelperImpl implements GridGgfsHelper {
     }
 
     /** {@inheritDoc} */
-    @Override public void validateCacheConfiguration(GridCacheConfiguration cfg) throws GridException {
+    @Override public void validateCacheConfiguration(GridCacheConfiguration cfg) throws IgniteCheckedException {
         GridCacheEvictionPolicy evictPlc =  cfg.getEvictionPolicy();
 
         if (evictPlc != null && evictPlc instanceof GridCacheGgfsPerBlockLruEvictionPolicy) {
             GridCacheEvictionFilter evictFilter = cfg.getEvictionFilter();
 
             if (evictFilter != null && !(evictFilter instanceof GridCacheGgfsEvictionFilter))
-                throw new GridException("Eviction filter cannot be set explicitly when using " +
+                throw new IgniteCheckedException("Eviction filter cannot be set explicitly when using " +
                     "GridCacheGgfsPerBlockLruEvictionPolicy:" + cfg.getName());
         }
     }