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/16 16:13:19 UTC

[08/13] incubator-ignite git commit: # IGNITE-26 Replaced IgniteCheckedException with IgniteException on public API.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java
index 176fdd0..76b42cf 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxFinishFuture.java
@@ -12,14 +12,13 @@ package org.gridgain.grid.kernal.processors.cache.distributed.near;
 import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.kernal.processors.cache.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.*;
-import org.gridgain.grid.util.typedef.*;
-import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.grid.util.future.*;
 import org.gridgain.grid.util.tostring.*;
+import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.io.*;
@@ -173,7 +172,7 @@ public final class GridNearTxFinishFuture<K, V> extends GridCompoundIdentityFutu
                 try {
                     tx.close();
                 }
-                catch (IgniteCheckedException ex) {
+                catch (IgniteException ex) {
                     U.error(log, "Failed to invalidate transaction: " + tx, ex);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
index 5ff258f..1c6ca83 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridNearTxLocal.java
@@ -792,7 +792,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
                 if (prepFut != null)
                     prepFut.get();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (log.isDebugEnabled())
                     log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']');
             }
@@ -814,7 +814,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
                         // Check for errors in prepare future.
                         f.get();
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         if (log.isDebugEnabled())
                             log.debug("Got optimistic tx failure [tx=" + this + ", err=" + e + ']');
                     }
@@ -955,7 +955,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
                 fut.onError(e);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Failed to prepare transaction: " + this, e);
 
                 fut.onError(e);
@@ -975,7 +975,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
 
                         fut.onError(e);
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         U.error(log, "Failed to prepare transaction: " + this, e);
 
                         fut.onError(e);
@@ -1006,7 +1006,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
                 if (prep != null)
                     prep.get();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (log.isDebugEnabled())
                     log.debug("Failed to prepare transaction during rollback (will ignore) [tx=" + this + ", msg=" +
                         e.getMessage() + ']');
@@ -1020,7 +1020,7 @@ public class GridNearTxLocal<K, V> extends GridDhtTxLocalAdapter<K, V> {
                     try {
                         f.get(); // Check for errors of a parent future.
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         log.debug("Failed to prepare transaction during rollback (will ignore) [tx=" + this + ", msg=" +
                             e.getMessage() + ']');
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
index b3d658b..3356665 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryFuture.java
@@ -72,7 +72,7 @@ public class GridCacheDistributedQueryFuture<K, V, R> extends GridCacheQueryFutu
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
+    @Override protected void cancelQuery() throws IgniteException {
         final GridCacheQueryManager<K, V> qryMgr = cctx.queries();
 
         assert qryMgr != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
index df55b83..0a6cb46 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheLocalQueryFuture.java
@@ -58,7 +58,7 @@ public class GridCacheLocalQueryFuture<K, V, R> extends GridCacheQueryFutureAdap
     }
 
     /** {@inheritDoc} */
-    @Override protected void cancelQuery() throws IgniteCheckedException {
+    @Override protected void cancelQuery() throws IgniteException {
         if (fut != null)
             fut.cancel();
     }

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

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java
index 797e31a..cb70fb6 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryFutureAdapter.java
@@ -455,7 +455,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<R> get() throws IgniteCheckedException {
+    @Override public Collection<R> get() throws IgniteException {
         if (!isDone())
             loadAllPages();
 
@@ -463,7 +463,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteCheckedException {
+    @Override public Collection<R> get(long timeout, TimeUnit unit) throws IgniteException {
         if (!isDone())
             loadAllPages();
 
@@ -498,7 +498,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /** {@inheritDoc} */
-    @Override public boolean cancel() throws IgniteCheckedException {
+    @Override public boolean cancel() throws IgniteException {
         if (onCancelled()) {
             cancelQuery();
 
@@ -509,9 +509,9 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
     }
 
     /**
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    protected abstract void cancelQuery() throws IgniteCheckedException;
+    protected abstract void cancelQuery() throws IgniteException;
 
     /** {@inheritDoc} */
     @Override public IgniteUuid timeoutId() {
@@ -530,7 +530,7 @@ public abstract class GridCacheQueryFutureAdapter<K, V, R> extends GridFutureAda
 
             onDone(new IgniteFutureTimeoutException("Query timed out."));
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             onDone(e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
index 25c0668..b6745e4 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheQueryManager.java
@@ -1397,7 +1397,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         try {
                             e.getValue().get().closeIfNotShared(recipient(sndId, e.getKey()));
                         }
-                        catch (IgniteCheckedException ex) {
+                        catch (IgniteException ex) {
                             U.error(log, "Failed to close query iterator.", ex);
                         }
                     }
@@ -1482,7 +1482,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 try {
                     fut.get().closeIfNotShared(recipient(sndId, reqId));
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     U.error(log, "Failed to close iterator.", e);
                 }
             }
@@ -1523,7 +1523,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                         try {
                             e.getValue().get().closeIfNotShared(recipient(sndId, e.getKey()));
                         }
-                        catch (IgniteCheckedException ex) {
+                        catch (IgniteException ex) {
                             U.error(log, "Failed to close fields query iterator.", ex);
                         }
                     }
@@ -1610,7 +1610,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
                 try {
                     fut.get().closeIfNotShared(recipient(sndId, reqId));
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     U.error(log, "Failed to close iterator.", e);
                 }
             }
@@ -2255,42 +2255,32 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         /**
          * @return Value.
-         * @throws IgniteCheckedException If failed.
+         * @throws IgniteException If failed.
          */
-        protected abstract V unmarshalValue() throws IgniteCheckedException;
+        protected abstract V unmarshalValue() throws IgniteException;
 
         /**
          * @return Key.
          */
         K key() {
-            try {
-                if (key != null)
-                    return key;
+            if (key != null)
+                return key;
 
-                key = cctx.marshaller().unmarshal(keyBytes(), cctx.deploy().globalLoader());
+            key = cctx.marshaller().unmarshal(keyBytes(), cctx.deploy().globalLoader());
 
-                return key;
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
+            return key;
         }
 
         /**
          * @return Value.
          */
         V value() {
-            try {
-                if (val != null)
-                    return val;
+            if (val != null)
+                return val;
 
-                val = unmarshalValue();
+            val = unmarshalValue();
 
-                return val;
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
+            return val;
         }
 
         /**
@@ -2330,7 +2320,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
 
         /** {@inheritDoc} */
         @SuppressWarnings("IfMayBeConditional")
-        @Override protected V unmarshalValue() throws IgniteCheckedException {
+        @Override protected V unmarshalValue() throws IgniteException {
             byte[] bytes = e.getValue();
 
             byte[] val = GridCacheSwapEntryImpl.getValueIfByteArray(bytes);
@@ -2393,7 +2383,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
         }
 
         /** {@inheritDoc} */
-        @Override protected V unmarshalValue() throws IgniteCheckedException {
+        @Override protected V unmarshalValue() throws IgniteException {
             long ptr = GridCacheOffheapSwapEntry.valueAddress(valPtr.get1(), valPtr.get2());
 
             V val = (V)cctx.portable().unmarshal(ptr, false);
@@ -2902,9 +2892,9 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
          * Close if this result does not have any other recipients.
          *
          * @param recipient ID of the recipient.
-         * @throws IgniteCheckedException If failed.
+         * @throws IgniteException If failed.
          */
-        public void closeIfNotShared(Object recipient) throws IgniteCheckedException {
+        public void closeIfNotShared(Object recipient) throws IgniteException {
             assert isDone();
 
             synchronized (recipients) {
@@ -3019,7 +3009,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             }
 
             /** {@inheritDoc} */
-            @Override public void close() throws IgniteCheckedException {
+            @Override public void close() throws IgniteException {
                 closeIfNotShared(recipient);
             }
 
@@ -3040,14 +3030,7 @@ public abstract class GridCacheQueryManager<K, V> extends GridCacheManagerAdapte
             private boolean fillNext() {
                 assert next.isEmpty();
 
-                IgniteSpiCloseableIterator<R> it;
-
-                try {
-                    it = get();
-                }
-                catch (IgniteCheckedException e) {
-                    throw new IgniteException(e);
-                }
+                IgniteSpiCloseableIterator<R> it = get();
 
                 synchronized (recipients) {
                     for (int i = 0; i < NEXT_SIZE; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
index 9b88858..4a8408b 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/continuous/GridCacheContinuousQueryAdapter.java
@@ -285,7 +285,7 @@ public class GridCacheContinuousQueryAdapter<K, V> implements GridCacheContinuou
     }
 
     /** {@inheritDoc} */
-    @Override public void close() throws IgniteCheckedException {
+    @Override public void close() throws IgniteException {
         closeLock.lock();
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
index 0f64ab8..49a8f19 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcMetadataTask.java
@@ -39,7 +39,7 @@ public class GridCacheQueryJdbcMetadataTask extends ComputeTaskAdapter<String, b
 
     /** {@inheritDoc} */
     @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid,
-        @Nullable String cacheName) throws IgniteCheckedException {
+        @Nullable String cacheName) throws IgniteException {
         Map<JdbcDriverMetadataJob, ClusterNode> map = new HashMap<>();
 
         for (ClusterNode n : subgrid)
@@ -53,7 +53,7 @@ public class GridCacheQueryJdbcMetadataTask extends ComputeTaskAdapter<String, b
     }
 
     /** {@inheritDoc} */
-    @Override public byte[] reduce(List<ComputeJobResult> results) throws IgniteCheckedException {
+    @Override public byte[] reduce(List<ComputeJobResult> results) throws IgniteException {
         return F.first(results).getData();
     }
 
@@ -92,7 +92,7 @@ public class GridCacheQueryJdbcMetadataTask extends ComputeTaskAdapter<String, b
         }
 
         /** {@inheritDoc} */
-        @Override public Object execute() throws IgniteCheckedException {
+        @Override public Object execute() throws IgniteException {
             byte status;
             byte[] data;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
index f70716f..70866e6 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcTask.java
@@ -53,7 +53,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
     private static final ScheduledExecutorService SCHEDULER = Executors.newScheduledThreadPool(1);
 
     /** {@inheritDoc} */
-    @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, byte[] arg) throws IgniteCheckedException {
+    @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, byte[] arg) throws IgniteException {
         assert arg != null;
 
         Map<String, Object> args = MARSHALLER.unmarshal(arg, null);
@@ -73,7 +73,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
                 if (n.id().equals(nodeId))
                     return F.asMap(new JdbcDriverJob(args, first), n);
 
-            throw new IgniteCheckedException("Node doesn't exist or left the grid: " + nodeId);
+            throw new IgniteException("Node doesn't exist or left the grid: " + nodeId);
         }
         else {
             String cache = (String)args.get("cache");
@@ -82,12 +82,12 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
                 if (U.hasCache(n, cache))
                     return F.asMap(new JdbcDriverJob(args, first), n);
 
-            throw new IgniteCheckedException("Can't find node with cache: " + cache);
+            throw new IgniteException("Can't find node with cache: " + cache);
         }
     }
 
     /** {@inheritDoc} */
-    @Override public byte[] reduce(List<ComputeJobResult> results) throws IgniteCheckedException {
+    @Override public byte[] reduce(List<ComputeJobResult> results) throws IgniteException {
         byte status;
         byte[] bytes;
 
@@ -114,7 +114,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
     }
 
     /** {@inheritDoc} */
-    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteCheckedException {
+    @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
         return WAIT;
     }
 
@@ -161,7 +161,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
         }
 
         /** {@inheritDoc} */
-        @Override public Object execute() throws IgniteCheckedException {
+        @Override public Object execute() throws IgniteException {
             String cacheName = argument("cache");
             String sql = argument("sql");
             Long timeout = argument("timeout");
@@ -206,7 +206,7 @@ public class GridCacheQueryJdbcTask extends ComputeTaskAdapter<byte[], byte[]> {
                     try {
                         fut.get();
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         if (e.hasCause(SQLException.class))
                             throw new GridInternalException(e.getCause(SQLException.class).getMessage(), e);
                     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java
index 0f3b4d9..6d436ab 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/jdbc/GridCacheQueryJdbcValidationTask.java
@@ -32,7 +32,7 @@ public class GridCacheQueryJdbcValidationTask extends ComputeTaskSplitAdapter<St
 
     /** {@inheritDoc} */
     @Override protected Collection<? extends ComputeJob> split(int gridSize,
-        @Nullable final String cacheName) throws IgniteCheckedException {
+        @Nullable final String cacheName) throws IgniteException {
         // Register big data usage.
         GridLicenseUseRegistry.onUsage(DATA_GRID, getClass());
 
@@ -51,7 +51,7 @@ public class GridCacheQueryJdbcValidationTask extends ComputeTaskSplitAdapter<St
     }
 
     /** {@inheritDoc} */
-    @Override public Boolean reduce(List<ComputeJobResult> results) throws IgniteCheckedException {
+    @Override public Boolean reduce(List<ComputeJobResult> results) throws IgniteException {
         return F.first(results).getData();
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java
index f19327e..c4c2345 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessor.java
@@ -195,11 +195,11 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param jobs Closures to map.
      * @param nodes Grid nodes.
      * @param lb Load balancer.
-     * @throws IgniteCheckedException Thrown in case of any errors.
+     * @throws IgniteException Thrown in case of any errors.
      * @return Mapping.
      */
     private Map<ComputeJob, ClusterNode> absMap(GridClosureCallMode mode, Collection<? extends Runnable> jobs,
-        Collection<ClusterNode> nodes, ComputeLoadBalancer lb) throws IgniteCheckedException {
+        Collection<ClusterNode> nodes, ComputeLoadBalancer lb) throws IgniteException {
         assert mode != null;
         assert jobs != null;
         assert nodes != null;
@@ -243,12 +243,12 @@ public class GridClosureProcessor extends GridProcessorAdapter {
      * @param jobs Closures to map.
      * @param nodes Grid nodes.
      * @param lb Load balancer.
-     * @throws IgniteCheckedException Thrown in case of any errors.
+     * @throws IgniteException Thrown in case of any errors.
      * @return Mapping.
      */
     private <R> Map<ComputeJob, ClusterNode> outMap(GridClosureCallMode mode,
         Collection<? extends Callable<R>> jobs, Collection<ClusterNode> nodes, ComputeLoadBalancer lb)
-        throws IgniteCheckedException {
+        throws IgniteException {
         assert mode != null;
         assert jobs != null;
         assert nodes != null;
@@ -969,7 +969,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     return job.apply(arg);
                 }
 
-                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException {
+                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException {
                     ((ComputeJobMasterLeaveAware)job).onMasterNodeLeft(ses);
                 }
             };
@@ -1004,7 +1004,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     }
                 }
 
-                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException {
+                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException {
                     ((ComputeJobMasterLeaveAware)c).onMasterNodeLeft(ses);
                 }
             };
@@ -1054,7 +1054,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     }
                 }
 
-                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException {
+                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException {
                     ((ComputeJobMasterLeaveAware)c).onMasterNodeLeft(ses);
                 }
             };
@@ -1099,7 +1099,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     return null;
                 }
 
-                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException {
+                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException {
                     ((ComputeJobMasterLeaveAware)r).onMasterNodeLeft(ses);
                 }
             };
@@ -1143,7 +1143,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
                     return null;
                 }
 
-                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteCheckedException {
+                @Override public void onMasterNodeLeft(ComputeTaskSession ses) throws IgniteException {
                     ((ComputeJobMasterLeaveAware)r).onMasterNodeLeft(ses);
                 }
             };
@@ -1188,9 +1188,9 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         /**
          * @param job Job.
          * @param node Node.
-         * @throws IgniteCheckedException In case of error.
+         * @throws IgniteException In case of error.
          */
-        public void map(ComputeJob job, ClusterNode node) throws IgniteCheckedException {
+        public void map(ComputeJob job, ClusterNode node) throws IgniteException {
             assert job != null;
             assert node != null;
 
@@ -1223,7 +1223,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public Void reduce(List<ComputeJobResult> results) throws IgniteCheckedException {
+        @Nullable @Override public Void reduce(List<ComputeJobResult> results) throws IgniteException {
             return null;
         }
     }
@@ -1258,7 +1258,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             return absMap(t.get1(), t.get2(), subgrid, lb);
         }
     }
@@ -1290,7 +1290,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             return absMap(t.get1(), F.asList(t.get2()), subgrid, lb);
         }
     }
@@ -1331,13 +1331,13 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             return outMap(t.get1(), t.get2(), subgrid, lb);
         }
 
         /** {@inheritDoc} */
         @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd)
-            throws IgniteCheckedException {
+            throws IgniteException {
             ComputeJobResultPolicy resPlc = super.result(res, rcvd);
 
             if (res.getException() == null && resPlc != FAILOVER && !t.get3().collect((R1)res.getData()))
@@ -1386,7 +1386,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             ComputeJob job = job(this.job, cacheName, affKey);
 
             return Collections.singletonMap(job, lb.getBalancedNode(job, null));
@@ -1427,20 +1427,20 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             ComputeJob job = job(this.job, cacheName, affKey);
 
             return Collections.singletonMap(job, lb.getBalancedNode(job, null));
         }
 
         /** {@inheritDoc} */
-        @Override public R reduce(List<ComputeJobResult> res) throws IgniteCheckedException {
+        @Override public R reduce(List<ComputeJobResult> res) throws IgniteException {
             for (ComputeJobResult r : res) {
                 if (r.getException() == null)
                     return r.getData();
             }
 
-            throw new IgniteCheckedException("Failed to find successful job result: " + res);
+            throw new IgniteException("Failed to find successful job result: " + res);
         }
     }
 
@@ -1478,7 +1478,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             return outMap(mode, jobs, subgrid, lb);
         }
 
@@ -1515,17 +1515,17 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             return outMap(t.get1(), F.asList(t.get2()), subgrid, lb);
         }
 
         /** {@inheritDoc} */
-        @Override public R reduce(List<ComputeJobResult> res) throws IgniteCheckedException {
+        @Override public R reduce(List<ComputeJobResult> res) throws IgniteException {
             for (ComputeJobResult r : res)
                 if (r.getException() == null)
                     return r.getData();
 
-            throw new IgniteCheckedException("Failed to find successful job result: " + res);
+            throw new IgniteException("Failed to find successful job result: " + res);
         }
     }
 
@@ -1558,19 +1558,19 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             ComputeJob job = job(this.job, this.arg);
 
             return Collections.singletonMap(job, lb.getBalancedNode(job, null));
         }
 
         /** {@inheritDoc} */
-        @Override public R reduce(List<ComputeJobResult> res) throws IgniteCheckedException {
+        @Override public R reduce(List<ComputeJobResult> res) throws IgniteException {
             for (ComputeJobResult r : res)
                 if (r.getException() == null)
                     return r.getData();
 
-            throw new IgniteCheckedException("Failed to find successful job result: " + res);
+            throw new IgniteException("Failed to find successful job result: " + res);
         }
     }
 
@@ -1603,7 +1603,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             Map<ComputeJob, ClusterNode> map = new HashMap<>(args.size(), 1);
 
             JobMapper mapper = new JobMapper(map);
@@ -1618,7 +1618,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public Collection<R> reduce(List<ComputeJobResult> res) throws IgniteCheckedException {
+        @Override public Collection<R> reduce(List<ComputeJobResult> res) throws IgniteException {
             return F.jobResults(res);
         }
     }
@@ -1657,7 +1657,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             Map<ComputeJob, ClusterNode> map = new HashMap<>(args.size(), 1);
 
             JobMapper mapper = new JobMapper(map);
@@ -1673,7 +1673,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd)
-            throws IgniteCheckedException {
+            throws IgniteException {
             ComputeJobResultPolicy resPlc = super.result(res, rcvd);
 
             if (res.getException() == null && resPlc != FAILOVER && !rdc.collect((R1) res.getData()))
@@ -1683,7 +1683,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
         }
 
         /** {@inheritDoc} */
-        @Override public R2 reduce(List<ComputeJobResult> res) throws IgniteCheckedException {
+        @Override public R2 reduce(List<ComputeJobResult> res) throws IgniteException {
             return rdc.reduce();
         }
     }
@@ -1714,7 +1714,7 @@ public class GridClosureProcessor extends GridProcessorAdapter {
 
         /** {@inheritDoc} */
         @Override public Map<? extends ComputeJob, ClusterNode> map(List<ClusterNode> subgrid, @Nullable Void arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             if (F.isEmpty(subgrid))
                 return Collections.emptyMap();
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java
index 14d89f3..f4e2a72 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/continuous/GridContinuousProcessor.java
@@ -222,7 +222,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                     try {
                         msg.data(marsh.unmarshal(msg.dataBytes(), null));
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         U.error(log, "Failed to process message (ignoring): " + msg, e);
 
                         return;
@@ -397,7 +397,7 @@ public class GridContinuousProcessor extends GridProcessorAdapter {
                         try {
                             msg.data(marsh.unmarshal(msg.dataBytes(), null));
                         }
-                        catch (IgniteCheckedException e) {
+                        catch (IgniteException e) {
                             U.error(log, "Failed to process message (ignoring): " + msg, e);
 
                             return;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java
index 792895d..3a8449c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderFuture.java
@@ -49,7 +49,7 @@ class GridDataLoaderFuture extends GridFutureAdapter<Object> {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean cancel() throws IgniteCheckedException {
+    @Override public boolean cancel() throws IgniteException {
         checkValid();
 
         if (onCancelled()) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java
index 10f0487..7d10334 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/dataload/GridDataLoaderProcessor.java
@@ -123,7 +123,7 @@ public class GridDataLoaderProcessor<K, V> extends GridProcessorAdapter {
             catch (GridInterruptedException e) {
                 U.warn(log, "Interrupted while waiting for completion of the data loader: " + ldr, e);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Failed to close data loader: " + ldr, e);
             }
         }
@@ -193,7 +193,7 @@ public class GridDataLoaderProcessor<K, V> extends GridProcessorAdapter {
             try {
                 topic = marsh.unmarshal(req.responseTopicBytes(), null);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Failed to unmarshal topic from request: " + req, e);
 
                 return;
@@ -235,7 +235,7 @@ public class GridDataLoaderProcessor<K, V> extends GridProcessorAdapter {
                 col = marsh.unmarshal(req.collectionBytes(), clsLdr);
                 updater = marsh.unmarshal(req.updaterBytes(), clsLdr);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Failed to unmarshal message [nodeId=" + nodeId + ", req=" + req + ']', e);
 
                 sendResponse(nodeId, topic, req.requestId(), e, false);
@@ -278,7 +278,7 @@ public class GridDataLoaderProcessor<K, V> extends GridProcessorAdapter {
         try {
             errBytes = err != null ? marsh.marshal(err) : null;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             U.error(log, "Failed to marshal message.", e);
 
             return;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/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 e6ce07e..3a3833b 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 IgniteCheckedException {
+    @Override public void isolated(boolean isolated) throws IgniteException {
         if (isolated())
             return;
 
         ClusterNode node = F.first(ctx.grid().forCache(cacheName).nodes());
 
         if (node == null)
-            throw new IgniteCheckedException("Failed to get node for cache: " + cacheName);
+            throw new IgniteException("Failed to get node for cache: " + cacheName);
 
         GridCacheAttributes a = U.cacheAttributes(node, cacheName);
 
@@ -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 IgniteCheckedException, IllegalStateException {
+    @Override public IgniteFuture<?> addData(Map.Entry<K, V> entry) throws IgniteException, IllegalStateException {
         A.notNull(entry, "entry");
 
         return addData(F.asList(entry));
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> addData(K key, V val) throws IgniteCheckedException, IllegalStateException {
+    @Override public IgniteFuture<?> addData(K key, V val) throws IgniteException, IllegalStateException {
         A.notNull(key, "key");
 
         return addData(new Entry0<>(key, val));
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFuture<?> removeData(K key) throws IgniteCheckedException, IllegalStateException {
+    @Override public IgniteFuture<?> removeData(K key) throws IgniteException, 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 IgniteCheckedException("Failed to finish operation (too many remaps): " + remaps));
+            resFut.onDone(new IgniteException("Failed to finish operation (too many remaps): " + remaps));
 
             return;
         }
@@ -479,12 +479,12 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                         if (activeKeys.isEmpty())
                             resFut.onDone();
                     }
-                    catch (IgniteCheckedException e1) {
+                    catch (IgniteException e1) {
                         if (log.isDebugEnabled())
                             log.debug("Future finished with error [nodeId=" + nodeId + ", err=" + e1 + ']');
 
                         if (cancelled) {
-                            resFut.onDone(new IgniteCheckedException("Data loader has been cancelled: " +
+                            resFut.onDone(new IgniteException("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 IgniteCheckedException If failed.
+     * @throws IgniteException If failed.
      */
-    private void doFlush() throws IgniteCheckedException {
+    private void doFlush() throws IgniteException {
         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 (IgniteCheckedException e) {
+                    catch (IgniteException 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 IgniteCheckedException {
+    @Override public void flush() throws IgniteException {
         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 IgniteCheckedException If failed.
+     * @throws IgniteException If failed.
      */
-    @Override public void close(boolean cancel) throws IgniteCheckedException {
+    @Override public void close(boolean cancel) throws IgniteException {
         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 + ']');
 
-        IgniteCheckedException e = null;
+        IgniteException 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 (IgniteCheckedException e0) {
+        catch (IgniteException e0) {
             e = e0;
         }
 
@@ -686,7 +686,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
     }
 
     /** {@inheritDoc} */
-    @Override public void close() throws IgniteCheckedException {
+    @Override public void close() throws IgniteException {
         close(false);
     }
 
@@ -910,7 +910,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
 
                             curFut.onDone(t.get());
                         }
-                        catch (IgniteCheckedException e) {
+                        catch (IgniteException 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 (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     U.error(log, "Failed to marshal (request will not be sent).", e);
 
                     return;
@@ -1071,7 +1071,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                         errBytes,
                         jobPda0 != null ? jobPda0.classLoader() : U.gridClassLoader());
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     f.onDone(null, new IgniteCheckedException("Failed to unmarshal response.", e));
 
                     return;
@@ -1094,7 +1094,7 @@ public class IgniteDataLoaderImpl<K, V> implements IgniteDataLoader<K, V>, Delay
                 try {
                     f.cancel();
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     U.error(log, "Failed to cancel mini-future.", e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/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 d47c661..b5edd99 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
@@ -34,7 +34,7 @@ public class GridGgfsAckMessage extends GridGgfsCommunicationMessage {
 
     /** Write exception. */
     @GridDirectTransient
-    private IgniteCheckedException err;
+    private IgniteException 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 IgniteCheckedException err) {
+    public GridGgfsAckMessage(IgniteUuid fileId, long id, @Nullable IgniteException err) {
         this.fileId = fileId;
         this.id = id;
         this.err = err;
@@ -74,7 +74,7 @@ public class GridGgfsAckMessage extends GridGgfsCommunicationMessage {
     /**
      * @return Error occurred when writing this batch, if any.
      */
-    public IgniteCheckedException error() {
+    public IgniteException error() {
         return err;
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/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 e22c0b7..ee1b4a5 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 IgniteCheckedException {
+    @Override public void format() throws IgniteException {
         saveOrGet(ggfs.formatAsync());
     }
 
     /** {@inheritDoc} */
     @Override public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
-        Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException {
+        Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException {
         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 IgniteCheckedException {
+        throws IgniteException {
         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 IgniteCheckedException {
+        @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException {
         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 IgniteCheckedException {
+        long maxRangeLen, @Nullable T arg) throws IgniteException {
         return saveOrGet(ggfs.executeAsync(taskCls, rslvr, paths, skipNonExistentFiles, maxRangeLen, arg));
     }
 
@@ -88,17 +88,17 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid
 
     /** {@inheritDoc} */
     @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize,
-        int seqReadsBeforePrefetch) throws IgniteCheckedException {
+        int seqReadsBeforePrefetch) throws IgniteException {
         return ggfs.open(path, bufSize, seqReadsBeforePrefetch);
     }
 
     /** {@inheritDoc} */
-    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteException {
         return ggfs.open(path);
     }
 
     /** {@inheritDoc} */
-    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteCheckedException {
+    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteException {
         return ggfs.open(path, bufSize);
     }
 
@@ -168,118 +168,118 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteException {
         return ggfs.summary(path);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteCheckedException {
+    @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteException {
         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 IgniteCheckedException {
+        long blockSize, @Nullable Map<String, String> props) throws IgniteException {
         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 IgniteCheckedException {
+        throws IgniteException {
         return ggfs.create(path, bufSize, overwrite, affKey, replication, blockSize, props);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteCheckedException {
+    @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteException {
         return ggfs.append(path, create);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFsOutputStream append(IgniteFsPath path, int bufSize, boolean create,
-        @Nullable Map<String, String> props) throws IgniteCheckedException {
+        @Nullable Map<String, String> props) throws IgniteException {
         return ggfs.append(path, bufSize, create, props);
     }
 
     /** {@inheritDoc} */
-    @Override public void setTimes(IgniteFsPath path, long accessTime, long modificationTime) throws IgniteCheckedException {
+    @Override public void setTimes(IgniteFsPath path, long accessTime, long modificationTime) throws IgniteException {
         ggfs.setTimes(path, accessTime, modificationTime);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len)
-        throws IgniteCheckedException {
+        throws IgniteException {
         return ggfs.affinity(path, start, len);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len, long maxLen)
-        throws IgniteCheckedException {
+        throws IgniteException {
         return ggfs.affinity(path, start, len, maxLen);
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsMetrics metrics() throws IgniteCheckedException {
+    @Override public IgniteFsMetrics metrics() throws IgniteException {
         return ggfs.metrics();
     }
 
     /** {@inheritDoc} */
-    @Override public void resetMetrics() throws IgniteCheckedException {
+    @Override public void resetMetrics() throws IgniteException {
         ggfs.resetMetrics();
     }
 
     /** {@inheritDoc} */
-    @Override public long size(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public long size(IgniteFsPath path) throws IgniteException {
         return ggfs.size(path);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean exists(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public boolean exists(IgniteFsPath path) throws IgniteException {
         return ggfs.exists(path);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws IgniteCheckedException {
+    @Nullable @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws IgniteException {
         return ggfs.update(path, props);
     }
 
     /** {@inheritDoc} */
-    @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws IgniteCheckedException {
+    @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws IgniteException {
         ggfs.rename(src, dest);
     }
 
     /** {@inheritDoc} */
-    @Override public boolean delete(IgniteFsPath path, boolean recursive) throws IgniteCheckedException {
+    @Override public boolean delete(IgniteFsPath path, boolean recursive) throws IgniteException {
         return ggfs.delete(path, recursive);
     }
 
     /** {@inheritDoc} */
-    @Override public void mkdirs(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public void mkdirs(IgniteFsPath path) throws IgniteException {
         ggfs.mkdirs(path);
     }
 
     /** {@inheritDoc} */
-    @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws IgniteCheckedException {
+    @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws IgniteException {
         ggfs.mkdirs(path, props);
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgniteFsPath> listPaths(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public Collection<IgniteFsPath> listPaths(IgniteFsPath path) throws IgniteException {
         return ggfs.listPaths(path);
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgniteFsFile> listFiles(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public Collection<IgniteFsFile> listFiles(IgniteFsPath path) throws IgniteException {
         return ggfs.listFiles(path);
     }
 
     /** {@inheritDoc} */
-    @Nullable @Override public IgniteFsFile info(IgniteFsPath path) throws IgniteCheckedException {
+    @Nullable @Override public IgniteFsFile info(IgniteFsPath path) throws IgniteException {
         return ggfs.info(path);
     }
 
     /** {@inheritDoc} */
-    @Override public long usedSpaceSize() throws IgniteCheckedException {
+    @Override public long usedSpaceSize() throws IgniteException {
         return ggfs.usedSpaceSize();
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/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 a42f530..1c259ad 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
@@ -40,10 +40,10 @@ import java.util.concurrent.*;
 import java.util.concurrent.atomic.*;
 import java.util.concurrent.locks.*;
 
+import static org.apache.ignite.events.IgniteEventType.*;
 import static org.gridgain.grid.cache.GridCacheAtomicityMode.*;
 import static org.gridgain.grid.cache.GridCacheTxConcurrency.*;
 import static org.gridgain.grid.cache.GridCacheTxIsolation.*;
-import static org.apache.ignite.events.IgniteEventType.*;
 import static org.gridgain.grid.kernal.GridTopic.*;
 import static org.gridgain.grid.kernal.managers.communication.GridIoPolicy.*;
 import static org.gridgain.grid.kernal.processors.cache.GridCacheUtils.*;
@@ -127,12 +127,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
      */
     void awaitInit() {
         if (!dataCacheStartFut.isDone()) {
-            try {
-                dataCacheStartFut.get();
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
+            dataCacheStartFut.get();
         }
     }
 
@@ -644,7 +639,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                 }
             }
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             log.error("Failed to clean up file range [fileInfo=" + fileInfo + ", range=" + range + ']', e);
         }
     }
@@ -1028,7 +1023,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                     try {
                         ggfsCtx.send(nodeId, topic, msg, SYSTEM_POOL);
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         completionFut.onError(nodeId, e);
                     }
 
@@ -1046,7 +1041,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
 
                                 completionFut.onWriteAck(nodeId, batchId);
                             }
-                            catch (IgniteCheckedException e) {
+                            catch (IgniteException e) {
                                 completionFut.onError(nodeId, e);
                             }
                         }
@@ -1274,12 +1269,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) {
-                IgniteCheckedException err = null;
+                IgniteException err = null;
 
                 try {
                     fut.get();
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     err = e;
                 }
 
@@ -1715,7 +1710,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
                     catch (GridInterruptedException ignored) {
                         // Ignore interruption during shutdown.
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         log.error("Failed to remove file contents: " + fileInfo, e);
                     }
                     finally {
@@ -1726,14 +1721,14 @@ public class GridGgfsDataManager extends GridGgfsManager {
                                 ldr.removeData(new GridGgfsBlockKey(fileId, fileInfo.affinityKey(),
                                     fileInfo.evictExclude(), block));
                         }
-                        catch (IgniteCheckedException e) {
+                        catch (IgniteException e) {
                             log.error("Failed to remove file contents: " + fileInfo, e);
                         }
                         finally {
                             try {
                                 ldr.close(isCancelled());
                             }
-                            catch (IgniteCheckedException e) {
+                            catch (IgniteException e) {
                                 log.error("Failed to stop data loader while shutting down ggfs async delete thread.", e);
                             }
                             finally {
@@ -1828,7 +1823,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
          * @param nodeId Node ID.
          * @param e Caught exception.
          */
-        private void onError(UUID nodeId, IgniteCheckedException e) {
+        private void onError(UUID nodeId, IgniteException e) {
             Set<Long> reqIds = pendingAcks.get(nodeId);
 
             // If waiting for ack from this node.
@@ -1844,7 +1839,7 @@ public class GridGgfsDataManager extends GridGgfsManager {
         /**
          * @param e Error.
          */
-        private void onLocalError(IgniteCheckedException e) {
+        private void onLocalError(IgniteException e) {
             if (e instanceof IgniteFsOutOfSpaceException)
                 onDone(new IgniteCheckedException("Failed to write data (not enough space on node): " +
                     ggfsCtx.kernalContext().localNodeId(), e));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/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 2e78702..8b3152b 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,13 +40,13 @@ public interface GridGgfsEx extends IgniteFs {
 
     /** {@inheritDoc} */
     @Override GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize, int seqReadsBeforePrefetch)
-        throws IgniteCheckedException;
+        throws IgniteException;
 
     /** {@inheritDoc} */
-    @Override GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteCheckedException;
+    @Override GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteException;
 
     /** {@inheritDoc} */
-    @Override GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteCheckedException;
+    @Override GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteException;
 
     /**
      * Gets global space counters.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java
index 44e110f..214191e 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsImpl.java
@@ -461,7 +461,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean exists(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public boolean exists(IgniteFsPath path) throws IgniteException {
         A.notNull(path, "path");
 
         if (log.isDebugEnabled())
@@ -497,7 +497,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsFile info(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public IgniteFsFile info(IgniteFsPath path) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -526,7 +526,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public IgniteFsPathSummary summary(IgniteFsPath path) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -554,7 +554,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws IgniteCheckedException {
+    @Override public IgniteFsFile update(IgniteFsPath path, Map<String, String> props) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -610,7 +610,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws IgniteCheckedException {
+    @Override public void rename(IgniteFsPath src, IgniteFsPath dest) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(src, "src");
@@ -719,7 +719,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public boolean delete(IgniteFsPath path, boolean recursive) throws IgniteCheckedException {
+    @Override public boolean delete(IgniteFsPath path, boolean recursive) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -815,12 +815,12 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public void mkdirs(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public void mkdirs(IgniteFsPath path) throws IgniteException {
         mkdirs(path, null);
     }
 
     /** {@inheritDoc} */
-    @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws IgniteCheckedException {
+    @Override public void mkdirs(IgniteFsPath path, @Nullable Map<String, String> props) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -907,7 +907,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgniteFsPath> listPaths(final IgniteFsPath path) throws IgniteCheckedException {
+    @Override public Collection<IgniteFsPath> listPaths(final IgniteFsPath path) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -959,7 +959,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public Collection<IgniteFsFile> listFiles(final IgniteFsPath path) throws IgniteCheckedException {
+    @Override public Collection<IgniteFsFile> listFiles(final IgniteFsPath path) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -1026,7 +1026,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public long usedSpaceSize() throws IgniteCheckedException {
+    @Override public long usedSpaceSize() throws IgniteException {
         return metrics().localSpaceSize();
     }
 
@@ -1036,18 +1036,18 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path) throws IgniteException {
         return open(path, cfg.getStreamBufferSize(), cfg.getSequentialReadsBeforePrefetch());
     }
 
     /** {@inheritDoc} */
-    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteCheckedException {
+    @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize) throws IgniteException {
         return open(path, bufSize, cfg.getSequentialReadsBeforePrefetch());
     }
 
     /** {@inheritDoc} */
     @Override public GridGgfsInputStreamAdapter open(IgniteFsPath path, int bufSize, int seqReadsBeforePrefetch)
-        throws IgniteCheckedException {
+        throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -1107,20 +1107,20 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteCheckedException {
+    @Override public IgniteFsOutputStream create(IgniteFsPath path, boolean overwrite) throws IgniteException {
         return create0(path, cfg.getStreamBufferSize(), overwrite, null, 0, null, true);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite, int replication,
-        long blockSize, @Nullable Map<String, String> props) throws IgniteCheckedException {
+        long blockSize, @Nullable Map<String, String> props) throws IgniteException {
         return create0(path, bufSize, overwrite, null, replication, props, false);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFsOutputStream create(IgniteFsPath path, int bufSize, boolean overwrite,
         @Nullable IgniteUuid affKey, int replication, long blockSize, @Nullable Map<String, String> props)
-        throws IgniteCheckedException {
+        throws IgniteException {
         return create0(path, bufSize, overwrite, affKey, replication, props, false);
     }
 
@@ -1248,13 +1248,13 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteCheckedException {
+    @Override public IgniteFsOutputStream append(IgniteFsPath path, boolean create) throws IgniteException {
         return append(path, cfg.getStreamBufferSize(), create, null);
     }
 
     /** {@inheritDoc} */
     @Override public IgniteFsOutputStream append(final IgniteFsPath path, final int bufSize, boolean create,
-        @Nullable Map<String, String> props) throws IgniteCheckedException {
+        @Nullable Map<String, String> props) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -1333,7 +1333,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
     /** {@inheritDoc} */
     @Override public void setTimes(IgniteFsPath path, long accessTime, long modificationTime)
-        throws IgniteCheckedException {
+        throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -1380,13 +1380,13 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len)
-        throws IgniteCheckedException {
+        throws IgniteException {
         return affinity(path, start, len, 0L);
     }
 
     /** {@inheritDoc} */
     @Override public Collection<IgniteFsBlockLocation> affinity(IgniteFsPath path, long start, long len, long maxLen)
-        throws IgniteCheckedException {
+        throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -1431,7 +1431,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public IgniteFsMetrics metrics() throws IgniteCheckedException {
+    @Override public IgniteFsMetrics metrics() throws IgniteException {
         if (enterBusy()) {
             try {
                 IgniteFsPathSummary sum = new IgniteFsPathSummary();
@@ -1482,7 +1482,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public long size(IgniteFsPath path) throws IgniteCheckedException {
+    @Override public long size(IgniteFsPath path) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -1534,7 +1534,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
     }
 
     /** {@inheritDoc} */
-    @Override public void format() throws IgniteCheckedException {
+    @Override public void format() throws IgniteException {
         formatAsync().get();
     }
 
@@ -1668,27 +1668,27 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
     /** {@inheritDoc} */
     @Override public <T, R> R execute(IgniteFsTask<T, R> task, @Nullable IgniteFsRecordResolver rslvr,
-        Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException {
+        Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException {
         return executeAsync(task, rslvr, paths, arg).get();
     }
 
     /** {@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 IgniteCheckedException {
+        throws IgniteException {
         return executeAsync(task, rslvr, paths, skipNonExistentFiles, maxRangeLen, arg).get();
     }
 
     /** {@inheritDoc} */
     @Override public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls,
-        @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteCheckedException {
+        @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, @Nullable T arg) throws IgniteException {
         return executeAsync(taskCls, rslvr, paths, arg).get();
     }
 
     /** {@inheritDoc} */
     @Override public <T, R> R execute(Class<? extends IgniteFsTask<T, R>> taskCls,
         @Nullable IgniteFsRecordResolver rslvr, Collection<IgniteFsPath> paths, boolean skipNonExistentFiles,
-        long maxRangeSize, @Nullable T arg) throws IgniteCheckedException {
+        long maxRangeSize, @Nullable T arg) throws IgniteException {
         return executeAsync(taskCls, rslvr, paths, skipNonExistentFiles, maxRangeSize, arg).get();
     }
 
@@ -1996,7 +1996,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
         /** {@inheritDoc} */
         @Override protected Collection<? extends ComputeJob> split(int gridSize, Object arg)
-            throws IgniteCheckedException {
+            throws IgniteException {
             Collection<ComputeJob> res = new ArrayList<>(gridSize);
 
             for (int i = 0; i < gridSize; i++) {
@@ -2005,7 +2005,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                     @IgniteInstanceResource
                     private Ignite g;
 
-                    @Nullable @Override public IgniteBiTuple<Long, Long> execute() throws IgniteCheckedException {
+                    @Nullable @Override public IgniteBiTuple<Long, Long> execute() throws IgniteException {
                         IgniteFs ggfs = ((GridKernal)g).context().ggfs().ggfs(ggfsName);
 
                         if (ggfs == null)
@@ -2024,7 +2024,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
         }
 
         /** {@inheritDoc} */
-        @Nullable @Override public IgniteBiTuple<Long, Long> reduce(List<ComputeJobResult> results) throws IgniteCheckedException {
+        @Nullable @Override public IgniteBiTuple<Long, Long> reduce(List<ComputeJobResult> results) throws IgniteException {
             long used = 0;
             long max = 0;
 
@@ -2041,7 +2041,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
         }
 
         /** {@inheritDoc} */
-        @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteCheckedException {
+        @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
             // Never failover.
             return ComputeJobResultPolicy.WAIT;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/8bc850c2/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java
index 26ffe06..dccee38 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsJobImpl.java
@@ -71,7 +71,7 @@ public class GridGgfsJobImpl implements ComputeJob, GridInternalWrapper<IgniteFs
     }
 
     /** {@inheritDoc} */
-    @Override public Object execute() throws IgniteCheckedException {
+    @Override public Object execute() throws IgniteException {
         IgniteFs ggfs = ignite.fileSystem(ggfsName);
 
         try (IgniteFsInputStream in = ggfs.open(path)) {
@@ -93,7 +93,7 @@ public class GridGgfsJobImpl implements ComputeJob, GridInternalWrapper<IgniteFs
             return job.execute(ggfs, new IgniteFsFileRange(path, split.start(), split.length()), in);
         }
         catch (IOException e) {
-            throw new IgniteCheckedException("Failed to execute GGFS job for file split [ggfsName=" + ggfsName +
+            throw new IgniteException("Failed to execute GGFS job for file split [ggfsName=" + ggfsName +
                 ", path=" + path + ", start=" + start + ", len=" + len + ']', e);
         }
     }