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 19:22:59 UTC

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

Repository: incubator-ignite
Updated Branches:
  refs/heads/ignite-26 8bc850c2d -> 4307bca1e


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetTaskAttributeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetTaskAttributeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetTaskAttributeSelfTest.java
index 3985380..4a0e4bf 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetTaskAttributeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetTaskAttributeSelfTest.java
@@ -139,7 +139,7 @@ public class GridSessionSetTaskAttributeSelfTest extends GridCommonAbstractTest
                                 return 1;
                         }
                         catch (InterruptedException e) {
-                            throw new IgniteCheckedException("Failed to get attribute due to interruption.", e);
+                            throw new IgniteException("Failed to get attribute due to interruption.", e);
                         }
 
                         return 0;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionTaskWaitJobAttributeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionTaskWaitJobAttributeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionTaskWaitJobAttributeSelfTest.java
index c3dde84..6a124ac 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionTaskWaitJobAttributeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionTaskWaitJobAttributeSelfTest.java
@@ -156,7 +156,7 @@ public class GridSessionTaskWaitJobAttributeSelfTest extends GridCommonAbstractT
                 assert "testVal".equals(val) : "Invalid attribute value: " + val;
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Failed to get attribute due to interruption.", e);
+                throw new IgniteException("Failed to get attribute due to interruption.", e);
             }
 
             return received.size() == SPLIT_COUNT ? ComputeJobResultPolicy.REDUCE : ComputeJobResultPolicy.WAIT;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionWaitAttributeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionWaitAttributeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionWaitAttributeSelfTest.java
index b43555b..8117c0c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionWaitAttributeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionWaitAttributeSelfTest.java
@@ -241,7 +241,7 @@ public class GridSessionWaitAttributeSelfTest extends GridCommonAbstractTest {
      * @throws IgniteCheckedException If failed.
      */
     private static void checkSessionAttributes(ComputeTaskSession ses, String prefix, WaitAttributeType type)
-        throws IgniteCheckedException {
+        throws IgniteException {
         assert ses != null;
         assert type != null;
 
@@ -385,7 +385,7 @@ public class GridSessionWaitAttributeSelfTest extends GridCommonAbstractTest {
             }
         }
         catch (InterruptedException e) {
-            throw new IgniteCheckedException("Got interrupted while waiting for session attributes.", e);
+            throw new IgniteException("Got interrupted while waiting for session attributes.", e);
         }
     }
 
@@ -464,7 +464,7 @@ public class GridSessionWaitAttributeSelfTest extends GridCommonAbstractTest {
                 taskSes.waitForAttribute("done", true, 0);
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Got interrupted while waiting for 'done' attribute.", e);
+                throw new IgniteException("Got interrupted while waiting for 'done' attribute.", e);
             }
 
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/util/future/GridEmbeddedFutureSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/util/future/GridEmbeddedFutureSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/util/future/GridEmbeddedFutureSelfTest.java
index c3ade47..8ad806e 100644
--- a/modules/core/src/test/java/org/gridgain/grid/util/future/GridEmbeddedFutureSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/util/future/GridEmbeddedFutureSelfTest.java
@@ -114,7 +114,7 @@ public class GridEmbeddedFutureSelfTest extends GridCommonAbstractTest {
             catch (IgniteFutureTimeoutException e) {
                 fail("Failed with timeout exception: " + e);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 info("Failed with unhandled exception (normal behaviour): " + e);
 
                 assertSame(x, e.getCause(x.getClass()));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureListenPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureListenPerformanceTest.java b/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureListenPerformanceTest.java
index f97de41..394e69c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureListenPerformanceTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/util/future/GridFutureListenPerformanceTest.java
@@ -71,7 +71,7 @@ public class GridFutureListenPerformanceTest {
                                         try {
                                             t.get();
                                         }
-                                        catch (IgniteCheckedException e) {
+                                        catch (IgniteException e) {
                                             e.printStackTrace();
                                         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapMapAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapMapAbstractSelfTest.java
index 65b287e..48b84fe 100644
--- a/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapMapAbstractSelfTest.java
@@ -668,7 +668,7 @@ public abstract class GridOffHeapMapAbstractSelfTest extends GridCommonAbstractT
                         it.close();
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
                 catch (InterruptedException e) {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
index 6f8e63c..3ce844d 100644
--- a/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/util/offheap/GridOffHeapPartitionedMapAbstractSelfTest.java
@@ -605,7 +605,7 @@ public abstract class GridOffHeapPartitionedMapAbstractSelfTest extends GridComm
                         }
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     fail("Unexpected exception caught: " + e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/colocation/GridTestLifecycleBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/colocation/GridTestLifecycleBean.java b/modules/core/src/test/java/org/gridgain/loadtests/colocation/GridTestLifecycleBean.java
index 6e7bcc6..4b15bcf 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/colocation/GridTestLifecycleBean.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/colocation/GridTestLifecycleBean.java
@@ -14,6 +14,7 @@ import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.resources.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
+import org.gridgain.grid.util.typedef.internal.*;
 
 /**
  * Lifecycle bean.
@@ -28,7 +29,12 @@ public class GridTestLifecycleBean implements LifecycleBean {
 
             assert cache != null;
 
-            cache.loadCache(null, 0, GridTestConstants.LOAD_THREADS, GridTestConstants.ENTRY_COUNT);
+            try {
+                cache.loadCache(null, 0, GridTestConstants.LOAD_THREADS, GridTestConstants.ENTRY_COUNT);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/communication/GridIoManagerBenchmark0.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/communication/GridIoManagerBenchmark0.java b/modules/core/src/test/java/org/gridgain/loadtests/communication/GridIoManagerBenchmark0.java
index 84be76e..a18f6c8 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/communication/GridIoManagerBenchmark0.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/communication/GridIoManagerBenchmark0.java
@@ -382,7 +382,7 @@ public class GridIoManagerBenchmark0 extends GridCommonAbstractTest {
                         }
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     X.println("Message send failed", e);
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsLoadTest.java b/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsLoadTest.java
index 0276456..cc23c6f 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsLoadTest.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsLoadTest.java
@@ -124,7 +124,7 @@ public class GridMultiSplitsLoadTest extends GridCommonAbstractTest {
                         if (taskCnt % 500 == 0)
                             info(stats.toString());
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         error("Failed to execute grid task.", e);
 
                         fail();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsRedeployLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsRedeployLoadTest.java b/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsRedeployLoadTest.java
index 1e473f7..97bbcef 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsRedeployLoadTest.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/direct/multisplit/GridMultiSplitsRedeployLoadTest.java
@@ -120,7 +120,7 @@ public class GridMultiSplitsRedeployLoadTest extends GridCommonAbstractTest {
                         if (taskCnt % 500 == 0)
                             info(stats.toString());
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         error("Failed to execute grid task.", e);
 
                         fail();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/direct/redeploy/GridSingleSplitsRedeployLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/direct/redeploy/GridSingleSplitsRedeployLoadTest.java b/modules/core/src/test/java/org/gridgain/loadtests/direct/redeploy/GridSingleSplitsRedeployLoadTest.java
index fce9b85..9f4a3d9 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/direct/redeploy/GridSingleSplitsRedeployLoadTest.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/direct/redeploy/GridSingleSplitsRedeployLoadTest.java
@@ -145,7 +145,7 @@ public class GridSingleSplitsRedeployLoadTest extends GridCommonAbstractTest {
                             info(stats.toString());
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     error("Failed to execute grid task.", e);
 
                     fail();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/direct/session/GridSessionLoadTestTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/direct/session/GridSessionLoadTestTask.java b/modules/core/src/test/java/org/gridgain/loadtests/direct/session/GridSessionLoadTestTask.java
index b2d8a62..85dce31 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/direct/session/GridSessionLoadTestTask.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/direct/session/GridSessionLoadTestTask.java
@@ -112,7 +112,7 @@ public class GridSessionLoadTestTask extends ComputeTaskAdapter<Integer, Boolean
                     Thread.sleep(100);
                 }
                 catch (InterruptedException e) {
-                    throw new IgniteCheckedException("Thread interrupted.", e);
+                    throw new IgniteException("Thread interrupted.", e);
                 }
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiClient.java b/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiClient.java
index 05de5b8..0baf20d 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiClient.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiClient.java
@@ -13,7 +13,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.util.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.testframework.*;
@@ -133,7 +132,7 @@ public class GridDsiClient implements Callable {
                 if (res2 != null)
                     srvStats = res2;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 e.printStackTrace();
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiLifecycleBean.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiLifecycleBean.java b/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiLifecycleBean.java
index e12cde9..d9dec14 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiLifecycleBean.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/dsi/GridDsiLifecycleBean.java
@@ -12,6 +12,7 @@ package org.gridgain.loadtests.dsi;
 import org.apache.ignite.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.resources.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.springframework.context.*;
 
 /**
@@ -38,7 +39,12 @@ public class GridDsiLifecycleBean implements LifecycleBean {
                 break;
 
             case AFTER_GRID_START:
-                ignite.cache("PARTITIONED_CACHE").dataStructures().atomicSequence("ID", 0, true);
+                try {
+                    ignite.cache("PARTITIONED_CACHE").dataStructures().atomicSequence("ID", 0, true);
+                }
+                catch (IgniteCheckedException e) {
+                    throw U.wrap(e);
+                }
                 break;
 
             case BEFORE_GRID_STOP:

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClient.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClient.java b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClient.java
index e6ce2b5..15cbad0 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClient.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClient.java
@@ -52,7 +52,7 @@ public class GridJobExecutionLoadTestClient implements Callable<Object> {
 
                 txCnt.increment();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 e.printStackTrace();
             }
         }
@@ -189,7 +189,7 @@ public class GridJobExecutionLoadTestClient implements Callable<Object> {
                 try {
                     rmts.execute(GridJobExecutionLoadTestTask.class, null);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
 
@@ -205,7 +205,7 @@ public class GridJobExecutionLoadTestClient implements Callable<Object> {
                 }
             });
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             throw new IllegalStateException(e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java
index d0f6735..653b93b 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionLoadTestClientSemaphore.java
@@ -209,7 +209,7 @@ public class GridJobExecutionLoadTestClientSemaphore implements Callable<Object>
                 try {
                     rmts.execute(GridJobExecutionLoadTestTask.class, null);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
 
@@ -225,7 +225,7 @@ public class GridJobExecutionLoadTestClientSemaphore implements Callable<Object>
                 }
             });
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             throw new IllegalStateException(e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionSingleNodeLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionSingleNodeLoadTest.java b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionSingleNodeLoadTest.java
index 867af1f..6c84d97 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionSingleNodeLoadTest.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobExecutionSingleNodeLoadTest.java
@@ -97,7 +97,7 @@ public class GridJobExecutionSingleNodeLoadTest {
                                 catch (ComputeTaskCancelledException ignored) {
                                     // No-op.
                                 }
-                                catch (IgniteCheckedException e) {
+                                catch (IgniteException e) {
                                     e.printStackTrace();
                                 }
                             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestJob.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestJob.java b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestJob.java
index 1136fef..0ac9926 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestJob.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestJob.java
@@ -115,7 +115,7 @@ public class GridJobLoadTestJob implements ComputeJob {
                 try {
                     taskSes.setAttribute(String.valueOf(i), i);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     log.error("Set attribute failed.", e);
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestSubmitter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestSubmitter.java b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestSubmitter.java
index f9b7084..50d4766 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestSubmitter.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/job/GridJobLoadTestSubmitter.java
@@ -76,7 +76,7 @@ public class GridJobLoadTestSubmitter implements Runnable {
 
                 futures.add(comp.<Integer>future());
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 // Should not be thrown since uses asynchronous execution.
                 throw new IgniteException(e);
             }
@@ -102,7 +102,7 @@ public class GridJobLoadTestSubmitter implements Runnable {
                 catch (IgniteFutureCancelledException ignored) {
                     ignite.log().info(">>> Task cancelled: " + fut.getTaskSession().getId());
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     ignite.log().warning(
                         ">>> Get operation for completed task failed: " + fut.getTaskSession().getId(), e);
                 }
@@ -127,7 +127,7 @@ public class GridJobLoadTestSubmitter implements Runnable {
                 futToCancel.cancel();
                 ignite.log().info("Task canceled: " + futToCancel.getTaskSession().getId());
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 ignite.log().warning(">>> Future cancellation failed: " + futToCancel.getTaskSession().getId(), e);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/mapper/GridContinuousMapperTask1.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/mapper/GridContinuousMapperTask1.java b/modules/core/src/test/java/org/gridgain/loadtests/mapper/GridContinuousMapperTask1.java
index 0071e15..6484a2f 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/mapper/GridContinuousMapperTask1.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/mapper/GridContinuousMapperTask1.java
@@ -110,7 +110,7 @@ public class GridContinuousMapperTask1 extends ComputeTaskAdapter<Integer, Integ
     /** {@inheritDoc} */
     @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> rcvd) throws IgniteException {
         if (res.getException() != null)
-            throw new IgniteCheckedException(res.getException());
+            throw new IgniteException(res.getException());
 
         TestObject o = res.getData();
 
@@ -134,7 +134,7 @@ public class GridContinuousMapperTask1 extends ComputeTaskAdapter<Integer, Integ
             t.join();
         }
         catch (InterruptedException e) {
-            throw new IgniteCheckedException(e);
+            throw new IgniteException(e);
         }
 
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/loadtests/streamer/EventClosure.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/loadtests/streamer/EventClosure.java b/modules/core/src/test/java/org/gridgain/loadtests/streamer/EventClosure.java
index c0eec6d..f857f2f 100644
--- a/modules/core/src/test/java/org/gridgain/loadtests/streamer/EventClosure.java
+++ b/modules/core/src/test/java/org/gridgain/loadtests/streamer/EventClosure.java
@@ -31,7 +31,7 @@ class EventClosure implements IgniteInClosure<IgniteStreamer> {
             try {
                 streamer.addEvent(rnd.nextInt(rndRange));
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 X.println("Failed to add streamer event: " + e);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/hadoop/src/test/java/org/gridgain/grid/kernal/processors/hadoop/GridHadoopMapReduceTest.java
----------------------------------------------------------------------
diff --git a/modules/hadoop/src/test/java/org/gridgain/grid/kernal/processors/hadoop/GridHadoopMapReduceTest.java b/modules/hadoop/src/test/java/org/gridgain/grid/kernal/processors/hadoop/GridHadoopMapReduceTest.java
index b92b432..2583e73 100644
--- a/modules/hadoop/src/test/java/org/gridgain/grid/kernal/processors/hadoop/GridHadoopMapReduceTest.java
+++ b/modules/hadoop/src/test/java/org/gridgain/grid/kernal/processors/hadoop/GridHadoopMapReduceTest.java
@@ -179,7 +179,7 @@ public class GridHadoopMapReduceTest extends GridHadoopAbstractWordCountTest {
                 try {
                     return ggfs.exists(statPath);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     throw new IgniteException(e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheAtomicFieldsQuerySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheAtomicFieldsQuerySelfTest.java b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheAtomicFieldsQuerySelfTest.java
index 3bc4da8..41b5902 100644
--- a/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheAtomicFieldsQuerySelfTest.java
+++ b/modules/indexing/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCacheAtomicFieldsQuerySelfTest.java
@@ -50,7 +50,7 @@ public class GridCacheAtomicFieldsQuerySelfTest extends GridCachePartitionedFiel
 
             fail("We don't support updates.");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             X.println("___ " + e.getMessage());
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/schedule/src/test/java/org/gridgain/grid/kernal/GridScheduleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/schedule/src/test/java/org/gridgain/grid/kernal/GridScheduleSelfTest.java b/modules/schedule/src/test/java/org/gridgain/grid/kernal/GridScheduleSelfTest.java
index f0b5297..447662c 100644
--- a/modules/schedule/src/test/java/org/gridgain/grid/kernal/GridScheduleSelfTest.java
+++ b/modules/schedule/src/test/java/org/gridgain/grid/kernal/GridScheduleSelfTest.java
@@ -227,7 +227,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
                 fail("IgniteCheckedException must have been thrown");
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 info("Caught expected exception: " + e);
             }
 
@@ -236,7 +236,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
                 fail("IgniteCheckedException must have been thrown");
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 info("Caught expected exception: " + e);
             }
         }
@@ -264,7 +264,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
             fail("IgniteCheckedException must have been thrown");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Caught expected exception: " + e);
         }
 
@@ -274,7 +274,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
             fail("IgniteCheckedException must have been thrown");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Caught expected exception: " + e);
         }
 
@@ -284,7 +284,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
             fail("IgniteCheckedException must have been thrown");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Caught expected exception: " + e);
         }
 
@@ -294,7 +294,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
             fail("IgniteCheckedException must have been thrown");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Caught expected exception: " + e);
         }
 
@@ -304,7 +304,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
             fail("IgniteCheckedException must have been thrown");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Caught expected exception: " + e);
         }
 
@@ -314,7 +314,7 @@ public class GridScheduleSelfTest extends GridCommonAbstractTest {
 
             fail("IgniteCheckedException must have been thrown");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Caught expected exception: " + e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceConcurrentUndeploySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceConcurrentUndeploySelfTest.java b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceConcurrentUndeploySelfTest.java
index 9f638d1..27e79fc 100644
--- a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceConcurrentUndeploySelfTest.java
+++ b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceConcurrentUndeploySelfTest.java
@@ -438,7 +438,7 @@ public class GridResourceConcurrentUndeploySelfTest extends GridCommonAbstractTe
                 }
             }
 
-            throw new IgniteCheckedException("Node not found");
+            throw new IgniteException("Node not found");
         }
 
         /**

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceFieldInjectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceFieldInjectionSelfTest.java b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceFieldInjectionSelfTest.java
index ffb998d..544aa9a 100644
--- a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceFieldInjectionSelfTest.java
+++ b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceFieldInjectionSelfTest.java
@@ -104,7 +104,7 @@ public class GridResourceFieldInjectionSelfTest extends GridCommonAbstractTest {
 
             assert false : "Did not get exception for non-transient field.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got correct exception for non-transient field: " + e.getMessage());
         }
         finally {
@@ -123,7 +123,7 @@ public class GridResourceFieldInjectionSelfTest extends GridCommonAbstractTest {
 
             assert false : "Did not get exception for non-transient field.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got correct exception for non-transient field: " + e.getMessage());
         }
 
@@ -141,7 +141,7 @@ public class GridResourceFieldInjectionSelfTest extends GridCommonAbstractTest {
 
             assert false : "Did not get exception for unknown Spring bean name.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got correct exception for with unknown Spring bean name: " + e.getMessage());
         }
 
@@ -159,7 +159,7 @@ public class GridResourceFieldInjectionSelfTest extends GridCommonAbstractTest {
 
             assert false : "Did not get exception for different Spring bean classes.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got correct exception for for different Spring bean classes: " + e.getMessage());
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIocSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIocSelfTest.java b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIocSelfTest.java
index af13906..c319b7a 100644
--- a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIocSelfTest.java
+++ b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIocSelfTest.java
@@ -220,7 +220,7 @@ public class GridResourceIocSelfTest extends GridCommonAbstractTest {
 
                         assert false : "Task must be undeployed";
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         info("Caught expected exception: " + e);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedClassLoaderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedClassLoaderSelfTest.java b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedClassLoaderSelfTest.java
index 3188d09..7e9de93 100644
--- a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedClassLoaderSelfTest.java
+++ b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedClassLoaderSelfTest.java
@@ -275,7 +275,7 @@ public class GridResourceIsolatedClassLoaderSelfTest extends GridCommonAbstractT
 
             assert false : "SharedResourceTask4 should not be allowed to deploy.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Received expected exception: " + e);
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedTaskSelfTest.java b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedTaskSelfTest.java
index f71a4b7..886fe0e 100644
--- a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedTaskSelfTest.java
+++ b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceIsolatedTaskSelfTest.java
@@ -254,7 +254,7 @@ public class GridResourceIsolatedTaskSelfTest extends GridCommonAbstractTest {
 
             assert false : "SharedResourceTask4 should not be allowed to deploy.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Received expected exception: " + e);
         }
         finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceSharedUndeploySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceSharedUndeploySelfTest.java b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceSharedUndeploySelfTest.java
index f36ccbd..c7c4b9d 100644
--- a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceSharedUndeploySelfTest.java
+++ b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridResourceSharedUndeploySelfTest.java
@@ -151,7 +151,7 @@ public class GridResourceSharedUndeploySelfTest extends GridCommonAbstractTest {
 
                 assert false : "SharedResourceTask4 should not be allowed to deploy.";
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 info("Received expected exception: " + e);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridServiceInjectionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridServiceInjectionSelfTest.java b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridServiceInjectionSelfTest.java
index 92f3e30..ae87ebd 100644
--- a/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridServiceInjectionSelfTest.java
+++ b/modules/spring/src/test/java/org/gridgain/grid/kernal/processors/resource/GridServiceInjectionSelfTest.java
@@ -130,7 +130,7 @@ public class GridServiceInjectionSelfTest extends GridCommonAbstractTest impleme
 
             fail();
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             assertTrue(e.getMessage().startsWith("Remote job threw user exception"));
         }
     }
@@ -227,7 +227,7 @@ public class GridServiceInjectionSelfTest extends GridCommonAbstractTest impleme
 
             fail();
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             assertTrue(e.getMessage().startsWith("Remote job threw user exception"));
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/urideploy/src/test/java/org/gridgain/grid/p2p/GridP2PDisabledSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/urideploy/src/test/java/org/gridgain/grid/p2p/GridP2PDisabledSelfTest.java b/modules/urideploy/src/test/java/org/gridgain/grid/p2p/GridP2PDisabledSelfTest.java
index 387e29d..ca97949 100644
--- a/modules/urideploy/src/test/java/org/gridgain/grid/p2p/GridP2PDisabledSelfTest.java
+++ b/modules/urideploy/src/test/java/org/gridgain/grid/p2p/GridP2PDisabledSelfTest.java
@@ -97,7 +97,7 @@ public class GridP2PDisabledSelfTest extends GridCommonAbstractTest {
 
                 assert false;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 info("Received expected exception: " + e);
             }
         }


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

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
index 77cf114..c0564a8 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/task/GridTaskCommandHandler.java
@@ -227,7 +227,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                             try {
                                 desc = new TaskDescriptor(true, f.get(), null);
                             }
-                            catch (IgniteCheckedException e) {
+                            catch (IgniteException e) {
                                 if (e.hasCause(ClusterTopologyException.class, ClusterGroupEmptyException.class))
                                     U.warn(log, "Failed to execute task due to topology issues (are all mapped " +
                                         "nodes alive?) [name=" + name + ", clientId=" + req.clientId() +
@@ -405,7 +405,7 @@ public class GridTaskCommandHandler extends GridRestCommandHandlerAdapter {
                 try {
                     res.result(ctx.config().getMarshaller().unmarshal(res.resultBytes(), null));
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     U.error(log, "Failed to unmarshal task result: " + res, e);
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java
index 0a7a688..8b66c79 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/service/GridServiceNotFoundException.java
@@ -14,7 +14,7 @@ import org.apache.ignite.*;
 /**
  * Exception thrown if service is not found.
  */
-public class GridServiceNotFoundException extends IgniteCheckedException {
+public class GridServiceNotFoundException extends IgniteException {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/util/future/GridEmbeddedFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/future/GridEmbeddedFuture.java b/modules/core/src/main/java/org/gridgain/grid/util/future/GridEmbeddedFuture.java
index 485d4bd..4a71684 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/future/GridEmbeddedFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/future/GridEmbeddedFuture.java
@@ -56,7 +56,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
                 try {
                     onDone(c.apply(embedded.get(), null));
                 }
-                catch (IgniteCheckedException| RuntimeException e) {
+                catch (RuntimeException e) {
                     onDone(c.apply(null, e));
                 }
                 catch (Error e) {
@@ -118,7 +118,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
                             catch (GridClosureException e) {
                                 onDone(e.unwrap());
                             }
-                            catch (IgniteCheckedException | RuntimeException e) {
+                            catch (RuntimeException e) {
                                 onDone(e);
                             }
                             catch (Error e) {
@@ -134,7 +134,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
 
                     onDone(e.unwrap());
                 }
-                catch (IgniteCheckedException | RuntimeException e) {
+                catch (RuntimeException e) {
                     c.apply(null, e);
 
                     onDone(e);
@@ -187,7 +187,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
 
                                 onDone(e.unwrap());
                             }
-                            catch (IgniteCheckedException | RuntimeException e) {
+                            catch (RuntimeException e) {
                                 c2.apply(null, e);
 
                                 onDone(e);
@@ -205,7 +205,7 @@ public class GridEmbeddedFuture<A, B> extends GridFutureAdapter<A> {
 
                     onDone(e.unwrap());
                 }
-                catch (IgniteCheckedException | RuntimeException e) {
+                catch (RuntimeException e) {
                     c1.apply(null, e);
 
                     onDone(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyArgumentBuilder.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyArgumentBuilder.java b/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyArgumentBuilder.java
index 0bec52f..f935023 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyArgumentBuilder.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyArgumentBuilder.java
@@ -144,9 +144,9 @@ public final class GridifyArgumentBuilder {
      * @param arg Task argument contains all necessary data for method invoke.
      * @param input Input collection used in job.
      * @return Argument for job.
-     * @throws IgniteCheckedException In case of error.
+     * @throws IgniteException In case of error.
      */
-    public GridifyArgument createJobArgument(GridifyRangeArgument arg, Collection<?> input) throws IgniteCheckedException {
+    public GridifyArgument createJobArgument(GridifyRangeArgument arg, Collection<?> input) throws IgniteException {
         GridifyArgumentAdapter res = new GridifyArgumentAdapter();
 
         res.setTarget(arg.getTarget());
@@ -169,7 +169,7 @@ public final class GridifyArgumentBuilder {
         Object paramValue = GridifyUtils.collectionToParameter(paramCls, input);
 
         if (paramValue == null)
-            throw new IgniteCheckedException("Failed to create job argument for type: " + paramCls.getName());
+            throw new IgniteException("Failed to create job argument for type: " + paramCls.getName());
 
         mtdArgs[arg.getParamIndex()] = paramValue;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyJobAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyJobAdapter.java b/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyJobAdapter.java
index cbb3215..90a00c1 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyJobAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/gridify/GridifyJobAdapter.java
@@ -70,7 +70,7 @@ public class GridifyJobAdapter extends ComputeJobAdapter {
                     mtd.setAccessible(true);
                 }
                 catch (SecurityException e) {
-                    throw new IgniteCheckedException("Got security exception when attempting to soften access control for " +
+                    throw new IgniteException("Got security exception when attempting to soften access control for " +
                         "@Gridify method: " + mtd, e);
                 }
 
@@ -84,16 +84,16 @@ public class GridifyJobAdapter extends ComputeJobAdapter {
             return mtd.invoke(obj, arg.getMethodParameters());
         }
         catch (InvocationTargetException e) {
-            if (e.getTargetException() instanceof IgniteCheckedException)
-                throw (IgniteCheckedException)e.getTargetException();
+            if (e.getTargetException() instanceof IgniteException)
+                throw (IgniteException)e.getTargetException();
 
-            throw new IgniteCheckedException("Failed to invoke a method due to user exception.", e.getTargetException());
+            throw new IgniteException("Failed to invoke a method due to user exception.", e.getTargetException());
         }
         catch (IllegalAccessException e) {
-            throw new IgniteCheckedException("Failed to access method for execution.", e);
+            throw new IgniteException("Failed to access method for execution.", e);
         }
         catch (NoSuchMethodException e) {
-            throw new IgniteCheckedException("Failed to find method for execution.", e);
+            throw new IgniteException("Failed to find method for execution.", e);
         }
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/util/lang/GridFunc.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/util/lang/GridFunc.java b/modules/core/src/main/java/org/gridgain/grid/util/lang/GridFunc.java
index 3b31ecf..1e16164 100644
--- a/modules/core/src/main/java/org/gridgain/grid/util/lang/GridFunc.java
+++ b/modules/core/src/main/java/org/gridgain/grid/util/lang/GridFunc.java
@@ -2769,12 +2769,7 @@ public class GridFunc {
         return new GridSerializableFuture<T>() {
             @Override public boolean cancel(boolean mayInterruptIfRunning) {
                 if (mayInterruptIfRunning) {
-                    try {
-                        return fut.cancel();
-                    }
-                    catch (IgniteCheckedException e) {
-                        throw new IgniteException(e);
-                    }
+                    return fut.cancel();
                 }
                 else
                     return false;
@@ -2798,7 +2793,7 @@ public class GridFunc {
                 catch (GridInterruptedException ignore) {
                     throw new InterruptedException("The computation was interrupted.");
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     throw new ExecutionException("The computation failed.", e);
                 }
             }
@@ -2817,7 +2812,7 @@ public class GridFunc {
                 catch (IgniteFutureTimeoutException e) {
                     throw new TimeoutException("The computation timed out: " + e.getMessage());
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     throw new ExecutionException("The computation failed.", e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/jdbc/util/GridJdbcUtils.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/jdbc/util/GridJdbcUtils.java b/modules/core/src/main/java/org/gridgain/jdbc/util/GridJdbcUtils.java
index 3d79267..ec59801 100644
--- a/modules/core/src/main/java/org/gridgain/jdbc/util/GridJdbcUtils.java
+++ b/modules/core/src/main/java/org/gridgain/jdbc/util/GridJdbcUtils.java
@@ -41,7 +41,7 @@ public class GridJdbcUtils {
         try {
             return MARSHALLER.marshal(args);
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             throw new SQLException("Failed to unmarshal result.", e);
         }
     }
@@ -70,7 +70,7 @@ public class GridJdbcUtils {
         try {
             return MARSHALLER.unmarshal(bytes, null);
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             throw new SQLException("Failed to unmarshal result.", e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedMarshallerTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedMarshallerTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedMarshallerTest.java
index 8583dc2..22b5b13 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedMarshallerTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedMarshallerTest.java
@@ -202,7 +202,7 @@ public class GridOptimizedMarshallerTest extends GridCommonAbstractTest {
 
             fail();
         }
-        catch (IgniteCheckedException ignore) {
+        catch (IgniteException ignore) {
             // No-op.
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedObjectStreamSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedObjectStreamSelfTest.java b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedObjectStreamSelfTest.java
index c197283..2bfe01b 100644
--- a/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedObjectStreamSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/marshaller/optimized/GridOptimizedObjectStreamSelfTest.java
@@ -212,7 +212,7 @@ public class GridOptimizedObjectStreamSelfTest extends GridCommonAbstractTest {
 
             assert false : "Exception not thrown.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             NotSerializableException serEx = e.getCause(NotSerializableException.class);
 
             if (serEx == null)

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest.java
index 56eb74b..2d9fe03 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/checkpoint/sharedfs/GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest.java
@@ -101,7 +101,7 @@ public class GridSharedFsCheckpointSpiMultipleDirectoriesSelfTest extends
         try {
             getSpi().saveCheckpoint(CHECK_POINT_KEY_PREFIX, GridTestIoUtils.serializeJdk(state), 0, true);
         }
-        catch (IgniteCheckedException ignored) {
+        catch (IgniteException ignored) {
             error = true;
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
index 5031b2f..0c3da6b 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridCacheDhtLockBackupSelfTest.java
@@ -265,7 +265,7 @@ public class GridCacheDhtLockBackupSelfTest extends GridCommonAbstractTest {
                     U.sleep(delayTime);
                 }
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Cannot process incoming message", e);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
index 2e579d0..aec6818 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridOrderedMessageCancelSelfTest.java
@@ -192,7 +192,7 @@ public class GridOrderedMessageCancelSelfTest extends GridCommonAbstractTest {
         @Override protected Collection<? extends ComputeJob> split(int gridSize, Void arg) throws IgniteException {
             return Collections.singleton(new ComputeJobAdapter() {
                 @Nullable @Override public Object execute() throws IgniteException {
-                    throw new IgniteCheckedException("Task failed.");
+                    throw new IgniteException("Task failed.");
                 }
             });
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java
index ff15986..1f407ac 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiLanTest.java
@@ -168,7 +168,7 @@ public class GridTcpCommunicationSpiLanTest extends GridSpiAbstractTest<TcpCommu
                         spi.sendMessage(remoteNode, msg);
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     fail("Unable to send message: " + e.getMessage());
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
index 1889cd2..9b5330c 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/communication/tcp/GridTcpCommunicationSpiMultithreadedSelfTest.java
@@ -193,7 +193,7 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
                         queue.offer(msg);
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     log().error("Unable to send message.", e);
 
                     fail("Unable to send message: " + e.getMessage());
@@ -281,7 +281,7 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
                                 spis.get(from.id()).sendMessage(node, msg);
                             }
                         }
-                        catch (IgniteCheckedException e) {
+                        catch (IgniteException e) {
                             log.warning(">>> Oops, unable to send message (safe to ignore).", e);
                         }
 
@@ -378,7 +378,7 @@ public abstract class GridTcpCommunicationSpiMultithreadedSelfTest extends GridS
                         spi.sendMessage(to, msg);
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     fail("Unable to send message: " + e.getMessage());
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
index cc8a5e0..67f745f 100644
--- a/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/spi/loadbalancing/roundrobin/GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest.java
@@ -115,7 +115,7 @@ public class GridRoundRobinLoadBalancingSpiNotPerTaskSelfTest
         try {
             getSpi().getBalancedNode(ses, notInTop, new GridTestJob());
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             assertTrue(e.getMessage().contains("Task topology does not have alive nodes"));
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java b/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java
index 060b6e4..1f2851d 100644
--- a/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java
+++ b/modules/core/src/test/java/org/apache/ignite/streamer/index/GridStreamerIndexSelfTest.java
@@ -86,7 +86,7 @@ public class GridStreamerIndexSelfTest extends GridCommonAbstractTest {
 
             fail("Exception should have been thrown.");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Caught expected exception: " + e);
         }
 
@@ -254,7 +254,7 @@ public class GridStreamerIndexSelfTest extends GridCommonAbstractTest {
                     try {
                         win.enqueue(evt);
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         if (e.getMessage().contains("Index unique key violation"))
                             nIdxErrors.incrementAndGet();
                         else
@@ -525,7 +525,7 @@ public class GridStreamerIndexSelfTest extends GridCommonAbstractTest {
 
                 fail("Exception should have been thrown.");
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 info("Caught expected exception: " + e);
             }
         }
@@ -670,7 +670,7 @@ public class GridStreamerIndexSelfTest extends GridCommonAbstractTest {
         /** {@inheritDoc} */
         @Nullable @Override public String onAdded(StreamerIndexEntry<String, String, String> entry, String evt)
             throws IgniteException {
-            throw new IgniteCheckedException("Unique key violation: " + evt);
+            throw new IgniteException("Unique key violation: " + evt);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/GridTestJobResult.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/GridTestJobResult.java b/modules/core/src/test/java/org/gridgain/grid/GridTestJobResult.java
index 4f7563b..11f6576 100644
--- a/modules/core/src/test/java/org/gridgain/grid/GridTestJobResult.java
+++ b/modules/core/src/test/java/org/gridgain/grid/GridTestJobResult.java
@@ -23,7 +23,7 @@ public class GridTestJobResult implements ComputeJobResult {
     private final Serializable data;
 
     /** */
-    private final IgniteCheckedException e;
+    private final IgniteException e;
 
     /** */
     private final ComputeJob job;
@@ -50,7 +50,7 @@ public class GridTestJobResult implements ComputeJobResult {
      * @param node Grid node.
      * @param jobCtx Job context.
      */
-    public GridTestJobResult(Serializable data, IgniteCheckedException e, ComputeJob job, ClusterNode node, ComputeJobContext jobCtx) {
+    public GridTestJobResult(Serializable data, IgniteException e, ComputeJob job, ClusterNode node, ComputeJobContext jobCtx) {
         this.data = data;
         this.e = e;
         this.job = job;
@@ -72,7 +72,7 @@ public class GridTestJobResult implements ComputeJobResult {
 
     /** {@inheritDoc} */ @Override public Serializable getData() { return data; }
 
-    /** {@inheritDoc} */ @Override public IgniteCheckedException getException() { return e; }
+    /** {@inheritDoc} */ @Override public IgniteException getException() { return e; }
 
     /** {@inheritDoc} */ @Override public boolean isCancelled() { return false; }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/ggfs/GridGgfsEventsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/ggfs/GridGgfsEventsAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/ggfs/GridGgfsEventsAbstractSelfTest.java
index ada1195..5c83b4f 100644
--- a/modules/core/src/test/java/org/gridgain/grid/ggfs/GridGgfsEventsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/ggfs/GridGgfsEventsAbstractSelfTest.java
@@ -510,7 +510,7 @@ public abstract class GridGgfsEventsAbstractSelfTest extends GridCommonAbstractT
         try {
             ggfs.delete(dir.parent(), false); // Will generate no events.
         }
-        catch (IgniteCheckedException ignore) {
+        catch (IgniteException ignore) {
             // No-op.
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java
index 8be4c9b..46effa1 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridAlwaysFailoverSpiFailSelfTest.java
@@ -61,7 +61,7 @@ public class GridAlwaysFailoverSpiFailSelfTest extends GridCommonAbstractTest {
 
             assert false;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             //No-op
         }
 
@@ -85,7 +85,7 @@ public class GridAlwaysFailoverSpiFailSelfTest extends GridCommonAbstractTest {
 
             assert false;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             //No-op
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelOnGridStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelOnGridStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelOnGridStopSelfTest.java
index 860d4f8..9498d32 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelOnGridStopSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelOnGridStopSelfTest.java
@@ -80,7 +80,7 @@ public class GridCancelOnGridStopSelfTest extends GridCommonAbstractTest {
                                 Thread.sleep(Long.MAX_VALUE);
                             }
                             catch (InterruptedException e) {
-                                throw new IgniteCheckedException(e);
+                                throw new IgniteException(e);
                             }
 
                             return null;
@@ -89,7 +89,7 @@ public class GridCancelOnGridStopSelfTest extends GridCommonAbstractTest {
                 }
             }
 
-            throw new IgniteCheckedException("Local node not found");
+            throw new IgniteException("Local node not found");
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelledJobsMetricsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelledJobsMetricsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelledJobsMetricsSelfTest.java
index d4ab8fc..a7a6619 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelledJobsMetricsSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridCancelledJobsMetricsSelfTest.java
@@ -86,7 +86,7 @@ public class GridCancelledJobsMetricsSelfTest extends GridCommonAbstractTest {
 
                 assert false : "Job was not interrupted.";
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (e.hasCause(InterruptedException.class))
                     throw new IgniteCheckedException("Test run has been interrupted.", e);
 
@@ -133,10 +133,10 @@ public class GridCancelledJobsMetricsSelfTest extends GridCommonAbstractTest {
                     Thread.sleep(1000);
                 }
                 catch (InterruptedException e1) {
-                    throw new IgniteCheckedException("Unexpected exception: ", e1);
+                    throw new IgniteException("Unexpected exception: ", e1);
                 }
 
-                throw new IgniteCheckedException("Job got interrupted while waiting for cancellation.");
+                throw new IgniteException("Job got interrupted while waiting for cancellation.");
             }
             finally {
                 X.println("Finished job.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridCommunicationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridCommunicationSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridCommunicationSelfTest.java
index a8caced..e59f3f7 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridCommunicationSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridCommunicationSelfTest.java
@@ -59,7 +59,7 @@ public class GridCommunicationSelfTest extends GridCommonAbstractTest {
             message(ignite.cluster().forNodes(nodes)).send(null,
                 new GridTestCommunicationMessage(cntr, ignite.cluster().localNode().id()));
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             error("Failed to send message.", e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobAnnotationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobAnnotationSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobAnnotationSelfTest.java
index 0f66aaf..14abe58 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobAnnotationSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobAnnotationSelfTest.java
@@ -102,7 +102,7 @@ public class GridContinuousJobAnnotationSelfTest extends GridCommonAbstractTest
                 mapper.send(((Class<ComputeJob>)arg).newInstance());
             }
             catch (Exception e) {
-                throw new IgniteCheckedException("Job instantination failed.", e);
+                throw new IgniteException("Job instantination failed.", e);
             }
 
             return null;
@@ -113,7 +113,7 @@ public class GridContinuousJobAnnotationSelfTest extends GridCommonAbstractTest
             throws IgniteException {
             if (res.getException() != null) {
                 if (res.getException() instanceof ComputeUserUndeclaredException)
-                    throw new IgniteCheckedException("Job threw unexpected exception.", res.getException());
+                    throw new IgniteException("Job threw unexpected exception.", res.getException());
 
                 return ComputeJobResultPolicy.FAILOVER;
             }
@@ -187,7 +187,7 @@ public class GridContinuousJobAnnotationSelfTest extends GridCommonAbstractTest
             if (fail.get()) {
                 fail.set(false);
 
-                throw new IgniteCheckedException("Expected test exception.");
+                throw new IgniteException("Expected test exception.");
             }
 
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobSiblingsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobSiblingsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobSiblingsSelfTest.java
index af09425..10528f4 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobSiblingsSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousJobSiblingsSelfTest.java
@@ -77,7 +77,7 @@ public class GridContinuousJobSiblingsSelfTest extends GridCommonAbstractTest {
         @Override public ComputeJobResultPolicy result(ComputeJobResult res, List<ComputeJobResult> received)
             throws IgniteException {
             if (res.getException() != null)
-                throw new IgniteCheckedException("Job resulted in error: " + res, res.getException());
+                throw new IgniteException("Job resulted in error: " + res, res.getException());
 
             assert ses.getJobSiblings().size() == jobCnt;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousTaskSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousTaskSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousTaskSelfTest.java
index 5b15269..85bc972 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousTaskSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridContinuousTaskSelfTest.java
@@ -81,7 +81,7 @@ public class GridContinuousTaskSelfTest extends GridCommonAbstractTest {
                         assert fut1.get() == 55;
                         assert fut2.get() == 55;
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         assert false : "Test task failed: " + e;
                     }
                 }
@@ -249,7 +249,7 @@ public class GridContinuousTaskSelfTest extends GridCommonAbstractTest {
                                 try {
                                     mapper.send(new SessionChainTestJob(cnt));
                                 }
-                                catch (IgniteCheckedException e) {
+                                catch (IgniteException e) {
                                     log.error("Failed to send new job.", e);
                                 }
                             }
@@ -325,7 +325,7 @@ public class GridContinuousTaskSelfTest extends GridCommonAbstractTest {
                 Thread.sleep(10000);
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Job has been interrupted.", e);
+                throw new IgniteException("Job has been interrupted.", e);
             }
 
             mapper.send(new TestJob(++cnt));

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentMultiThreadedSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentMultiThreadedSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentMultiThreadedSelfTest.java
index 0af0b2e..c2008df 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentMultiThreadedSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentMultiThreadedSelfTest.java
@@ -57,7 +57,7 @@ public class GridDeploymentMultiThreadedSelfTest extends GridCommonAbstractTest
                         if (++iterCnt % 100 == 0)
                             info("Iterations count: " + iterCnt);
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         U.error(log, "Failed to undeploy task message.", e);
 
                         fail("See logs for details.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentSelfTest.java
index a986dae..998d424 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridDeploymentSelfTest.java
@@ -217,7 +217,7 @@ public class GridDeploymentSelfTest extends GridCommonAbstractTest {
 
                 assert false : "Should not be able to deploy 2 task with same task name";
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 info("Received expected grid exception: " + e);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageCheckAllEventsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageCheckAllEventsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageCheckAllEventsSelfTest.java
index 8ae2215..592b812 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageCheckAllEventsSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridEventStorageCheckAllEventsSelfTest.java
@@ -154,7 +154,7 @@ public class GridEventStorageCheckAllEventsSelfTest extends GridCommonAbstractTe
 
             assert false : "Grid with locally executed job with timeout should throw GridComputeTaskTimeoutException.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Expected exception caught [taskFuture=" + fut + ", exception=" + e + ']');
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridExplicitImplicitDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridExplicitImplicitDeploymentSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridExplicitImplicitDeploymentSelfTest.java
index def4d0a..829ce7b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridExplicitImplicitDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridExplicitImplicitDeploymentSelfTest.java
@@ -14,7 +14,6 @@ import org.apache.ignite.cluster.*;
 import org.apache.ignite.compute.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.resources.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.*;
@@ -460,7 +459,7 @@ public class GridExplicitImplicitDeploymentSelfTest extends GridCommonAbstractTe
                 return null;
             }
             catch (IOException e) {
-                throw new IgniteCheckedException("Failed to execute job.", e);
+                throw new IgniteException("Failed to execute job.", e);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridFactoryVmShutdownTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFactoryVmShutdownTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFactoryVmShutdownTest.java
index 37aa005..da45c0a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFactoryVmShutdownTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFactoryVmShutdownTest.java
@@ -70,7 +70,7 @@ public class GridFactoryVmShutdownTest {
                 try {
                     G.start(cfg);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     throw new IgniteException("Failed to start grid in shutdown hook.", e);
                 }
                 finally {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java
index 05bf11a..4ced278 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverCustomTopologySelfTest.java
@@ -92,7 +92,7 @@ public class GridFailoverCustomTopologySelfTest extends GridCommonAbstractTest {
 
                 info("Task result: " + res);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 info("Got unexpected grid exception: " + e);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java
index e52bae3..ae26a28 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverSelfTest.java
@@ -94,7 +94,7 @@ public class GridFailoverSelfTest extends GridCommonAbstractTest {
                         fail = ses.<String, Boolean>waitForAttribute("fail", 0);
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Got interrupted while waiting for attribute to be set.", e);
+                        throw new IgniteException("Got interrupted while waiting for attribute to be set.", e);
                     }
 
                     if (fail) {
@@ -102,7 +102,7 @@ public class GridFailoverSelfTest extends GridCommonAbstractTest {
 
                         assert nodeRef.get().id().equals(locId);
 
-                        throw new IgniteCheckedException("Job exception.");
+                        throw new IgniteException("Job exception.");
                     }
 
                     assert !nodeRef.get().id().equals(locId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
index 27c3cd5..c44a4c3 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTaskWithPredicateSelfTest.java
@@ -204,13 +204,13 @@ public class GridFailoverTaskWithPredicateSelfTest extends GridCommonAbstractTes
                         fail = ses.<String, Boolean>waitForAttribute("fail", 0);
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Got interrupted while waiting for attribute to be set.", e);
+                        throw new IgniteException("Got interrupted while waiting for attribute to be set.", e);
                     }
 
                     if (fail) {
                         ses.setAttribute("fail", false);
 
-                        throw new IgniteCheckedException("Job exception.");
+                        throw new IgniteException("Job exception.");
                     }
 
                     // This job does not return any result.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java
index eb7b69c..5803b2f 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridFailoverTopologySelfTest.java
@@ -87,7 +87,7 @@ public class GridFailoverTopologySelfTest extends GridCommonAbstractTest {
             try {
                 compute(ignite1.cluster().forRemotes()).execute(JobFailTask.class, null);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 info("Got expected grid exception: " + e);
             }
 
@@ -121,7 +121,7 @@ public class GridFailoverTopologySelfTest extends GridCommonAbstractTest {
 
             return Collections.singletonMap(new ComputeJobAdapter(arg) {
                 @Override public Serializable execute() throws IgniteException {
-                    throw new IgniteCheckedException("Job exception.");
+                    throw new IgniteException("Job exception.");
                 }
             }, remoteNode);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobMasterLeaveAwareSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobMasterLeaveAwareSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobMasterLeaveAwareSelfTest.java
index f5db8d6..3f306a3 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobMasterLeaveAwareSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobMasterLeaveAwareSelfTest.java
@@ -517,7 +517,7 @@ public class GridJobMasterLeaveAwareSelfTest extends GridCommonAbstractTest {
         try {
             fut.get();
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             log.debug("Task failed: " + e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java
index b68536c..8aa9735 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingSelfTest.java
@@ -420,7 +420,7 @@ public class GridJobStealingSelfTest extends GridCommonAbstractTest {
             catch (InterruptedException e) {
                 log.info("Job got interrupted on node: " + ignite.cluster().localNode().id());
 
-                throw new IgniteCheckedException("Job got interrupted.", e);
+                throw new IgniteException("Job got interrupted.", e);
             }
             finally {
                 log.info("Job finished on node: " + ignite.cluster().localNode().id());

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java
index d04fffc..d5a6ad8 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridJobStealingZeroActiveJobsSelfTest.java
@@ -153,7 +153,7 @@ public class GridJobStealingZeroActiveJobsSelfTest extends GridCommonAbstractTes
                 Thread.sleep(sleep);
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Job got interrupted.", e);
+                throw new IgniteException("Job got interrupted.", e);
             }
 
             return ignite.name();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleBeanSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleBeanSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleBeanSelfTest.java
index 889a747..27c3ae2 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleBeanSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridLifecycleBeanSelfTest.java
@@ -278,7 +278,7 @@ public class GridLifecycleBeanSelfTest extends GridCommonAbstractTest {
         @Override public void onLifecycleEvent(LifecycleEventType evt) throws IgniteException {
             if (evt == errType) {
                 if (gridErr)
-                    throw new IgniteCheckedException("Expected exception for event: " + evt) {
+                    throw new IgniteException("Expected exception for event: " + evt) {
                         @Override public void printStackTrace(PrintStream s) {
                             // No-op.
                         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridListenActorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridListenActorSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridListenActorSelfTest.java
index 53f13b8..abe1dbe 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridListenActorSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridListenActorSelfTest.java
@@ -13,6 +13,7 @@ import org.apache.ignite.*;
 import org.apache.ignite.cluster.*;
 import org.apache.ignite.messaging.*;
 import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.*;
 import org.gridgain.testframework.junits.common.*;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java
index fff71c3..1bbe55a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleSpisSelfTest.java
@@ -104,7 +104,7 @@ public class GridMultipleSpisSelfTest extends GridCommonAbstractTest {
             try {
                 ignite1.compute().execute(GridTestMultipleSpisTask.class.getName(), ignite1.cluster().localNode().id());
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 e.printStackTrace();
 
                 assert false : "Unexpected exception.";
@@ -284,7 +284,7 @@ public class GridMultipleSpisSelfTest extends GridCommonAbstractTest {
 
             // Should always fail on task originating node and work on another one.
             if (locId.equals(argument(0)))
-                throw new IgniteCheckedException("Expected exception to failover job.");
+                throw new IgniteException("Expected exception to failover job.");
 
             // Use checkpoint on job side. This will happen on remote node.
             jobSes.loadCheckpoint("test");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleVersionsDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleVersionsDeploymentSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleVersionsDeploymentSelfTest.java
index 879dde3..c69b6ae 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleVersionsDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultipleVersionsDeploymentSelfTest.java
@@ -286,7 +286,7 @@ public class GridMultipleVersionsDeploymentSelfTest extends GridCommonAbstractTe
                 return res - 48;
             }
             catch (IOException | InterruptedException e) {
-                throw new IgniteCheckedException("Failed to execute job.", e);
+                throw new IgniteException("Failed to execute job.", e);
             }
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java
index ff74770..4a0a683 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridMultithreadedJobStealingSelfTest.java
@@ -91,7 +91,7 @@ public class GridMultithreadedJobStealingSelfTest extends GridCommonAbstractTest
                         }
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     log.error("Failed to execute task.", e);
 
                     fail.getAndSet(e);
@@ -210,7 +210,7 @@ public class GridMultithreadedJobStealingSelfTest extends GridCommonAbstractTest
                 Thread.sleep(sleep);
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Job got interrupted.", e);
+                throw new IgniteException("Job got interrupted.", e);
             }
 
             return ignite.name();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridRuntimeExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridRuntimeExceptionSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridRuntimeExceptionSelfTest.java
index c114840..b12db3a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridRuntimeExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridRuntimeExceptionSelfTest.java
@@ -76,7 +76,7 @@ public class GridRuntimeExceptionSelfTest extends GridCommonAbstractTest {
 
             assert false;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got expected grid exception: " + e);
         }
 
@@ -106,7 +106,7 @@ public class GridRuntimeExceptionSelfTest extends GridCommonAbstractTest {
 
             assert false;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got expected grid exception: " + e);
         }
 
@@ -136,7 +136,7 @@ public class GridRuntimeExceptionSelfTest extends GridCommonAbstractTest {
 
             assert false;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got expected grid exception: " + e);
         }
 
@@ -166,7 +166,7 @@ public class GridRuntimeExceptionSelfTest extends GridCommonAbstractTest {
 
             assert false;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got expected grid exception: " + e);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java
index 815e59c..bcb5126 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridSpiExceptionSelfTest.java
@@ -70,7 +70,7 @@ public class GridSpiExceptionSelfTest extends GridCommonAbstractTest {
 
                 assert false : "Exception should be thrown";
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 assert e.getCause() instanceof GridTestSpiException : "Wrong cause exception type. " + e;
 
                 assert e.getCause().getMessage().startsWith(TEST_MSG) : "Wrong exception message." + e.getMessage();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithCancelSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithCancelSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithCancelSelfTest.java
index bbe995e..49996df 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithCancelSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithCancelSelfTest.java
@@ -95,7 +95,7 @@ public class GridStopWithCancelSelfTest extends GridCommonAbstractTest {
                                 Thread.sleep(Long.MAX_VALUE);
                             }
                             catch (InterruptedException e) {
-                                throw new IgniteCheckedException(e);
+                                throw new IgniteException(e);
                             }
 
                             return null;
@@ -104,7 +104,7 @@ public class GridStopWithCancelSelfTest extends GridCommonAbstractTest {
                 }
             }
 
-            throw new IgniteCheckedException("Local node not found");
+            throw new IgniteException("Local node not found");
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java
index 4ba0c82..ba8621f 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridStopWithWaitSelfTest.java
@@ -194,7 +194,7 @@ public class GridStopWithWaitSelfTest extends GridCommonAbstractTest {
                         fail = ses.waitForAttribute("fail", 0);
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Got interrupted while waiting for attribute to be set.", e);
+                        throw new IgniteException("Got interrupted while waiting for attribute to be set.", e);
                     }
 
                     log.info("Failed attribute: " + fail);
@@ -207,7 +207,7 @@ public class GridStopWithWaitSelfTest extends GridCommonAbstractTest {
 
                         log.info("Throwing grid exception from job.");
 
-                        throw new IgniteCheckedException("Job exception.");
+                        throw new IgniteException("Job exception.");
                     }
 
                     assert !nodeRef.get().id().equals(locId);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskContinuousMapperSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskContinuousMapperSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskContinuousMapperSelfTest.java
index 5c6c7dd..d7ed2e6 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskContinuousMapperSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskContinuousMapperSelfTest.java
@@ -164,7 +164,7 @@ public class GridTaskContinuousMapperSelfTest extends GridCommonAbstractTest {
 
                 assert false;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (log.isInfoEnabled())
                     log.info("Expected exception: " + e);
             }
@@ -174,7 +174,7 @@ public class GridTaskContinuousMapperSelfTest extends GridCommonAbstractTest {
 
                 assert false;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (log.isInfoEnabled())
                     log.info("Expected exception: " + e);
             }
@@ -184,7 +184,7 @@ public class GridTaskContinuousMapperSelfTest extends GridCommonAbstractTest {
 
                 assert false;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (log.isInfoEnabled())
                     log.info("Expected exception: " + e);
             }
@@ -194,7 +194,7 @@ public class GridTaskContinuousMapperSelfTest extends GridCommonAbstractTest {
 
                 assert false;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (log.isInfoEnabled())
                     log.info("Expected exception: " + e);
             }
@@ -253,7 +253,7 @@ public class GridTaskContinuousMapperSelfTest extends GridCommonAbstractTest {
 
                 assert false;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (log.isInfoEnabled())
                     log.info("Expected exception: " + e);
             }
@@ -283,7 +283,7 @@ public class GridTaskContinuousMapperSelfTest extends GridCommonAbstractTest {
 
                 assert false;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (log.isInfoEnabled())
                     log.info("Expected exception: " + e);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskInstantiationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskInstantiationSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskInstantiationSelfTest.java
index df76582..7600d20 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskInstantiationSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/GridTaskInstantiationSelfTest.java
@@ -69,7 +69,7 @@ public class GridTaskInstantiationSelfTest extends GridCommonAbstractTest {
                         }
                     }, node);
 
-            throw new IgniteCheckedException("Local node not found.");
+            throw new IgniteException("Local node not found.");
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java
index 566a201..90f2cb8 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/checkpoint/GridCheckpointManagerAbstractSelfTest.java
@@ -365,7 +365,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 Thread.sleep(6000);
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException(e);
+                throw new IgniteException(e);
             }
 
             assertWithRetries(new GridAbsClosureX() {
@@ -433,7 +433,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 startLatch.await();
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Thread has been interrupted.", e);
+                throw new IgniteException("Thread has been interrupted.", e);
             }
 
             X.println(">>> Producer started.");
@@ -447,7 +447,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 read1FinishedLatch.await();
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Thread has been interrupted.", e);
+                throw new IgniteException("Thread has been interrupted.", e);
             }
 
             // No retries here as other thread should have seen checkpoint already.
@@ -463,7 +463,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 read2FinishedLatch.await();
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Thread has been interrupted.", e);
+                throw new IgniteException("Thread has been interrupted.", e);
             }
 
             assert GLOBAL_VAL.equals(taskSes.loadCheckpoint(GLOBAL_KEY));
@@ -479,7 +479,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 read3FinishedLatch.await();
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Thread has been interrupted.", e);
+                throw new IgniteException("Thread has been interrupted.", e);
             }
 
             assert SES_VAL_OVERWRITTEN.equals(taskSes.loadCheckpoint(GLOBAL_KEY));
@@ -536,7 +536,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 startLatch.await();
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Thread has been interrupted.", e);
+                throw new IgniteException("Thread has been interrupted.", e);
             }
 
             X.println(">>> Consumer started.");
@@ -545,7 +545,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 read1Latch.await();
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Thread has been interrupted.", e);
+                throw new IgniteException("Thread has been interrupted.", e);
             }
 
             // Test that checkpoints were saved properly.
@@ -562,7 +562,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 read2Latch.await();
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Thread has been interrupted.", e);
+                throw new IgniteException("Thread has been interrupted.", e);
             }
 
             // Test that checkpoints were not overwritten.
@@ -579,7 +579,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 read3Latch.await();
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Thread has been interrupted.", e);
+                throw new IgniteException("Thread has been interrupted.", e);
             }
 
             assertWithRetries(new GridAbsClosureX() {
@@ -595,7 +595,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                 rmvLatch.await();
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Thread has been interrupted.", e);
+                throw new IgniteException("Thread has been interrupted.", e);
             }
             // Check checkpoints are actually removed.
             assert !taskSes.removeCheckpoint(GLOBAL_KEY);
@@ -664,7 +664,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                         startLatch.await();
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Thread has been interrupted.", e);
+                        throw new IgniteException("Thread has been interrupted.", e);
                     }
 
                     X.println(">>> Global consumer started.");
@@ -673,7 +673,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                         read1Latch.await();
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Thread has been interrupted.", e);
+                        throw new IgniteException("Thread has been interrupted.", e);
                     }
 
                     // Test that checkpoints were saved properly.
@@ -686,7 +686,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                         read2Latch.await();
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Thread has been interrupted.", e);
+                        throw new IgniteException("Thread has been interrupted.", e);
                     }
 
                     // Test that checkpoints were not overwritten.
@@ -699,7 +699,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                         read3Latch.await();
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Thread has been interrupted.", e);
+                        throw new IgniteException("Thread has been interrupted.", e);
                     }
 
                     assert GLOBAL_VAL_OVERWRITTEN.equals(taskSes.loadCheckpoint(SES_KEY));
@@ -711,7 +711,7 @@ public abstract class GridCheckpointManagerAbstractSelfTest extends GridCommonAb
                         rmvLatch.await();
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Thread has been interrupted.", e);
+                        throw new IgniteException("Thread has been interrupted.", e);
                     }
                     // Check checkpoints are actually removed.
                     assert !taskSes.removeCheckpoint(GLOBAL_KEY);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/managers/communication/GridCommunicationManagerListenersSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/communication/GridCommunicationManagerListenersSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/communication/GridCommunicationManagerListenersSelfTest.java
index e8737af..b55985e 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/managers/communication/GridCommunicationManagerListenersSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/managers/communication/GridCommunicationManagerListenersSelfTest.java
@@ -100,7 +100,7 @@ public class GridCommunicationManagerListenersSelfTest extends GridCommonAbstrac
             try {
                 G.ignite(getTestGridName()).compute().execute(t.getClass(), null);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 assert false : "Failed to execute task [iteration=" + i + ", err=" + e.getMessage() + ']';
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
index db3b76d..8bbd797 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAbstractFullApiSelfTest.java
@@ -3244,7 +3244,12 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
             assert !forLocal(dfltIgnite).call(new Callable<Boolean>() {
                 @Override public Boolean call() throws IgniteException {
-                    return cache().lock("key", 100);
+                    try {
+                        return cache().lock("key", 100);
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw U.wrap(e);
+                    }
                 }
             });
 
@@ -3272,7 +3277,12 @@ public abstract class GridCacheAbstractFullApiSelfTest extends GridCacheAbstract
 
             assert !forLocal(dfltIgnite).call(new Callable<Boolean>() {
                 @Override public Boolean call() throws IgniteException {
-                    return e.lock(100);
+                    try {
+                        return e.lock(100);
+                    }
+                    catch (IgniteCheckedException e1) {
+                        throw U.wrap(e1);
+                    }
                 }
             });
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAffinityRoutingSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAffinityRoutingSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAffinityRoutingSelfTest.java
index d49d0fe..efdc396 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAffinityRoutingSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheAffinityRoutingSelfTest.java
@@ -226,7 +226,7 @@ public class GridCacheAffinityRoutingSelfTest extends GridCommonAbstractTest {
 
             fail();
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Caught expected exception: " + e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
index 1cd917c..dc845a4 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheConcurrentTxMultiNodeTest.java
@@ -327,7 +327,7 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
 
                     latency.addAndGet(t1 - t0);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
             }
@@ -681,7 +681,7 @@ public class GridCacheConcurrentTxMultiNodeTest extends GridCommonAbstractTest {
 
                 return entry.getValue();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 e.printStackTrace();
 
                 return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
index c812755..24acbd9 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockFailoverSelfTest.java
@@ -328,7 +328,7 @@ public class GridCacheGroupLockFailoverSelfTest extends GridCommonAbstractTest {
                 try {
                     f.get(); //if something went wrong - we'll get exception here
                 }
-                catch (IgniteCheckedException ignore) {
+                catch (IgniteException ignore) {
                     info("Put task failed, going to remap keys: " + dataChunk.size());
 
                     fail = true;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java
index 718205b..013f039 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheGroupLockPutTask.java
@@ -17,6 +17,7 @@ import org.apache.ignite.resources.*;
 import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.util.*;
@@ -97,12 +98,17 @@ class GridCacheGroupLockPutTask extends ComputeTaskAdapter<Collection<Integer>,
                         Object affKey = pair.get1();
 
                         // Group lock partition.
-                        try (GridCacheTx tx = cache.txStartPartition(cache.affinity().partition(affKey),
-                            optimistic ? OPTIMISTIC : PESSIMISTIC, REPEATABLE_READ, 0, pair.get2().size())) {
-                            for (Integer val : pair.get2())
-                                cache.put(val, val);
-
-                            tx.commit();
+                        try {
+                            try (GridCacheTx tx = cache.txStartPartition(cache.affinity().partition(affKey),
+                                optimistic ? OPTIMISTIC : PESSIMISTIC, REPEATABLE_READ, 0, pair.get2().size())) {
+                                for (Integer val : pair.get2())
+                                    cache.put(val, val);
+
+                                tx.commit();
+                            }
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw U.wrap(e);
                         }
                     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
index 7998316..03dd609 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCachePutAllFailoverSelfTest.java
@@ -249,7 +249,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
                             try {
                                 taskFut.get(); //if something went wrong - we'll get exception here
                             }
-                            catch (IgniteCheckedException e) {
+                            catch (IgniteException e) {
                                 log.error("Job failed", e);
 
                                 jobFailed.set(true);
@@ -421,7 +421,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
                             try {
                                 taskFut.get(); //if something went wrong - we'll get exception here
                             }
-                            catch (IgniteCheckedException e) {
+                            catch (IgniteException e) {
                                 log.error("Job failed", e);
 
                                 jobFailed.set(true);
@@ -472,7 +472,7 @@ public class GridCachePutAllFailoverSelfTest extends GridCommonAbstractTest {
                         try {
                             taskFut.get(); //if something went wrong - we'll get exception here
                         }
-                        catch (IgniteCheckedException e) {
+                        catch (IgniteException e) {
                             log.error("Job failed", e);
 
                             jobFailed.set(true);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java
index 3402297..ae5e63a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheStopSelfTest.java
@@ -170,7 +170,7 @@ public class GridCacheStopSelfTest extends GridCommonAbstractTest {
                 try {
                     fut.get();
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     if (!e.getMessage().startsWith(EXPECTED_MSG))
                         e.printStackTrace();
 


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

Posted by sb...@apache.org.
# IGNITE-26 Replaced IgniteCheckedException with IgniteException on public API.


Project: http://git-wip-us.apache.org/repos/asf/incubator-ignite/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-ignite/commit/4307bca1
Tree: http://git-wip-us.apache.org/repos/asf/incubator-ignite/tree/4307bca1
Diff: http://git-wip-us.apache.org/repos/asf/incubator-ignite/diff/4307bca1

Branch: refs/heads/ignite-26
Commit: 4307bca1e17aebd5b71c5b7980e625978a236f00
Parents: 8bc850c
Author: sevdokimov <se...@gridgain.com>
Authored: Tue Dec 16 21:22:42 2014 +0300
Committer: sevdokimov <se...@gridgain.com>
Committed: Tue Dec 16 21:22:42 2014 +0300

----------------------------------------------------------------------
 .../compute/ComputeScheduleExample.java         |   2 +-
 .../failover/ComputeFailoverExample.java        |   2 +-
 .../examples/datagrid/CacheApiExample.java      |   2 +-
 .../datagrid/CachePopularNumbersExample.java    |   2 +-
 .../examples/messaging/MessagingExample.java    |   4 +-
 .../messaging/MessagingPingPongExample.java     |   4 +-
 .../streaming/StreamingCheckInExample.java      |   2 +-
 .../StreamingPopularNumbersExample.java         |   2 +-
 .../streaming/StreamingPriceBarsExample.java    |   2 +-
 .../StreamingRunningAverageExample.java         |   4 +-
 .../singlesplit/GridSingleSplitsLoadTest.java   |   2 +-
 .../GridClientTestPortableAffinityKeyTask.java  |   2 +-
 .../client/GridTaskSingleJobSplitAdapter.java   |   2 +-
 .../kernal/GridTaskEventSubjectIdSelfTest.java  |   2 +-
 .../ignite/messaging/MessagingListenActor.java  |  14 +-
 .../sharedfs/SharedFsCheckpointSpi.java         |   7 +-
 .../sharedfs/SharedFsTimeoutTask.java           |   4 +-
 .../discovery/tcp/TcpClientDiscoverySpi.java    |   2 +-
 .../spi/failover/always/AlwaysFailoverSpi.java  |   2 +-
 .../jobstealing/JobStealingFailoverSpi.java     |   2 +-
 .../spi/swapspace/file/FileSwapSpaceSpi.java    |   2 +-
 .../index/tree/StreamerTreeIndexProvider.java   |  10 +-
 .../window/StreamerBoundedTimeWindow.java       |   4 +-
 .../GridClientConnectionManagerAdapter.java     |   2 +-
 .../GridClientOptimizedMarshaller.java          |   6 +-
 .../impl/GridRouterCommandLineStartup.java      |   4 +-
 .../org/gridgain/grid/cache/GridCacheTx.java    |   4 +-
 .../cache/GridCacheTxHeuristicException.java    |   2 +-
 .../GridCacheRendezvousAffinityFunction.java    |  23 +-
 .../grid/kernal/GridKernalContextImpl.java      |   2 +-
 .../kernal/executor/GridExecutorService.java    |  30 +--
 .../ggfs/common/GridGgfsControlResponse.java    |   6 +-
 .../cache/GridCacheTxLocalAdapter.java          |   2 +-
 .../processors/cache/GridCacheTxProxyImpl.java  |   2 +-
 .../distributed/dht/GridDhtTxFinishFuture.java  |   2 +-
 .../query/GridCacheDistributedQueryManager.java |   4 +-
 .../processors/ggfs/GridGgfsAsyncImpl.java      |   8 +-
 .../kernal/processors/ggfs/GridGgfsImpl.java    | 145 +++++++----
 .../ggfs/GridGgfsInputStreamImpl.java           |   2 +-
 .../processors/ggfs/GridGgfsMetaManager.java    |   9 +-
 .../kernal/processors/ggfs/GridGgfsServer.java  |   4 +-
 .../ggfs/IgniteFsOutputStreamImpl.java          |   2 +-
 .../cache/GridCacheQueryCommandHandler.java     |   6 +-
 .../handlers/task/GridTaskCommandHandler.java   |   4 +-
 .../service/GridServiceNotFoundException.java   |   2 +-
 .../grid/util/future/GridEmbeddedFuture.java    |  10 +-
 .../util/gridify/GridifyArgumentBuilder.java    |   6 +-
 .../grid/util/gridify/GridifyJobAdapter.java    |  12 +-
 .../org/gridgain/grid/util/lang/GridFunc.java   |  11 +-
 .../org/gridgain/jdbc/util/GridJdbcUtils.java   |   4 +-
 .../optimized/GridOptimizedMarshallerTest.java  |   2 +-
 .../GridOptimizedObjectStreamSelfTest.java      |   2 +-
 ...heckpointSpiMultipleDirectoriesSelfTest.java |   2 +-
 .../tcp/GridCacheDhtLockBackupSelfTest.java     |   2 +-
 .../tcp/GridOrderedMessageCancelSelfTest.java   |   2 +-
 .../tcp/GridTcpCommunicationSpiLanTest.java     |   2 +-
 ...cpCommunicationSpiMultithreadedSelfTest.java |   6 +-
 ...RobinLoadBalancingSpiNotPerTaskSelfTest.java |   2 +-
 .../index/GridStreamerIndexSelfTest.java        |   8 +-
 .../org/gridgain/grid/GridTestJobResult.java    |   6 +-
 .../ggfs/GridGgfsEventsAbstractSelfTest.java    |   2 +-
 .../GridAlwaysFailoverSpiFailSelfTest.java      |   4 +-
 .../kernal/GridCancelOnGridStopSelfTest.java    |   4 +-
 .../GridCancelledJobsMetricsSelfTest.java       |   6 +-
 .../grid/kernal/GridCommunicationSelfTest.java  |   2 +-
 .../GridContinuousJobAnnotationSelfTest.java    |   6 +-
 .../GridContinuousJobSiblingsSelfTest.java      |   2 +-
 .../grid/kernal/GridContinuousTaskSelfTest.java |   6 +-
 .../GridDeploymentMultiThreadedSelfTest.java    |   2 +-
 .../grid/kernal/GridDeploymentSelfTest.java     |   2 +-
 .../GridEventStorageCheckAllEventsSelfTest.java |   2 +-
 .../GridExplicitImplicitDeploymentSelfTest.java |   3 +-
 .../grid/kernal/GridFactoryVmShutdownTest.java  |   2 +-
 .../GridFailoverCustomTopologySelfTest.java     |   2 +-
 .../grid/kernal/GridFailoverSelfTest.java       |   4 +-
 .../GridFailoverTaskWithPredicateSelfTest.java  |   4 +-
 .../kernal/GridFailoverTopologySelfTest.java    |   4 +-
 .../kernal/GridJobMasterLeaveAwareSelfTest.java |   2 +-
 .../grid/kernal/GridJobStealingSelfTest.java    |   2 +-
 .../GridJobStealingZeroActiveJobsSelfTest.java  |   2 +-
 .../grid/kernal/GridLifecycleBeanSelfTest.java  |   2 +-
 .../grid/kernal/GridListenActorSelfTest.java    |   1 +
 .../grid/kernal/GridMultipleSpisSelfTest.java   |   4 +-
 .../GridMultipleVersionsDeploymentSelfTest.java |   2 +-
 .../GridMultithreadedJobStealingSelfTest.java   |   4 +-
 .../kernal/GridRuntimeExceptionSelfTest.java    |   8 +-
 .../grid/kernal/GridSpiExceptionSelfTest.java   |   2 +-
 .../grid/kernal/GridStopWithCancelSelfTest.java |   4 +-
 .../grid/kernal/GridStopWithWaitSelfTest.java   |   4 +-
 .../GridTaskContinuousMapperSelfTest.java       |  12 +-
 .../kernal/GridTaskInstantiationSelfTest.java   |   2 +-
 .../GridCheckpointManagerAbstractSelfTest.java  |  30 +--
 ...idCommunicationManagerListenersSelfTest.java |   2 +-
 .../cache/GridCacheAbstractFullApiSelfTest.java |  14 +-
 .../cache/GridCacheAffinityRoutingSelfTest.java |   2 +-
 .../GridCacheConcurrentTxMultiNodeTest.java     |   4 +-
 .../GridCacheGroupLockFailoverSelfTest.java     |   2 +-
 .../cache/GridCacheGroupLockPutTask.java        |  18 +-
 .../cache/GridCachePutAllFailoverSelfTest.java  |   6 +-
 .../processors/cache/GridCacheStopSelfTest.java |   2 +-
 .../cache/GridCacheTxMultiNodeAbstractTest.java | 257 ++++++++++---------
 ...cheAtomicReferenceMultiNodeAbstractTest.java |  94 ++++---
 ...GridCacheQueueMultiNodeAbstractSelfTest.java |  40 ++-
 ...CacheQueueRotativeMultiNodeAbstractTest.java |  57 +++-
 ...dCacheSequenceMultiNodeAbstractSelfTest.java |  57 ++--
 ...dCachePartitionedQueueEntryMoveSelfTest.java |  20 +-
 .../dht/GridCacheDhtPreloadUnloadSelfTest.java  |  12 +-
 ...achePartitionedPreloadLifecycleSelfTest.java |  15 +-
 .../closure/GridClosureProcessorSelfTest.java   |   2 +-
 .../continuous/GridEventConsumeSelfTest.java    |   4 +-
 ...heGgfsPerBlockLruEvictionPolicySelfTest.java |   2 +-
 .../ggfs/GridGgfsAbstractSelfTest.java          |  44 ++--
 .../ggfs/GridGgfsDualAbstractSelfTest.java      |  12 +-
 .../ggfs/GridGgfsFileInfoSelfTest.java          |   8 +-
 .../GridGgfsProcessorValidationSelfTest.java    |   2 +-
 .../cache/GridCacheCommandHandlerSelfTest.java  |   4 +-
 .../GridServiceProcessorAbstractSelfTest.java   |   2 +-
 .../lang/GridFutureListenPerformanceTest.java   |   2 +-
 ...idFileSwapSpaceSpiMultithreadedLoadTest.java |   2 +-
 .../p2p/GridP2PLocalDeploymentSelfTest.java     |   2 +-
 .../GridP2PMissedResourceCacheSizeSelfTest.java |   6 +-
 .../grid/p2p/GridP2PTimeoutSelfTest.java        |   4 +-
 .../GridSessionCheckpointAbstractSelfTest.java  |   4 +-
 .../GridSessionCollisionSpiSelfTest.java        |   2 +-
 .../session/GridSessionJobFailoverSelfTest.java |   6 +-
 ...GridSessionJobWaitTaskAttributeSelfTest.java |   2 +-
 .../GridSessionSetFutureAttributeSelfTest.java  |   2 +-
 ...nSetFutureAttributeWaitListenerSelfTest.java |   2 +-
 .../GridSessionSetJobAttribute2SelfTest.java    |   2 +-
 ...GridSessionSetJobAttributeOrderSelfTest.java |   4 +-
 .../GridSessionSetJobAttributeSelfTest.java     |   2 +-
 ...sionSetJobAttributeWaitListenerSelfTest.java |   2 +-
 .../GridSessionSetTaskAttributeSelfTest.java    |   2 +-
 ...GridSessionTaskWaitJobAttributeSelfTest.java |   2 +-
 .../GridSessionWaitAttributeSelfTest.java       |   6 +-
 .../util/future/GridEmbeddedFutureSelfTest.java |   2 +-
 .../future/GridFutureListenPerformanceTest.java |   2 +-
 .../offheap/GridOffHeapMapAbstractSelfTest.java |   2 +-
 ...idOffHeapPartitionedMapAbstractSelfTest.java |   2 +-
 .../colocation/GridTestLifecycleBean.java       |   8 +-
 .../communication/GridIoManagerBenchmark0.java  |   2 +-
 .../multisplit/GridMultiSplitsLoadTest.java     |   2 +-
 .../GridMultiSplitsRedeployLoadTest.java        |   2 +-
 .../GridSingleSplitsRedeployLoadTest.java       |   2 +-
 .../direct/session/GridSessionLoadTestTask.java |   2 +-
 .../gridgain/loadtests/dsi/GridDsiClient.java   |   3 +-
 .../loadtests/dsi/GridDsiLifecycleBean.java     |   8 +-
 .../job/GridJobExecutionLoadTestClient.java     |   6 +-
 ...GridJobExecutionLoadTestClientSemaphore.java |   4 +-
 .../job/GridJobExecutionSingleNodeLoadTest.java |   2 +-
 .../loadtests/job/GridJobLoadTestJob.java       |   2 +-
 .../loadtests/job/GridJobLoadTestSubmitter.java |   6 +-
 .../mapper/GridContinuousMapperTask1.java       |   4 +-
 .../loadtests/streamer/EventClosure.java        |   2 +-
 .../hadoop/GridHadoopMapReduceTest.java         |   2 +-
 .../GridCacheAtomicFieldsQuerySelfTest.java     |   2 +-
 .../grid/kernal/GridScheduleSelfTest.java       |  16 +-
 .../GridResourceConcurrentUndeploySelfTest.java |   2 +-
 .../GridResourceFieldInjectionSelfTest.java     |   8 +-
 .../resource/GridResourceIocSelfTest.java       |   2 +-
 ...GridResourceIsolatedClassLoaderSelfTest.java |   2 +-
 .../GridResourceIsolatedTaskSelfTest.java       |   2 +-
 .../GridResourceSharedUndeploySelfTest.java     |   2 +-
 .../resource/GridServiceInjectionSelfTest.java  |   4 +-
 .../grid/p2p/GridP2PDisabledSelfTest.java       |   2 +-
 165 files changed, 802 insertions(+), 608 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/compute/ComputeScheduleExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/compute/ComputeScheduleExample.java b/examples/src/main/java/org/gridgain/examples/compute/ComputeScheduleExample.java
index e80e1f2..37b33bf 100644
--- a/examples/src/main/java/org/gridgain/examples/compute/ComputeScheduleExample.java
+++ b/examples/src/main/java/org/gridgain/examples/compute/ComputeScheduleExample.java
@@ -58,7 +58,7 @@ public class ComputeScheduleExample {
                                 }
                             );
                         }
-                        catch (IgniteCheckedException e) {
+                        catch (IgniteException e) {
                             throw new IgniteException(e);
                         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/compute/failover/ComputeFailoverExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/compute/failover/ComputeFailoverExample.java b/examples/src/main/java/org/gridgain/examples/compute/failover/ComputeFailoverExample.java
index dff46a3..436f0c2 100644
--- a/examples/src/main/java/org/gridgain/examples/compute/failover/ComputeFailoverExample.java
+++ b/examples/src/main/java/org/gridgain/examples/compute/failover/ComputeFailoverExample.java
@@ -119,7 +119,7 @@ public class ComputeFailoverExample {
 
                 return sum;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 throw new GridClosureException(e);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/datagrid/CacheApiExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/datagrid/CacheApiExample.java b/examples/src/main/java/org/gridgain/examples/datagrid/CacheApiExample.java
index a30c20f..1564af1 100644
--- a/examples/src/main/java/org/gridgain/examples/datagrid/CacheApiExample.java
+++ b/examples/src/main/java/org/gridgain/examples/datagrid/CacheApiExample.java
@@ -80,7 +80,7 @@ public class CacheApiExample {
                 try {
                     System.out.println("Put operation completed [previous-value=" + fut.get() + ']');
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/datagrid/CachePopularNumbersExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/datagrid/CachePopularNumbersExample.java b/examples/src/main/java/org/gridgain/examples/datagrid/CachePopularNumbersExample.java
index 69a1216..aee3780 100644
--- a/examples/src/main/java/org/gridgain/examples/datagrid/CachePopularNumbersExample.java
+++ b/examples/src/main/java/org/gridgain/examples/datagrid/CachePopularNumbersExample.java
@@ -137,7 +137,7 @@ public class CachePopularNumbersExample {
 
                     System.out.println("----------------");
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/messaging/MessagingExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/messaging/MessagingExample.java b/examples/src/main/java/org/gridgain/examples/messaging/MessagingExample.java
index fa51a78..4e0ce98 100644
--- a/examples/src/main/java/org/gridgain/examples/messaging/MessagingExample.java
+++ b/examples/src/main/java/org/gridgain/examples/messaging/MessagingExample.java
@@ -106,7 +106,7 @@ public final class MessagingExample {
                 try {
                     g.message(g.cluster().forNodeId(nodeId)).send(TOPIC.ORDERED, msg);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
 
@@ -125,7 +125,7 @@ public final class MessagingExample {
                 try {
                     g.message(g.cluster().forNodeId(nodeId)).send(TOPIC.UNORDERED, msg);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/messaging/MessagingPingPongExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/messaging/MessagingPingPongExample.java b/examples/src/main/java/org/gridgain/examples/messaging/MessagingPingPongExample.java
index ed9585b..e164a2b 100644
--- a/examples/src/main/java/org/gridgain/examples/messaging/MessagingPingPongExample.java
+++ b/examples/src/main/java/org/gridgain/examples/messaging/MessagingPingPongExample.java
@@ -73,7 +73,7 @@ public class MessagingPingPongExample {
 
                         return false; // Unsubscribe.
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         throw new GridClosureException(e);
                     }
                 }
@@ -105,7 +105,7 @@ public class MessagingPingPongExample {
 
                         return true; // Continue listening.
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         throw new GridClosureException(e);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/streaming/StreamingCheckInExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/streaming/StreamingCheckInExample.java b/examples/src/main/java/org/gridgain/examples/streaming/StreamingCheckInExample.java
index d0826e9..3920c84 100644
--- a/examples/src/main/java/org/gridgain/examples/streaming/StreamingCheckInExample.java
+++ b/examples/src/main/java/org/gridgain/examples/streaming/StreamingCheckInExample.java
@@ -214,7 +214,7 @@ public class StreamingCheckInExample {
 
                     System.out.print(sb.toString());
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/streaming/StreamingPopularNumbersExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/streaming/StreamingPopularNumbersExample.java b/examples/src/main/java/org/gridgain/examples/streaming/StreamingPopularNumbersExample.java
index e4bf863..b385da6 100644
--- a/examples/src/main/java/org/gridgain/examples/streaming/StreamingPopularNumbersExample.java
+++ b/examples/src/main/java/org/gridgain/examples/streaming/StreamingPopularNumbersExample.java
@@ -178,7 +178,7 @@ public class StreamingPopularNumbersExample {
 
                     System.out.println("----------------");
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/streaming/StreamingPriceBarsExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/streaming/StreamingPriceBarsExample.java b/examples/src/main/java/org/gridgain/examples/streaming/StreamingPriceBarsExample.java
index 23e9443..82d352f 100644
--- a/examples/src/main/java/org/gridgain/examples/streaming/StreamingPriceBarsExample.java
+++ b/examples/src/main/java/org/gridgain/examples/streaming/StreamingPriceBarsExample.java
@@ -158,7 +158,7 @@ public class StreamingPriceBarsExample {
 
                     System.out.println("-----------------");
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/examples/src/main/java/org/gridgain/examples/streaming/StreamingRunningAverageExample.java
----------------------------------------------------------------------
diff --git a/examples/src/main/java/org/gridgain/examples/streaming/StreamingRunningAverageExample.java b/examples/src/main/java/org/gridgain/examples/streaming/StreamingRunningAverageExample.java
index cd29378..2432557 100644
--- a/examples/src/main/java/org/gridgain/examples/streaming/StreamingRunningAverageExample.java
+++ b/examples/src/main/java/org/gridgain/examples/streaming/StreamingRunningAverageExample.java
@@ -93,7 +93,7 @@ public class StreamingRunningAverageExample {
 
                         System.out.println("Got streamer query result [avg=" + avg + ", idealAvg=" + (rndRange / 2) + ']');
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         System.out.println("Failed to execute streamer query: " + e);
                     }
                 }
@@ -110,7 +110,7 @@ public class StreamingRunningAverageExample {
                     try {
                         streamer.addEvent(rnd.nextInt(rndRange));
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         System.out.println("Failed to add streamer event: " + e);
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/aop/src/test/java/org/gridgain/loadtests/direct/singlesplit/GridSingleSplitsLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/aop/src/test/java/org/gridgain/loadtests/direct/singlesplit/GridSingleSplitsLoadTest.java b/modules/aop/src/test/java/org/gridgain/loadtests/direct/singlesplit/GridSingleSplitsLoadTest.java
index 0266d3a..eed3f4f 100644
--- a/modules/aop/src/test/java/org/gridgain/loadtests/direct/singlesplit/GridSingleSplitsLoadTest.java
+++ b/modules/aop/src/test/java/org/gridgain/loadtests/direct/singlesplit/GridSingleSplitsLoadTest.java
@@ -114,7 +114,7 @@ public class GridSingleSplitsLoadTest extends GridCommonAbstractTest {
                         if (taskCnt % 500 == 0)
                             info(stats.toString());
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         error("Failed to execute grid task.", e);
 
                         fail();

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/clients/src/test/java/org/gridgain/client/GridClientTestPortableAffinityKeyTask.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/gridgain/client/GridClientTestPortableAffinityKeyTask.java b/modules/clients/src/test/java/org/gridgain/client/GridClientTestPortableAffinityKeyTask.java
index ae7d11f..8c711b2 100644
--- a/modules/clients/src/test/java/org/gridgain/client/GridClientTestPortableAffinityKeyTask.java
+++ b/modules/clients/src/test/java/org/gridgain/client/GridClientTestPortableAffinityKeyTask.java
@@ -39,7 +39,7 @@ public class GridClientTestPortableAffinityKeyTask extends ComputeTaskAdapter<Ob
                 }, node);
         }
 
-        throw new IgniteCheckedException("Failed to find local node in task topology: " + clusterNodes);
+        throw new IgniteException("Failed to find local node in task topology: " + clusterNodes);
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/clients/src/test/java/org/gridgain/client/GridTaskSingleJobSplitAdapter.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/gridgain/client/GridTaskSingleJobSplitAdapter.java b/modules/clients/src/test/java/org/gridgain/client/GridTaskSingleJobSplitAdapter.java
index 53d205d..82767cd 100644
--- a/modules/clients/src/test/java/org/gridgain/client/GridTaskSingleJobSplitAdapter.java
+++ b/modules/clients/src/test/java/org/gridgain/client/GridTaskSingleJobSplitAdapter.java
@@ -43,7 +43,7 @@ public abstract class GridTaskSingleJobSplitAdapter<T, R> extends ComputeTaskSpl
         ComputeJobResult res = results.get(0);
 
         if (res.isCancelled())
-            throw new IgniteCheckedException("Reduce receives failed job.");
+            throw new IgniteException("Reduce receives failed job.");
 
         return res.getData();
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/clients/src/test/java/org/gridgain/grid/kernal/GridTaskEventSubjectIdSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/clients/src/test/java/org/gridgain/grid/kernal/GridTaskEventSubjectIdSelfTest.java b/modules/clients/src/test/java/org/gridgain/grid/kernal/GridTaskEventSubjectIdSelfTest.java
index d294460..1f86599 100644
--- a/modules/clients/src/test/java/org/gridgain/grid/kernal/GridTaskEventSubjectIdSelfTest.java
+++ b/modules/clients/src/test/java/org/gridgain/grid/kernal/GridTaskEventSubjectIdSelfTest.java
@@ -355,7 +355,7 @@ public class GridTaskEventSubjectIdSelfTest extends GridCommonAbstractTest {
 
         /** {@inheritDoc} */
         @Nullable @Override public Object reduce(List<ComputeJobResult> results) throws IgniteException {
-            throw new IgniteCheckedException("Task failed.");
+            throw new IgniteException("Task failed.");
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/apache/ignite/messaging/MessagingListenActor.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/messaging/MessagingListenActor.java b/modules/core/src/main/java/org/apache/ignite/messaging/MessagingListenActor.java
index 1235c4b..0689f26 100644
--- a/modules/core/src/main/java/org/apache/ignite/messaging/MessagingListenActor.java
+++ b/modules/core/src/main/java/org/apache/ignite/messaging/MessagingListenActor.java
@@ -154,9 +154,9 @@ public abstract class MessagingListenActor<T> implements IgniteBiPredicate<UUID,
      *
      * @param respMsg Optional response message. If not {@code null} - it will be sent to the original
      *      sender node.
-     * @throws IgniteCheckedException Thrown in case of any errors.
+     * @throws IgniteException Thrown in case of any errors.
      */
-    protected final void respond(@Nullable Object respMsg) throws IgniteCheckedException {
+    protected final void respond(@Nullable Object respMsg) throws IgniteException {
         checkReversing();
 
         keepGoing = true;
@@ -174,9 +174,9 @@ public abstract class MessagingListenActor<T> implements IgniteBiPredicate<UUID,
      * @param id ID of the node to send the message to, if any.
      * @param respMsg Optional response message. If not {@code null} - it will be sent to the original
      *      sender node.
-     * @throws IgniteCheckedException Thrown in case of any errors.
+     * @throws IgniteException Thrown in case of any errors.
      */
-    protected final void respond(UUID id, @Nullable Object respMsg) throws IgniteCheckedException {
+    protected final void respond(UUID id, @Nullable Object respMsg) throws IgniteException {
         checkReversing();
 
         keepGoing = true;
@@ -198,9 +198,9 @@ public abstract class MessagingListenActor<T> implements IgniteBiPredicate<UUID,
      *
      * @param nodeId ID of the node to send message to.
      * @param respMsg Message to send.
-     * @throws IgniteCheckedException Thrown in case of any errors.
+     * @throws IgniteException Thrown in case of any errors.
      */
-    private void send(UUID nodeId, @Nullable Object respMsg) throws IgniteCheckedException {
+    private void send(UUID nodeId, @Nullable Object respMsg) throws IgniteException {
         assert nodeId != null;
 
         if (respMsg != null) {
@@ -209,7 +209,7 @@ public abstract class MessagingListenActor<T> implements IgniteBiPredicate<UUID,
             if (node != null)
                 ignite.message(ignite.cluster().forNode(node)).send(null, respMsg); // Can still fail.
             else
-                throw new IgniteCheckedException("Failed to send message since destination node has " +
+                throw new IgniteException("Failed to send message since destination node has " +
                     "left topology (ignoring) [nodeId=" +nodeId + ", respMsg=" + respMsg + ']');
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
index 3c93370..ffb4730 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsCheckpointSpi.java
@@ -13,7 +13,6 @@ import org.apache.ignite.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.resources.*;
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
 import org.apache.ignite.spi.checkpoint.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
@@ -302,7 +301,7 @@ public class SharedFsCheckpointSpi extends IgniteSpiAdapter implements Checkpoin
                                 log.debug("Registered existing checkpoint from: " + file.getAbsolutePath());
                         }
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         U.error(log, "Failed to unmarshal objects in checkpoint file (ignoring): " +
                             file.getAbsolutePath(), e);
                     }
@@ -363,7 +362,7 @@ public class SharedFsCheckpointSpi extends IgniteSpiAdapter implements Checkpoin
                         null
                     : null;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 throw new IgniteSpiException("Failed to unmarshal objects in checkpoint file: " +
                     file.getAbsolutePath(), e);
             }
@@ -413,7 +412,7 @@ public class SharedFsCheckpointSpi extends IgniteSpiAdapter implements Checkpoin
                     throw new IgniteSpiException("Failed to write checkpoint data into file: " +
                         file.getAbsolutePath(), e);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 throw new IgniteSpiException("Failed to marshal checkpoint data into file: " +
                     file.getAbsolutePath(), e);
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsTimeoutTask.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsTimeoutTask.java b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsTimeoutTask.java
index 442caef..ff6c4c8 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsTimeoutTask.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/checkpoint/sharedfs/SharedFsTimeoutTask.java
@@ -12,9 +12,9 @@ package org.apache.ignite.spi.checkpoint.sharedfs;
 import org.apache.ignite.*;
 import org.apache.ignite.marshaller.*;
 import org.apache.ignite.spi.*;
-import org.gridgain.grid.*;
 import org.apache.ignite.spi.checkpoint.*;
 import org.gridgain.grid.util.typedef.internal.*;
+
 import java.io.*;
 import java.util.*;
 
@@ -106,7 +106,7 @@ class SharedFsTimeoutTask extends IgniteSpiThread {
                         if (timeData.getLastAccessTime() != file.lastModified())
                             timeData.setExpireTime(SharedFsUtils.read(file, marshaller, log).getExpireTime());
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         U.error(log, "Failed to marshal/unmarshal in checkpoint file: " + file.getAbsolutePath(), e);
 
                         continue;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
index bf1b783..a3c66d8 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/discovery/tcp/TcpClientDiscoverySpi.java
@@ -741,7 +741,7 @@ public class TcpClientDiscoverySpi extends TcpDiscoverySpiAdapter implements Tcp
 
                         msgWrk.addMessage(msg);
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         if (log.isDebugEnabled())
                             U.error(log, "Failed to read message [sock=" + sock0 + ", locNodeId=" + locNodeId +
                                 ", rmtNodeId=" + nodeId + ']', e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
index 46a071b..7c1cc21 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/failover/always/AlwaysFailoverSpi.java
@@ -219,7 +219,7 @@ public class AlwaysFailoverSpi extends IgniteSpiAdapter implements FailoverSpi,
 
             return node;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             U.error(log, "Failed to get next balanced node for failover: " + ctx, e);
 
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java
index 8fea012..20662b6 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/failover/jobstealing/JobStealingFailoverSpi.java
@@ -324,7 +324,7 @@ public class JobStealingFailoverSpi extends IgniteSpiAdapter implements Failover
 
             return thief;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             U.error(log, "Failed to get next balanced node for failover: " + ctx, e);
 
             return null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
index 45406dc..c27eda1 100644
--- a/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
+++ b/modules/core/src/main/java/org/apache/ignite/spi/swapspace/file/FileSwapSpaceSpi.java
@@ -556,7 +556,7 @@ public class FileSwapSpaceSpi extends IgniteSpiAdapter implements SwapSpaceSpi,
             try {
                 keyBytes = marsh.marshal(key.key());
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 throw new IgniteSpiException("Failed to marshal key: " + key.key(), e);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/apache/ignite/streamer/index/tree/StreamerTreeIndexProvider.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/streamer/index/tree/StreamerTreeIndexProvider.java b/modules/core/src/main/java/org/apache/ignite/streamer/index/tree/StreamerTreeIndexProvider.java
index a9a81a4..4c188b8 100644
--- a/modules/core/src/main/java/org/apache/ignite/streamer/index/tree/StreamerTreeIndexProvider.java
+++ b/modules/core/src/main/java/org/apache/ignite/streamer/index/tree/StreamerTreeIndexProvider.java
@@ -129,7 +129,7 @@ public class StreamerTreeIndexProvider<E, K, V> extends StreamerIndexProviderAda
 
             if (isUnique()) {
                 if (old != null)
-                    throw new IgniteCheckedException("Index unique key violation [evt=" + evt + ", key=" + key +
+                    throw new IgniteException("Index unique key violation [evt=" + evt + ", key=" + key +
                         ", idxKey=" + idxKey + ']');
             }
             else
@@ -190,7 +190,7 @@ public class StreamerTreeIndexProvider<E, K, V> extends StreamerIndexProviderAda
 
                 if (isUnique()) {
                     if (old != null)
-                        throw new IgniteCheckedException("Index unique key violation [evt=" + evt + ", key=" + key +
+                        throw new IgniteException("Index unique key violation [evt=" + evt + ", key=" + key +
                             ", idxKey=" + newIdxKey + ']');
                 }
                 else
@@ -285,7 +285,7 @@ public class StreamerTreeIndexProvider<E, K, V> extends StreamerIndexProviderAda
 
                 if (isUnique()) {
                     if (old != null)
-                        throw new IgniteCheckedException("Index unique key violation [evt=" + evt + ", key=" + key +
+                        throw new IgniteException("Index unique key violation [evt=" + evt + ", key=" + key +
                             ", idxKey=" + newIdxKey + ']');
                 }
                 else
@@ -310,10 +310,10 @@ public class StreamerTreeIndexProvider<E, K, V> extends StreamerIndexProviderAda
      * @param key2 Key.
      * @param order Keys comparison result.
      * @param sync Sync.
-     * @throws IgniteCheckedException If interrupted.
+     * @throws IgniteException If interrupted.
      */
     private void lockKeys(IndexKey<V> key1, IndexKey<V> key2, int order, StreamerIndexUpdateSync sync)
-        throws IgniteCheckedException {
+        throws IgniteException {
         assert isUnique();
         assert key1 != null;
         assert key2 != null;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/apache/ignite/streamer/window/StreamerBoundedTimeWindow.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/apache/ignite/streamer/window/StreamerBoundedTimeWindow.java b/modules/core/src/main/java/org/apache/ignite/streamer/window/StreamerBoundedTimeWindow.java
index 2ab5a9d..e6ed797 100644
--- a/modules/core/src/main/java/org/apache/ignite/streamer/window/StreamerBoundedTimeWindow.java
+++ b/modules/core/src/main/java/org/apache/ignite/streamer/window/StreamerBoundedTimeWindow.java
@@ -96,12 +96,12 @@ public class StreamerBoundedTimeWindow<E> extends StreamerWindowAdapter<E> {
     /** {@inheritDoc} */
     @Override public void checkConfiguration() throws IgniteException {
         if (timeInterval <= 0)
-            throw new IgniteCheckedException("Failed to initialize window (timeInterval must be positive): [windowClass=" +
+            throw new IgniteException("Failed to initialize window (timeInterval must be positive): [windowClass=" +
                 getClass().getSimpleName() + ", maxSize=" + maxSize + ", timeInterval=" + timeInterval + ", unique=" +
                 unique + ']');
 
         if (maxSize < 0)
-            throw new IgniteCheckedException("Failed to initialize window (maximumSize cannot be negative): [windowClass=" +
+            throw new IgniteException("Failed to initialize window (maximumSize cannot be negative): [windowClass=" +
                 getClass().getSimpleName() + ", maxSize=" + maxSize + ", timeInterval=" + timeInterval + ", unique=" +
                 unique + ']');
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java b/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java
index a5a27a6..023431a 100644
--- a/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/client/impl/connection/GridClientConnectionManagerAdapter.java
@@ -473,7 +473,7 @@ abstract class GridClientConnectionManagerAdapter implements GridClientConnectio
                 if (cfg.getSecurityCredentialsProvider() != null)
                     cred = cfg.getSecurityCredentialsProvider().credentials();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 throw new GridClientException("Failed to obtain client credentials.", e);
             }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/client/marshaller/optimized/GridClientOptimizedMarshaller.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/client/marshaller/optimized/GridClientOptimizedMarshaller.java b/modules/core/src/main/java/org/gridgain/client/marshaller/optimized/GridClientOptimizedMarshaller.java
index 5a63d93..1e14aab 100644
--- a/modules/core/src/main/java/org/gridgain/client/marshaller/optimized/GridClientOptimizedMarshaller.java
+++ b/modules/core/src/main/java/org/gridgain/client/marshaller/optimized/GridClientOptimizedMarshaller.java
@@ -55,7 +55,7 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
         try {
             opMarsh = new IgniteOptimizedMarshaller(requireSer, clsNames, clsNamesPath, poolSize);
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             throw new IOException(e);
         }
     }
@@ -79,7 +79,7 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
 
             return buf;
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             throw new IOException(e);
         }
     }
@@ -89,7 +89,7 @@ public class GridClientOptimizedMarshaller implements GridClientMarshaller {
         try {
             return opMarsh.unmarshal(bytes, null);
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             throw new IOException(e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/client/router/impl/GridRouterCommandLineStartup.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/client/router/impl/GridRouterCommandLineStartup.java b/modules/core/src/main/java/org/gridgain/client/router/impl/GridRouterCommandLineStartup.java
index 4a03f9c..87fd016 100644
--- a/modules/core/src/main/java/org/gridgain/client/router/impl/GridRouterCommandLineStartup.java
+++ b/modules/core/src/main/java/org/gridgain/client/router/impl/GridRouterCommandLineStartup.java
@@ -61,7 +61,7 @@ public class GridRouterCommandLineStartup {
                 try {
                     tcpRouter.start();
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     U.error(log, "Failed to start TCP router on port " + tcpCfg.getPort() + ": " + e.getMessage(), e);
 
                     tcpRouter = null;
@@ -78,7 +78,7 @@ public class GridRouterCommandLineStartup {
             try {
                 tcpRouter.stop();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Error while stopping the router.", e);
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTx.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTx.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTx.java
index 473d5b9..807646c 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTx.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTx.java
@@ -223,9 +223,9 @@ public interface GridCacheTx extends GridMetadataAware, AutoCloseable {
     /**
      * Ends the transaction. Transaction will be rolled back if it has not been committed.
      *
-     * @throws IgniteCheckedException If transaction could not be gracefully ended.
+     * @throws IgniteException If transaction could not be gracefully ended.
      */
-    @Override public void close() throws IgniteException;
+    @Override public void close() throws IgniteCheckedException;
 
     /**
      * Asynchronously commits this transaction by initiating {@code two-phase-commit} process.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTxHeuristicException.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTxHeuristicException.java b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTxHeuristicException.java
index 4633659..0ed873f 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTxHeuristicException.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/GridCacheTxHeuristicException.java
@@ -18,7 +18,7 @@ import org.apache.ignite.*;
  * integrity, by invalidating all values participating in this transaction
  * on remote nodes.
  */
-public class GridCacheTxHeuristicException extends IgniteCheckedException {
+public class GridCacheTxHeuristicException extends IgniteException {
     /** */
     private static final long serialVersionUID = 0L;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
index ee3cdb6..47c9557 100644
--- a/modules/core/src/main/java/org/gridgain/grid/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
+++ b/modules/core/src/main/java/org/gridgain/grid/cache/affinity/rendezvous/GridCacheRendezvousAffinityFunction.java
@@ -282,20 +282,19 @@ public class GridCacheRendezvousAffinityFunction implements GridCacheAffinityFun
         for (ClusterNode node : nodes) {
             Object nodeHash = hashIdRslvr.resolve(node);
 
-            try {
-                ByteArrayOutputStream out = new ByteArrayOutputStream();
+            ByteArrayOutputStream out = new ByteArrayOutputStream();
 
-                byte[] nodeHashBytes = marshaller.marshal(nodeHash);
+            byte[] nodeHashBytes = marshaller.marshal(nodeHash);
 
-                out.write(nodeHashBytes, 0, nodeHashBytes.length); // Avoid IOException.
-                out.write(U.intToBytes(part), 0, 4); // Avoid IOException.
+            out.write(nodeHashBytes, 0, nodeHashBytes.length); // Avoid IOException.
+            out.write(U.intToBytes(part), 0, 4); // Avoid IOException.
 
-                d.reset();
+            d.reset();
 
-                byte[] bytes = d.digest(out.toByteArray());
+            byte[] bytes = d.digest(out.toByteArray());
 
-                long hash =
-                      (bytes[0] & 0xFFL)
+            long hash =
+                (bytes[0] & 0xFFL)
                     | ((bytes[1] & 0xFFL) << 8)
                     | ((bytes[2] & 0xFFL) << 16)
                     | ((bytes[3] & 0xFFL) << 24)
@@ -304,11 +303,7 @@ public class GridCacheRendezvousAffinityFunction implements GridCacheAffinityFun
                     | ((bytes[6] & 0xFFL) << 48)
                     | ((bytes[7] & 0xFFL) << 56);
 
-                lst.add(F.t(hash, node));
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
+            lst.add(F.t(hash, node));
         }
 
         Collections.sort(lst, COMPARATOR);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernalContextImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernalContextImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernalContextImpl.java
index b6f4ac8..ef2c182 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernalContextImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/GridKernalContextImpl.java
@@ -333,7 +333,7 @@ public class GridKernalContextImpl extends GridMetadataAwareAdapter implements G
         try {
             spring = SPRING.create(false);
         }
-        catch (IgniteCheckedException ignored) {
+        catch (IgniteException ignored) {
             if (log != null && log.isDebugEnabled())
                 log.debug("Failed to load spring component, will not be able to extract userVersion from " +
                     "META-INF/gridgain.xml.");

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/executor/GridExecutorService.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/executor/GridExecutorService.java b/modules/core/src/main/java/org/gridgain/grid/kernal/executor/GridExecutorService.java
index 8be6cc1..c4c0107 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/executor/GridExecutorService.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/executor/GridExecutorService.java
@@ -158,7 +158,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
             try {
                 task.cancel();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Failed to cancel task: " + task, e);
             }
         }
@@ -212,7 +212,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
                 return false;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Failed to get task result: " + fut, e);
 
                 if (e.getCause() instanceof InterruptedException)
@@ -238,7 +238,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
             return addFuture(comp.<T>future());
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             // Should not be thrown since uses asynchronous execution.
             return addFuture(new GridFinishedFutureEx<T>(e));
         }
@@ -256,7 +256,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
             comp.run(task);
 
             IgniteFuture<T> fut = comp.future().chain(new CX1<IgniteFuture<?>, T>() {
-                @Override public T applyx(IgniteFuture<?> fut) throws IgniteCheckedException {
+                @Override public T applyx(IgniteFuture<?> fut) throws IgniteException {
                     fut.get();
 
                     return res;
@@ -265,7 +265,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
             return addFuture(fut);
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             // Should not be thrown since uses asynchronous execution.
             return addFuture(new GridFinishedFutureEx<T>(e));
         }
@@ -284,7 +284,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
             return addFuture(comp.future());
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             // Should not be thrown since uses asynchronous execution.
             return addFuture(new GridFinishedFutureEx<>(e));
         }
@@ -353,7 +353,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
                 fut = comp.future();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 // Should not be thrown since uses asynchronous execution.
                 fut = new GridFinishedFutureEx<>(e);
             }
@@ -376,7 +376,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
                     cancelFuture(fut);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     if (e.getCause() instanceof InterruptedException) {
                         // This invokeAll() method was interrupted (therefore, need to cancel all tasks).
                         // Note: that execution may be interrupted on remote node. Possible bug.
@@ -418,7 +418,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
         try {
             fut.cancel();
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             U.error(log, "Failed to cancel task: " + fut, e);
         }
     }
@@ -493,7 +493,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
                 fut = comp.future();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 // Should not be thrown since uses asynchronous execution.
                 fut = new GridFinishedFutureEx<>(e);
             }
@@ -528,7 +528,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
                     cancel = true;
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     // This invokeAll() method was interrupted (therefore, need to cancel all tasks).
                     // Note: that execution may be interrupted on remote node. Possible bug.
                     if (e.getCause() instanceof InterruptedException)
@@ -573,7 +573,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
             addFuture(comp.future());
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             // Should not be thrown since uses asynchronous execution.
             addFuture(new GridFinishedFutureEx(e));
         }
@@ -647,7 +647,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
             try {
                 fut.cancel();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Failed to cancel task: " + fut, e);
             }
 
@@ -674,7 +674,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
 
                 return res;
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 // Task cancellation may cause throwing exception.
                 if (fut.isCancelled()) {
                     RuntimeException ex = new CancellationException("Task was cancelled: " + fut);
@@ -712,7 +712,7 @@ public class GridExecutorService extends GridMetadataAwareAdapter implements Exe
             catch (ComputeTaskTimeoutException e) {
                 throw new ExecutionException("Task execution timed out during waiting for task result: " + fut, e);
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 // Task cancellation may cause throwing exception.
                 if (fut.isCancelled()) {
                     RuntimeException ex = new CancellationException("Task was cancelled: " + fut);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/ggfs/common/GridGgfsControlResponse.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/ggfs/common/GridGgfsControlResponse.java b/modules/core/src/main/java/org/gridgain/grid/kernal/ggfs/common/GridGgfsControlResponse.java
index df1ce3f..a979f4e 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/ggfs/common/GridGgfsControlResponse.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/ggfs/common/GridGgfsControlResponse.java
@@ -287,7 +287,7 @@ public class GridGgfsControlResponse extends GridGgfsMessage {
     /**
      * @param e Error if occurred.
      */
-    public void error(IgniteCheckedException e) {
+    public void error(IgniteException e) {
         err = e.getMessage();
         errCode = errorCode(e);
     }
@@ -311,7 +311,7 @@ public class GridGgfsControlResponse extends GridGgfsMessage {
      * @param e Exception to analyze.
      * @return Error code.
      */
-    private int errorCode(IgniteCheckedException e) {
+    private int errorCode(IgniteException e) {
         return errorCode(e, true);
     }
 
@@ -323,7 +323,7 @@ public class GridGgfsControlResponse extends GridGgfsMessage {
      * @return Error code.
      */
     @SuppressWarnings("unchecked")
-    private int errorCode(IgniteCheckedException e, boolean checkIo) {
+    private int errorCode(IgniteException e, boolean checkIo) {
         if (X.hasCause(e, IgniteFsFileNotFoundException.class))
             return ERR_FILE_NOT_FOUND;
         else if (IgniteFsPathAlreadyExistsException.class.isInstance(e))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
index 0f46d1c..e64da29 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxLocalAdapter.java
@@ -835,7 +835,7 @@ public abstract class GridCacheTxLocalAdapter<K, V> extends GridCacheTxAdapter<K
                             throw ex;
                         }
                         else {
-                            IgniteCheckedException err = new GridCacheTxHeuristicException("Failed to locally write to cache " +
+                            IgniteException err = new GridCacheTxHeuristicException("Failed to locally write to cache " +
                                 "(all transaction entries will be invalidated, however there was a window when " +
                                 "entries for this transaction were visible to others): " + this, ex);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxProxyImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxProxyImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxProxyImpl.java
index e51ef0f..4e61db6 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxProxyImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxProxyImpl.java
@@ -178,7 +178,7 @@ public class GridCacheTxProxyImpl<K, V> implements GridCacheTxProxy, Externaliza
     }
 
     /** {@inheritDoc} */
-    @Override public void close() throws IgniteException {
+    @Override public void close() throws IgniteCheckedException {
         enter();
 
         try {

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
index 9a3caee..3f31a46 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridDhtTxFinishFuture.java
@@ -174,7 +174,7 @@ public final class GridDhtTxFinishFuture<K, V> extends GridCompoundIdentityFutur
                 catch (GridCacheTxHeuristicException ignore) {
                     // Future should complete with GridCacheTxHeuristicException.
                 }
-                catch (IgniteCheckedException err) {
+                catch (IgniteException err) {
                     U.error(log, "Failed to invalidate transaction: " + tx, err);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
index b9d2474..093ccc7 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/cache/query/GridCacheDistributedQueryManager.java
@@ -373,7 +373,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
             try {
                 fut.cancel();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 U.error(log, "Failed to cancel running query future: " + fut, e);
             }
 
@@ -389,7 +389,7 @@ public class GridCacheDistributedQueryManager<K, V> extends GridCacheQueryManage
             try {
                 fut.get();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 if (log.isDebugEnabled())
                     log.debug("Received query error while waiting for query to finish [queryFuture= " + fut +
                         ", error= " + e + ']');

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/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 ee1b4a5..7479c56 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
@@ -14,6 +14,7 @@ import org.apache.ignite.fs.*;
 import org.apache.ignite.fs.mapreduce.*;
 import org.apache.ignite.lang.*;
 import org.gridgain.grid.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.jetbrains.annotations.*;
 
 import java.net.*;
@@ -42,7 +43,12 @@ public class GridGgfsAsyncImpl extends IgniteAsyncSupportAdapter implements Grid
 
     /** {@inheritDoc} */
     @Override public void format() throws IgniteException {
-        saveOrGet(ggfs.formatAsync());
+        try {
+            saveOrGet(ggfs.formatAsync());
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
+        }
     }
 
     /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/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 214191e..9599a32 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
@@ -470,27 +470,32 @@ public final class GridGgfsImpl implements GridGgfsEx {
         IgniteFsMode mode = modeRslvr.resolveMode(path);
 
         if (mode == PROXY)
-            throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + path);
+            throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + path);
 
         boolean res = false;
 
-        switch (mode) {
-            case PRIMARY:
-                res = meta.fileId(path) != null;
+        try {
+            switch (mode) {
+                case PRIMARY:
+                    res = meta.fileId(path) != null;
 
-                break;
+                    break;
 
-            case DUAL_SYNC:
-            case DUAL_ASYNC:
-                res = meta.fileId(path) != null;
+                case DUAL_SYNC:
+                case DUAL_ASYNC:
+                    res = meta.fileId(path) != null;
 
-                if (!res)
-                    res = secondaryFs.exists(path);
+                    if (!res)
+                        res = secondaryFs.exists(path);
 
-                break;
+                    break;
 
-            default:
-                assert false : "Unknown mode.";
+                default:
+                    assert false : "Unknown mode.";
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
         }
 
         return res;
@@ -508,7 +513,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 IgniteFsMode mode = modeRslvr.resolveMode(path);
 
                 if (mode == PROXY)
-                    throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + path);
+                    throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + path);
 
                 GridGgfsFileInfo info = resolveFileInfo(path, mode);
 
@@ -545,6 +550,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
                 return sum;
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -567,7 +575,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 IgniteFsMode mode = modeRslvr.resolveMode(path);
 
                 if (mode == PROXY)
-                    throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + path);
+                    throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + path);
                 else if (mode != PRIMARY) {
                     assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
 
@@ -601,6 +609,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 else
                     return null;
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -623,7 +634,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 Set<IgniteFsMode> childrenModes = modeRslvr.resolveChildrenModes(src);
 
                 if (mode == PROXY)
-                    throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + src);
+                    throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + src);
 
                 if (src.equals(dest))
                     return; // Rename to itself is a no-op.
@@ -710,6 +721,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
                         evts.record(new IgniteFsEvent(src, dest, localNode(), EVT_GGFS_DIR_RENAMED));
                 }
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -764,6 +778,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
                 return res;
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -834,7 +851,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 IgniteFsMode mode = modeRslvr.resolveMode(path);
 
                 if (mode == PROXY)
-                    throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + path);
+                    throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + path);
                 else if (mode != PRIMARY) {
                     assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
 
@@ -898,6 +915,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
                     parentId = fileId;
                 }
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -918,7 +938,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 IgniteFsMode mode = modeRslvr.resolveMode(path);
 
                 if (mode == PROXY)
-                    throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + path);
+                    throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + path);
 
                 Set<IgniteFsMode> childrenModes = modeRslvr.resolveChildrenModes(path);
 
@@ -950,6 +970,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
                     }
                 });
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -970,7 +993,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 IgniteFsMode mode = modeRslvr.resolveMode(path);
 
                 if (mode == PROXY)
-                    throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + path);
+                    throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + path);
 
                 Set<IgniteFsMode> childrenModes = modeRslvr.resolveChildrenModes(path);
 
@@ -1017,6 +1040,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
                 return files;
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -1063,7 +1089,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 IgniteFsMode mode = modeRslvr.resolveMode(path);
 
                 if (mode == PROXY)
-                    throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + path);
+                    throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + path);
                 else if (mode != PRIMARY) {
                     assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
 
@@ -1098,6 +1124,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
                 return os;
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -1145,7 +1174,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
         final int replication,
         @Nullable Map<String, String> props,
         final boolean simpleCreate
-    ) throws IgniteCheckedException {
+    ) throws IgniteException {
         if (enterBusy()) {
             try {
                 A.notNull(path, "path");
@@ -1160,7 +1189,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 GridGgfsFileWorkerBatch batch = null;
 
                 if (mode == PROXY)
-                    throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + path);
+                    throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + path);
                 else if (mode != PRIMARY) {
                     assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
 
@@ -1239,6 +1268,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
                 return os;
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -1269,7 +1301,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 GridGgfsFileWorkerBatch batch = null;
 
                 if (mode == PROXY)
-                    throw new IgniteCheckedException("PROXY mode cannot be used in GGFS directly: " + path);
+                    throw new IgniteException("PROXY mode cannot be used in GGFS directly: " + path);
                 else if (mode != PRIMARY) {
                     assert mode == DUAL_SYNC || mode == DUAL_ASYNC;
 
@@ -1323,6 +1355,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
                 return new GgfsEventAwareOutputStream(path, info, parentId, bufSize == 0 ?
                     cfg.getStreamBufferSize() : bufSize, mode, batch);
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -1355,6 +1390,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
                 meta.updateTimes(desc.parentId, desc.fileId, desc.fileName, accessTime, modificationTime);
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -1422,6 +1460,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
                 return data.affinity(info, start, len, maxLen);
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -1444,7 +1485,7 @@ public final class GridGgfsImpl implements GridGgfsEx {
                     try {
                         secondarySpaceSize = secondaryFs.usedSpaceSize();
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         LT.warn(log, e, "Failed to get secondary file system consumed space size.");
 
                         secondarySpaceSize = -1;
@@ -1468,6 +1509,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
                     metrics.writeBytes(),
                     metrics.writeBytesTime());
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -1498,6 +1542,9 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
                 return sum.totalLength();
             }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
             finally {
                 busyLock.leaveBusy();
             }
@@ -1535,7 +1582,12 @@ public final class GridGgfsImpl implements GridGgfsEx {
 
     /** {@inheritDoc} */
     @Override public void format() throws IgniteException {
-        formatAsync().get();
+        try {
+            formatAsync().get();
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
+        }
     }
 
     /**
@@ -1781,37 +1833,42 @@ public final class GridGgfsImpl implements GridGgfsEx {
      * @param path Path.
      * @param mode Mode.
      * @return File info or {@code null} in case file is not found.
-     * @throws IgniteCheckedException If failed.
+     * @throws IgniteException If failed.
      */
-    private GridGgfsFileInfo resolveFileInfo(IgniteFsPath path, IgniteFsMode mode) throws IgniteCheckedException {
+    private GridGgfsFileInfo resolveFileInfo(IgniteFsPath path, IgniteFsMode mode) throws IgniteException {
         assert path != null;
         assert mode != null;
 
         GridGgfsFileInfo info = null;
 
-        switch (mode) {
-            case PRIMARY:
-                info = meta.info(meta.fileId(path));
+        try {
+            switch (mode) {
+                case PRIMARY:
+                    info = meta.info(meta.fileId(path));
 
-                break;
+                    break;
 
-            case DUAL_SYNC:
-            case DUAL_ASYNC:
-                info = meta.info(meta.fileId(path));
+                case DUAL_SYNC:
+                case DUAL_ASYNC:
+                    info = meta.info(meta.fileId(path));
 
-                if (info == null) {
-                    IgniteFsFile status = secondaryFs.info(path);
+                    if (info == null) {
+                        IgniteFsFile status = secondaryFs.info(path);
 
-                    if (status != null)
-                        info = status.isDirectory() ? new GridGgfsFileInfo(true, status.properties()) :
-                            new GridGgfsFileInfo(status.blockSize(), status.length(), null, null, false,
-                            status.properties());
-                }
+                        if (status != null)
+                            info = status.isDirectory() ? new GridGgfsFileInfo(true, status.properties()) :
+                                new GridGgfsFileInfo(status.blockSize(), status.length(), null, null, false,
+                                status.properties());
+                    }
 
-                break;
+                    break;
 
-            default:
-                assert false : "Unknown mode: " + mode;
+                default:
+                    assert false : "Unknown mode: " + mode;
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
         }
 
         return info;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsInputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsInputStreamImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsInputStreamImpl.java
index d34c4c8..86bbda5 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsInputStreamImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsInputStreamImpl.java
@@ -277,7 +277,7 @@ public class GridGgfsInputStreamImpl extends GridGgfsInputStreamAdapter {
                     try {
                         fut.get();
                     }
-                    catch (IgniteCheckedException ignore) {
+                    catch (IgniteException ignore) {
                         // No-op.
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsMetaManager.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsMetaManager.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsMetaManager.java
index 7ea2c03..77e9dda 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsMetaManager.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsMetaManager.java
@@ -74,12 +74,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
      */
     void awaitInit() {
         if (!metaCacheStartFut.isDone()) {
-            try {
-                metaCacheStartFut.get();
-            }
-            catch (IgniteCheckedException e) {
-                throw new IgniteException(e);
-            }
+            metaCacheStartFut.get();
         }
     }
 
@@ -1674,7 +1669,7 @@ public class GridGgfsMetaManager extends GridGgfsManager {
 
                                                 evts.record(new IgniteFsEvent(path, locNode, EVT_GGFS_FILE_PURGED));
                                             }
-                                            catch (IgniteCheckedException e) {
+                                            catch (IgniteException e) {
                                                 LT.warn(log, e, "Old file deletion failed in DUAL mode [path=" + path +
                                                     ", simpleCreate=" + simpleCreate + ", props=" + props +
                                                     ", overwrite=" + overwrite + ", bufferSize=" + bufSize +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsServer.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsServer.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsServer.java
index ac5b3f3..49809eb 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsServer.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsServer.java
@@ -259,7 +259,7 @@ public class GridGgfsServer {
                             try {
                                 res = fut.get();
                             }
-                            catch (IgniteCheckedException e) {
+                            catch (IgniteException e) {
                                 res = new GridGgfsControlResponse();
 
                                 ((GridGgfsControlResponse)res).error(e);
@@ -287,7 +287,7 @@ public class GridGgfsServer {
                                     try {
                                         res = fut.get();
                                     }
-                                    catch (IgniteCheckedException e) {
+                                    catch (IgniteException e) {
                                         res = new GridGgfsControlResponse();
 
                                         ((GridGgfsControlResponse)res).error(e);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/IgniteFsOutputStreamImpl.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/IgniteFsOutputStreamImpl.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/IgniteFsOutputStreamImpl.java
index 04ab237..b5fa7fc 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/IgniteFsOutputStreamImpl.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/ggfs/IgniteFsOutputStreamImpl.java
@@ -335,7 +335,7 @@ class IgniteFsOutputStreamImpl extends IgniteFsOutputStreamAdapter {
 
                     writeCompletionFut.get();
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     err = new IOException("Failed to close stream [path=" + path + ", fileInfo=" + fileInfo + ']', e);
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
----------------------------------------------------------------------
diff --git a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
index 3303b41..b7999b5 100644
--- a/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
+++ b/modules/core/src/main/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheQueryCommandHandler.java
@@ -124,7 +124,7 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
 
                 return comp.future();
             }
-            catch (IgniteCheckedException e) {
+            catch (IgniteException e) {
                 // Should not be thrown since uses asynchronous execution.
                 return new GridFinishedFutureEx<>(e);
             }
@@ -151,13 +151,13 @@ public class GridCacheQueryCommandHandler extends GridRestCommandHandlerAdapter
 
                         return new GridRestResponse();
                     }
-                    catch (IgniteCheckedException e) {
+                    catch (IgniteException e) {
                         throw new GridClosureException(e);
                     }
                 }
             });
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             // Should not be thrown since uses asynchronous execution.
             return new GridFinishedFutureEx<>(e);
         }


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

Posted by sb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiNodeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiNodeAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiNodeAbstractTest.java
index 863c2c7..8146521 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiNodeAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/GridCacheTxMultiNodeAbstractTest.java
@@ -145,10 +145,10 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
      * @param ignite Grid.
      * @param itemKey Item key.
      * @param retry Retry count.
-     * @throws IgniteCheckedException If failed.
+     * @throws IgniteException If failed.
      */
     @SuppressWarnings("unchecked")
-    private void onItemNear(boolean putCntr, Ignite ignite, String itemKey, int retry) throws IgniteCheckedException {
+    private void onItemNear(boolean putCntr, Ignite ignite, String itemKey, int retry) throws IgniteException {
         GridCache<String, Integer> cache = ignite.cache(null);
 
         UUID locId = ignite.cluster().localNode().id();
@@ -157,37 +157,42 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
 
         boolean isCntrPrimary = cntrPrimaryId.equals(locId);
 
-        try (GridCacheTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ)) {
-            if (DEBUG)
-                info("Before near get [retry=" + retry + ", xid=" + tx.xid() + ", node=" + ignite.name() +
-                    ", isCntrPrimary=" + isCntrPrimary + ", nearId=" + locId +
-                    ", nearEntry=" + nearEntry(locId, CNTR_KEY) +
-                    (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, CNTR_KEY) : "") + ']');
+        try {
+            try (GridCacheTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                if (DEBUG)
+                    info("Before near get [retry=" + retry + ", xid=" + tx.xid() + ", node=" + ignite.name() +
+                        ", isCntrPrimary=" + isCntrPrimary + ", nearId=" + locId +
+                        ", nearEntry=" + nearEntry(locId, CNTR_KEY) +
+                        (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, CNTR_KEY) : "") + ']');
 
-            Integer cntr = cache.get(CNTR_KEY);
+                Integer cntr = cache.get(CNTR_KEY);
 
-            int newVal = cntr + 1;
+                int newVal = cntr + 1;
 
-            if (putCntr) {
-                if (DEBUG)
-                    info("Before near put counter [retry=" + retry + ", isCntrPrimary=" + isCntrPrimary +
-                        ", cur=" + cntr + ", new=" + newVal + ", nearEntry=" + nearEntry(locId, CNTR_KEY) +
-                        (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, CNTR_KEY) : "") + ']');
+                if (putCntr) {
+                    if (DEBUG)
+                        info("Before near put counter [retry=" + retry + ", isCntrPrimary=" + isCntrPrimary +
+                            ", cur=" + cntr + ", new=" + newVal + ", nearEntry=" + nearEntry(locId, CNTR_KEY) +
+                            (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, CNTR_KEY) : "") + ']');
 
-                cache.putx(CNTR_KEY, newVal);
-            }
+                    cache.putx(CNTR_KEY, newVal);
+                }
 
-            if (DEBUG)
-                info("Before near put item [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr + ", new=" + newVal +
-                    ", nearEntry=" + nearEntry(locId, itemKey) + ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
+                if (DEBUG)
+                    info("Before near put item [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr + ", new=" + newVal +
+                        ", nearEntry=" + nearEntry(locId, itemKey) + ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
 
-            cache.putx(itemKey, newVal);
+                cache.putx(itemKey, newVal);
 
-            if (DEBUG)
-                info("After near put item [retry=" + retry + ", key=" + itemKey + ", old=" + cntr + ", new=" + newVal +
-                    ", nearEntry=" + nearEntry(locId, itemKey) + ", dhtEntry" + dhtEntry(itemPrimaryId, itemKey) + ']');
+                if (DEBUG)
+                    info("After near put item [retry=" + retry + ", key=" + itemKey + ", old=" + cntr + ", new=" + newVal +
+                        ", nearEntry=" + nearEntry(locId, itemKey) + ", dhtEntry" + dhtEntry(itemPrimaryId, itemKey) + ']');
 
-            tx.commit();
+                tx.commit();
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
         }
     }
 
@@ -200,7 +205,7 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
      * @throws IgniteCheckedException If failed.
      */
     @SuppressWarnings("unchecked")
-    private void onItemPrimary(boolean putCntr, Ignite ignite, String itemKey, int retry) throws IgniteCheckedException {
+    private void onItemPrimary(boolean putCntr, Ignite ignite, String itemKey, int retry) throws IgniteException {
         GridCache<String, Integer> cache = ignite.cache(null);
 
         UUID locId = ignite.cluster().localNode().id();
@@ -209,39 +214,44 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
 
         boolean isCntrPrimary = cntrPrimaryId.equals(locId);
 
-        try (GridCacheTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ)) {
-            if (DEBUG)
-                info("Before item primary get [retry=" + retry + ", xid=" + tx.xid() + ", node=" + ignite.name() +
-                    ", isCntrPrimary=" + isCntrPrimary + ", nearId=" + locId +
-                    ", nearEntry=" + nearEntry(locId, CNTR_KEY) +
-                    (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, CNTR_KEY) : "") + ']');
+        try {
+            try (GridCacheTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                if (DEBUG)
+                    info("Before item primary get [retry=" + retry + ", xid=" + tx.xid() + ", node=" + ignite.name() +
+                        ", isCntrPrimary=" + isCntrPrimary + ", nearId=" + locId +
+                        ", nearEntry=" + nearEntry(locId, CNTR_KEY) +
+                        (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, CNTR_KEY) : "") + ']');
 
-            Integer cntr = cache.get(CNTR_KEY);
+                Integer cntr = cache.get(CNTR_KEY);
 
-            int newVal = cntr + 1;
+                int newVal = cntr + 1;
 
-            if (putCntr) {
-                if (DEBUG)
-                    info("Before item primary put counter [retry=" + retry + ", isCntrPrimary=" + isCntrPrimary +
-                        ", cur=" + cntr + ", new=" + newVal + ", nearEntry=" + nearEntry(locId, CNTR_KEY) +
-                        (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, CNTR_KEY) : "") + ']');
+                if (putCntr) {
+                    if (DEBUG)
+                        info("Before item primary put counter [retry=" + retry + ", isCntrPrimary=" + isCntrPrimary +
+                            ", cur=" + cntr + ", new=" + newVal + ", nearEntry=" + nearEntry(locId, CNTR_KEY) +
+                            (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, CNTR_KEY) : "") + ']');
 
-                cache.putx(CNTR_KEY, newVal);
-            }
+                    cache.putx(CNTR_KEY, newVal);
+                }
 
-            if (DEBUG)
-                info("Before item primary put item [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr +
-                    ", new=" + newVal + ", nearEntry=" + nearEntry(locId, itemKey) +
-                    ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
+                if (DEBUG)
+                    info("Before item primary put item [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr +
+                        ", new=" + newVal + ", nearEntry=" + nearEntry(locId, itemKey) +
+                        ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
 
-            cache.putx(itemKey, cntr);
+                cache.putx(itemKey, cntr);
 
-            if (DEBUG)
-                info("After item primary put item [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr +
-                    ", new=" + newVal + ", nearEntry=" + nearEntry(locId, itemKey) +
-                    ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
+                if (DEBUG)
+                    info("After item primary put item [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr +
+                        ", new=" + newVal + ", nearEntry=" + nearEntry(locId, itemKey) +
+                        ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
 
-            tx.commit();
+                tx.commit();
+            }
+        }
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
         }
     }
 
@@ -250,10 +260,10 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
      * @param putCntr Put counter to cache.
      * @param ignite Grid.
      * @param retry Retry count.
-     * @throws IgniteCheckedException If failed.
+     * @throws IgniteException If failed.
      */
     @SuppressWarnings("unchecked")
-    private void onRemoveItemQueried(boolean putCntr, Ignite ignite, int retry) throws IgniteCheckedException {
+    private void onRemoveItemQueried(boolean putCntr, Ignite ignite, int retry) throws IgniteException {
         GridCache<String, Integer> cache = ignite.cache(null);
 
         UUID locId = ignite.cluster().localNode().id();
@@ -261,77 +271,82 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
 
         boolean isCntrPrimary = cntrPrimaryId.equals(locId);
 
-        try (GridCacheTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ)) {
-            if (DEBUG)
-                ignite.log().info("Before item lock [retry=" + retry + ", xid=" + tx.xid() + ", node=" + ignite.name() +
-                    ", isCntrPrimary=" + isCntrPrimary + ", nearId=" + locId +
-                    ", nearEntry=" + nearEntry(locId, RMVD_CNTR_KEY) +
-                    (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, RMVD_CNTR_KEY) : "") + ']');
+        try {
+            try (GridCacheTx tx = cache.txStart(PESSIMISTIC, REPEATABLE_READ)) {
+                if (DEBUG)
+                    ignite.log().info("Before item lock [retry=" + retry + ", xid=" + tx.xid() + ", node=" + ignite.name() +
+                        ", isCntrPrimary=" + isCntrPrimary + ", nearId=" + locId +
+                        ", nearEntry=" + nearEntry(locId, RMVD_CNTR_KEY) +
+                        (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, RMVD_CNTR_KEY) : "") + ']');
 
-            Integer cntr = cache.get(RMVD_CNTR_KEY);
+                Integer cntr = cache.get(RMVD_CNTR_KEY);
 
-            assert cntr != null : "Received null counter [retry=" + retry + ", isCntrPrimary=" + isCntrPrimary +
-                ", nearEntry=" + nearEntry(locId, RMVD_CNTR_KEY) +
-                (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, RMVD_CNTR_KEY) : "") + ']';
+                assert cntr != null : "Received null counter [retry=" + retry + ", isCntrPrimary=" + isCntrPrimary +
+                    ", nearEntry=" + nearEntry(locId, RMVD_CNTR_KEY) +
+                    (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, RMVD_CNTR_KEY) : "") + ']';
 
-            int newVal = cntr - 1;
+                int newVal = cntr - 1;
 
-            if (putCntr) {
-                if (DEBUG)
-                    ignite.log().info("Before item put counter [retry=" + retry + ", isCntrPrimary=" + isCntrPrimary +
-                        ", cur=" + cntr + ", new=" + newVal + ", nearEntry=" + nearEntry(locId, RMVD_CNTR_KEY) +
-                        (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, RMVD_CNTR_KEY) : "") + ']');
+                if (putCntr) {
+                    if (DEBUG)
+                        ignite.log().info("Before item put counter [retry=" + retry + ", isCntrPrimary=" + isCntrPrimary +
+                            ", cur=" + cntr + ", new=" + newVal + ", nearEntry=" + nearEntry(locId, RMVD_CNTR_KEY) +
+                            (isCntrPrimary ? ", dhtEntry=" + dhtEntry(locId, RMVD_CNTR_KEY) : "") + ']');
 
-                cache.putx(RMVD_CNTR_KEY, newVal);
-            }
+                    cache.putx(RMVD_CNTR_KEY, newVal);
+                }
 
-            while (true) {
-                GridCacheQuery<Map.Entry<String, Integer>> qry =
-                    cache.queries().createSqlQuery(Integer.class, "_key != 'RMVD_CNTR_KEY' and _val >= 0");
+                while (true) {
+                    GridCacheQuery<Map.Entry<String, Integer>> qry =
+                        cache.queries().createSqlQuery(Integer.class, "_key != 'RMVD_CNTR_KEY' and _val >= 0");
 
-                if (DEBUG)
-                    ignite.log().info("Before executing query [retry=" + retry + ", locId=" + locId +
-                        ", txId=" + tx.xid() + ']');
+                    if (DEBUG)
+                        ignite.log().info("Before executing query [retry=" + retry + ", locId=" + locId +
+                            ", txId=" + tx.xid() + ']');
 
-                Map.Entry<String, Integer> entry = qry.execute().next();
+                    Map.Entry<String, Integer> entry = qry.execute().next();
 
-                if (entry == null) {
-                    ignite.log().info("*** Queue is empty.");
+                    if (entry == null) {
+                        ignite.log().info("*** Queue is empty.");
 
-                    return;
-                }
+                        return;
+                    }
 
-                String itemKey = entry.getKey();
+                    String itemKey = entry.getKey();
 
-                UUID itemPrimaryId = primaryId(ignite, itemKey);
+                    UUID itemPrimaryId = primaryId(ignite, itemKey);
 
-                // Lock the item key.
-                if (cache.get(itemKey) != null) {
-                    if (DEBUG)
-                        ignite.log().info("Before item remove [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr +
-                            ", nearEntry=" + nearEntry(locId, itemKey) +
-                            ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
+                    // Lock the item key.
+                    if (cache.get(itemKey) != null) {
+                        if (DEBUG)
+                            ignite.log().info("Before item remove [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr +
+                                ", nearEntry=" + nearEntry(locId, itemKey) +
+                                ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
 
-                    assert cache.removex(itemKey) : "Failed to remove key [locId=" + locId +
-                        ", primaryId=" + itemPrimaryId + ", key=" + itemKey + ']';
+                        assert cache.removex(itemKey) : "Failed to remove key [locId=" + locId +
+                            ", primaryId=" + itemPrimaryId + ", key=" + itemKey + ']';
 
-                    if (DEBUG)
-                        info("After item remove item [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr +
-                            ", new=" + newVal + ", nearEntry=" + nearEntry(locId, itemKey) +
-                            ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
+                        if (DEBUG)
+                            info("After item remove item [retry=" + retry + ", key=" + itemKey + ", cur=" + cntr +
+                                ", new=" + newVal + ", nearEntry=" + nearEntry(locId, itemKey) +
+                                ", dhtEntry=" + dhtEntry(itemPrimaryId, itemKey) + ']');
 
-                    break;
+                        break;
+                    }
+                    else
+                        cache.removex(itemKey);
                 }
-                else
-                    cache.removex(itemKey);
+
+                tx.commit();
             }
+            catch (Error e) {
+                ignite.log().error("Error in test.", e);
 
-            tx.commit();
+                throw e;
+            }
         }
-        catch (Error e) {
-            ignite.log().error("Error in test.", e);
-
-            throw e;
+        catch (IgniteCheckedException e) {
+            throw U.wrap(e);
         }
     }
 
@@ -410,7 +425,7 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
      * @param ignite Grid.
      * @throws IgniteCheckedException If failed.
      */
-    private void retries(Ignite ignite, boolean putCntr) throws IgniteCheckedException {
+    private void retries(Ignite ignite, boolean putCntr) throws IgniteException {
         UUID nodeId = ignite.cluster().localNode().id();
 
         for (int i = 0; i < RETRIES; i++) {
@@ -436,9 +451,9 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
      *
      * @param putCntr Put counter to cache.
      * @param ignite Grid.
-     * @throws IgniteCheckedException If failed.
+     * @throws IgniteException If failed.
      */
-    private void removeRetriesQueried(Ignite ignite, boolean putCntr) throws IgniteCheckedException {
+    private void removeRetriesQueried(Ignite ignite, boolean putCntr) throws IgniteException {
         for (int i = 0; i < RETRIES; i++) {
             if (DEBUG)
                 ignite.log().info("***");
@@ -542,12 +557,7 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
 
                 threads.add(new Thread("thread-#" + i) {
                     @Override public void run() {
-                        try {
-                            retries(grid(gridId), false);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw new IgniteException(e);
-                        }
+                        retries(grid(gridId), false);
                     }
                 });
             }
@@ -585,12 +595,7 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
 
                 threads.add(new Thread() {
                     @Override public void run() {
-                        try {
-                            retries(grid(gridId), true);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw new IgniteException(e);
-                        }
+                        retries(grid(gridId), true);
                     }
                 });
             }
@@ -770,12 +775,7 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
 
                 threads.add(new Thread() {
                     @Override public void run() {
-                        try {
-                            removeRetriesQueried(grid(gridId), true);
-                        }
-                        catch (IgniteCheckedException e) {
-                            throw new IgniteException(e);
-                        }
+                        removeRetriesQueried(grid(gridId), true);
                     }
                 });
             }
@@ -905,7 +905,12 @@ public abstract class GridCacheTxMultiNodeAbstractTest extends GridCommonAbstrac
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + "]");
 
-            removeRetriesSimple(ignite, true);
+            try {
+                removeRetriesSimple(ignite, true);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             return 0;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java
index 9badb83..e2228fa 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheAtomicReferenceMultiNodeAbstractTest.java
@@ -12,12 +12,12 @@ package org.gridgain.grid.kernal.processors.cache.datastructures;
 import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.cache.datastructures.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.gridgain.grid.cache.datastructures.*;
 import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.junits.common.*;
 
 import java.util.*;
@@ -90,11 +90,16 @@ public abstract class GridCacheAtomicReferenceMultiNodeAbstractTest extends Grid
         // Execute task on all grid nodes.
         ignite.compute().call(new IgniteCallable<Object>() {
             @Override public String call() throws IgniteException {
-                GridCacheAtomicReference<String> ref = ignite.cache(null).dataStructures().atomicReference(refName, val, true);
+                try {
+                    GridCacheAtomicReference<String> ref = ignite.cache(null).dataStructures().atomicReference(refName, val, true);
 
-                assertEquals(val, ref.get());
+                    assertEquals(val, ref.get());
 
-                return ref.get();
+                    return ref.get();
+                }
+                catch (IgniteCheckedException e) {
+                    throw U.wrap(e);
+                }
             }
         });
 
@@ -103,11 +108,16 @@ public abstract class GridCacheAtomicReferenceMultiNodeAbstractTest extends Grid
         // Execute task on all grid nodes.
         ignite.compute().call(new IgniteCallable<String>() {
             @Override public String call() throws IgniteException {
-                GridCacheAtomicReference<String> ref = ignite.cache(null).dataStructures().atomicReference(refName, val, true);
+                try {
+                    GridCacheAtomicReference<String> ref = ignite.cache(null).dataStructures().atomicReference(refName, val, true);
 
-                assertEquals(val, ref.get());
+                    assertEquals(val, ref.get());
 
-                return ref.get();
+                    return ref.get();
+                }
+                catch (IgniteCheckedException e) {
+                    throw U.wrap(e);
+                }
             }
         });
 
@@ -116,11 +126,16 @@ public abstract class GridCacheAtomicReferenceMultiNodeAbstractTest extends Grid
         // Execute task on all grid nodes.
         ignite.compute().call(new IgniteCallable<String>() {
             @Override public String call() throws IgniteException {
-                GridCacheAtomicReference<String> ref = ignite.cache(null).dataStructures().atomicReference(refName, val, true);
+                try {
+                    GridCacheAtomicReference<String> ref = ignite.cache(null).dataStructures().atomicReference(refName, val, true);
 
-                assertEquals(newVal, ref.get());
+                    assertEquals(newVal, ref.get());
 
-                return ref.get();
+                    return ref.get();
+                }
+                catch (IgniteCheckedException e) {
+                    throw U.wrap(e);
+                }
             }
         });
     }
@@ -151,13 +166,18 @@ public abstract class GridCacheAtomicReferenceMultiNodeAbstractTest extends Grid
         // Execute task on all grid nodes.
         ignite.compute().call(new IgniteCallable<String>() {
             @Override public String call() throws IgniteException {
-                GridCacheAtomicStamped<String, String> stamped = ignite.cache(null).dataStructures()
-                    .atomicStamped(stampedName, val, stamp, true);
-
-                assertEquals(val, stamped.value());
-                assertEquals(stamp, stamped.stamp());
-
-                return stamped.value();
+                try {
+                    GridCacheAtomicStamped<String, String> stamped = ignite.cache(null).dataStructures()
+                        .atomicStamped(stampedName, val, stamp, true);
+
+                    assertEquals(val, stamped.value());
+                    assertEquals(stamp, stamped.stamp());
+
+                    return stamped.value();
+                }
+                catch (IgniteCheckedException e) {
+                    throw U.wrap(e);
+                }
             }
         });
 
@@ -166,13 +186,18 @@ public abstract class GridCacheAtomicReferenceMultiNodeAbstractTest extends Grid
         // Execute task on all grid nodes.
         ignite.compute().call(new IgniteCallable<String>() {
             @Override public String call() throws IgniteException {
-                GridCacheAtomicStamped<String, String> stamped = ignite.cache(null).dataStructures()
-                    .atomicStamped(stampedName, val, stamp, true);
-
-                assertEquals(val, stamped.value());
-                assertEquals(stamp, stamped.stamp());
-
-                return stamped.value();
+                try {
+                    GridCacheAtomicStamped<String, String> stamped = ignite.cache(null).dataStructures()
+                        .atomicStamped(stampedName, val, stamp, true);
+
+                    assertEquals(val, stamped.value());
+                    assertEquals(stamp, stamped.stamp());
+
+                    return stamped.value();
+                }
+                catch (IgniteCheckedException e) {
+                    throw U.wrap(e);
+                }
             }
         });
 
@@ -181,13 +206,18 @@ public abstract class GridCacheAtomicReferenceMultiNodeAbstractTest extends Grid
         // Execute task on all grid nodes.
         ignite.compute().call(new IgniteCallable<String>() {
             @Override public String call() throws IgniteException {
-                GridCacheAtomicStamped<String, String> stamped = ignite.cache(null).dataStructures()
-                    .atomicStamped(stampedName, val, stamp, true);
-
-                assertEquals(newVal, stamped.value());
-                assertEquals(newStamp, stamped.stamp());
-
-                return stamped.value();
+                try {
+                    GridCacheAtomicStamped<String, String> stamped = ignite.cache(null).dataStructures()
+                        .atomicStamped(stampedName, val, stamp, true);
+
+                    assertEquals(newVal, stamped.value());
+                    assertEquals(newStamp, stamped.stamp());
+
+                    return stamped.value();
+                }
+                catch (IgniteCheckedException e) {
+                    throw U.wrap(e);
+                }
             }
         });
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
index 7c335bf..fe8d511 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueMultiNodeAbstractSelfTest.java
@@ -647,8 +647,14 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + "]");
 
-            GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAPACITY,
-                false, true);
+            GridCacheQueue<Integer> queue = null;
+            try {
+                queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAPACITY,
+                    false, true);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             assertNotNull(queue);
 
@@ -694,8 +700,14 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + "]");
 
-            GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAPACITY,
-                false, true);
+            GridCacheQueue<Integer> queue = null;
+            try {
+                queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAPACITY,
+                    false, true);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             assertNotNull(queue);
 
@@ -750,8 +762,14 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + "]");
 
-            GridCacheQueue<String> queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAPACITY,
-                false, true);
+            GridCacheQueue<String> queue = null;
+            try {
+                queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAPACITY,
+                    false, true);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             assertNotNull(queue);
 
@@ -802,8 +820,14 @@ public abstract class GridCacheQueueMultiNodeAbstractSelfTest extends GridCommon
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + ']');
 
-            GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAPACITY,
-                false, true);
+            GridCacheQueue<Integer> queue = null;
+            try {
+                queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAPACITY,
+                    false, true);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             assertNotNull(queue);
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java
index 7fe2792..b00a65b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheQueueRotativeMultiNodeAbstractTest.java
@@ -159,7 +159,7 @@ public abstract class GridCacheQueueRotativeMultiNodeAbstractTest extends GridCo
                 try {
                     assert grid(1).compute().call(new TakeJob(queueName));
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     error(e.getMessage(), e);
                 }
             }
@@ -214,8 +214,14 @@ public abstract class GridCacheQueueRotativeMultiNodeAbstractTest extends GridCo
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + "]");
 
-            GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures()
-                .queue(queueName, QUEUE_CAPACITY, true, true);
+            GridCacheQueue<Integer> queue = null;
+            try {
+                queue = ignite.cache(null).dataStructures()
+                    .queue(queueName, QUEUE_CAPACITY, true, true);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             assertNotNull(queue);
 
@@ -266,8 +272,14 @@ public abstract class GridCacheQueueRotativeMultiNodeAbstractTest extends GridCo
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + "]");
 
-            GridCacheQueue<String> queue = ignite.cache(null).dataStructures()
-                .queue(queueName, QUEUE_CAPACITY, true, true);
+            GridCacheQueue<String> queue = null;
+            try {
+                queue = ignite.cache(null).dataStructures()
+                    .queue(queueName, QUEUE_CAPACITY, true, true);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             assertNotNull(queue);
 
@@ -318,8 +330,14 @@ public abstract class GridCacheQueueRotativeMultiNodeAbstractTest extends GridCo
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + ']');
 
-            GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures()
-                .queue(queueName, QUEUE_CAPACITY, true, true);
+            GridCacheQueue<Integer> queue = null;
+            try {
+                queue = ignite.cache(null).dataStructures()
+                    .queue(queueName, QUEUE_CAPACITY, true, true);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             assertNotNull(queue);
 
@@ -366,8 +384,14 @@ public abstract class GridCacheQueueRotativeMultiNodeAbstractTest extends GridCo
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + ']');
 
-            GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures()
-                .queue(queueName, QUEUE_CAPACITY, true, true);
+            GridCacheQueue<Integer> queue = null;
+            try {
+                queue = ignite.cache(null).dataStructures()
+                    .queue(queueName, QUEUE_CAPACITY, true, true);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             assertNotNull(queue);
 
@@ -414,14 +438,19 @@ public abstract class GridCacheQueueRotativeMultiNodeAbstractTest extends GridCo
 
             ignite.log().info("Running job [node=" + ignite.cluster().localNode().id() + ", job=" + this + "]");
 
-            GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures()
-                .queue(queueName, QUEUE_CAPACITY, true, false);
+            try {
+                GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures()
+                    .queue(queueName, QUEUE_CAPACITY, true, false);
 
-            assert queue != null;
+                assert queue != null;
 
-            assert queue.capacity() == QUEUE_CAPACITY;
+                assert queue.capacity() == QUEUE_CAPACITY;
 
-            assert ignite.cache(null).dataStructures().removeQueue(queueName);
+                assert ignite.cache(null).dataStructures().removeQueue(queueName);
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
 
             return true;
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSequenceMultiNodeAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSequenceMultiNodeAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSequenceMultiNodeAbstractSelfTest.java
index c8e2ad7..4069adc 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSequenceMultiNodeAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/GridCacheSequenceMultiNodeAbstractSelfTest.java
@@ -13,11 +13,10 @@ import org.apache.ignite.*;
 import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.resources.*;
-import org.gridgain.grid.*;
-import org.gridgain.grid.cache.datastructures.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.gridgain.grid.cache.datastructures.*;
 import org.gridgain.grid.util.typedef.*;
 import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.junits.common.*;
@@ -201,23 +200,28 @@ public abstract class GridCacheSequenceMultiNodeAbstractSelfTest extends GridCom
             if (log.isInfoEnabled())
                 log.info("Running IncrementAndGetJob on node: " + ignite.cluster().localNode().id());
 
-            GridCacheAtomicSequence seq = ignite.cache(null).dataStructures().atomicSequence(seqName, 0, true);
+            try {
+                GridCacheAtomicSequence seq = ignite.cache(null).dataStructures().atomicSequence(seqName, 0, true);
 
-            assert seq != null;
+                assert seq != null;
 
-            // Result set.
-            Set<Long> resSet = new HashSet<>();
+                // Result set.
+                Set<Long> resSet = new HashSet<>();
 
-            // Get sequence value and try to put it result set.
-            for (int i = 0; i < retries; i++) {
-                long val = seq.incrementAndGet();
+                // Get sequence value and try to put it result set.
+                for (int i = 0; i < retries; i++) {
+                    long val = seq.incrementAndGet();
 
-                assert !resSet.contains(val) : "Element already in set : " + val;
+                    assert !resSet.contains(val) : "Element already in set : " + val;
 
-                resSet.add(val);
-            }
+                    resSet.add(val);
+                }
 
-            return resSet;
+                return resSet;
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
         }
     }
 
@@ -255,23 +259,28 @@ public abstract class GridCacheSequenceMultiNodeAbstractSelfTest extends GridCom
             if (log.isInfoEnabled())
                 log.info("Running GetAndIncrementJob on node: " + ignite.cluster().localNode().id());
 
-            GridCacheAtomicSequence seq = ignite.cache(null).dataStructures().atomicSequence(seqName, 0, true);
+            try {
+                GridCacheAtomicSequence seq = ignite.cache(null).dataStructures().atomicSequence(seqName, 0, true);
 
-            assert seq != null;
+                assert seq != null;
 
-            // Result set.
-            Set<Long> resSet = new HashSet<>();
+                // Result set.
+                Set<Long> resSet = new HashSet<>();
 
-            // Get sequence value and try to put it result set.
-            for (int i = 0; i < retries; i++) {
-                long val = seq.getAndIncrement();
+                // Get sequence value and try to put it result set.
+                for (int i = 0; i < retries; i++) {
+                    long val = seq.getAndIncrement();
 
-                assert !resSet.contains(val) : "Element already in set : " + val;
+                    assert !resSet.contains(val) : "Element already in set : " + val;
 
-                resSet.add(val);
-            }
+                    resSet.add(val);
+                }
 
-            return resSet;
+                return resSet;
+            }
+            catch (IgniteCheckedException e) {
+                throw U.wrap(e);
+            }
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
index a399a81..65eb10b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/datastructures/partitioned/GridCachePartitionedQueueEntryMoveSelfTest.java
@@ -97,8 +97,14 @@ public class GridCachePartitionedQueueEntryMoveSelfTest extends GridCommonAbstra
                 @Override public Void call() throws IgniteException {
                     Ignite ignite = grid(0);
 
-                    GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAP,
-                        true, true);
+                    GridCacheQueue<Integer> queue = null;
+                    try {
+                        queue = ignite.cache(null).dataStructures().queue(queueName, QUEUE_CAP,
+                            true, true);
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw U.wrap(e);
+                    }
 
                     for (int i = 0; i < QUEUE_CAP * 2; i++) {
                         if (i == QUEUE_CAP) {
@@ -137,8 +143,14 @@ public class GridCachePartitionedQueueEntryMoveSelfTest extends GridCommonAbstra
                 @Override public Void call() throws IgniteException {
                     Ignite ignite = grid(GRID_CNT);
 
-                    GridCacheQueue<Integer> queue = ignite.cache(null).dataStructures().
-                        queue(queueName, Integer.MAX_VALUE, true, true);
+                    GridCacheQueue<Integer> queue = null;
+                    try {
+                        queue = ignite.cache(null).dataStructures().
+                            queue(queueName, Integer.MAX_VALUE, true, true);
+                    }
+                    catch (IgniteCheckedException e) {
+                        throw U.wrap(e);
+                    }
 
                     int cnt = 0;
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
index 8185447..6748c88 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/dht/GridCacheDhtPreloadUnloadSelfTest.java
@@ -19,6 +19,7 @@ import org.gridgain.grid.cache.affinity.consistenthash.*;
 import org.apache.ignite.spi.discovery.tcp.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.*;
 import org.apache.ignite.spi.discovery.tcp.ipfinder.vm.*;
+import org.gridgain.grid.util.typedef.internal.*;
 import org.gridgain.testframework.junits.common.*;
 
 import static org.apache.ignite.configuration.IgniteDeploymentMode.*;
@@ -260,10 +261,15 @@ public class GridCacheDhtPreloadUnloadSelfTest extends GridCommonAbstractTest {
                     if (evt == LifecycleEventType.AFTER_GRID_START) {
                         GridCache<Integer, String> c = ignite.cache(null);
 
-                        if (c.putxIfAbsent(-1, "true")) {
-                            populate(ignite.<Integer, String>cache(null), cnt);
+                        try {
+                            if (c.putxIfAbsent(-1, "true")) {
+                                populate(ignite.<Integer, String>cache(null), cnt);
 
-                            info(">>> POPULATED GRID <<<");
+                                info(">>> POPULATED GRID <<<");
+                            }
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw U.wrap(e);
                         }
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
index a1b3425..653e9bd 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/cache/distributed/near/GridCachePartitionedPreloadLifecycleSelfTest.java
@@ -14,11 +14,11 @@ import org.apache.ignite.configuration.*;
 import org.apache.ignite.lang.*;
 import org.apache.ignite.lifecycle.*;
 import org.apache.ignite.resources.*;
-import org.gridgain.grid.*;
 import org.gridgain.grid.cache.*;
 import org.gridgain.grid.cache.query.*;
 import org.gridgain.grid.kernal.processors.cache.distributed.*;
 import org.gridgain.grid.util.typedef.*;
+import org.gridgain.grid.util.typedef.internal.*;
 
 import java.util.*;
 
@@ -95,11 +95,16 @@ public class GridCachePartitionedPreloadLifecycleSelfTest extends GridCachePrelo
 
                         int i = 0;
 
-                        for (Object key : keys) {
-                            c1.put(key, new MyValue(value(key)));
+                        try {
+                            for (Object key : keys) {
+                                c1.put(key, new MyValue(value(key)));
 
-                            if (i++ % 2 == 0)
-                                c2.put(key, new MyValue(value(key)));
+                                if (i++ % 2 == 0)
+                                    c2.put(key, new MyValue(value(key)));
+                            }
+                        }
+                        catch (IgniteCheckedException e) {
+                            throw U.wrap(e);
                         }
 
                         assert c1.size() == keys.length : "Invalid cache1 size [size=" + c1.size() +

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessorSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessorSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessorSelfTest.java
index cc4278a..7bd6531 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessorSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/closure/GridClosureProcessorSelfTest.java
@@ -411,7 +411,7 @@ public class GridClosureProcessorSelfTest extends GridCommonAbstractTest {
 
             assert false : "Exception should have been thrown.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Caught expected exception: " + e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java
index c42022a..1597cb6 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/continuous/GridEventConsumeSelfTest.java
@@ -516,7 +516,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
             assert false : "Exception was not thrown.";
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             assertTrue(e.getMessage().startsWith(
                 "Failed to register remote continuous listener (projection is empty)."));
         }
@@ -1036,7 +1036,7 @@ public class GridEventConsumeSelfTest extends GridCommonAbstractTest {
 
                         comp.future().get(3000);
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         // Ignore all job execution related errors.
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridCacheGgfsPerBlockLruEvictionPolicySelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridCacheGgfsPerBlockLruEvictionPolicySelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridCacheGgfsPerBlockLruEvictionPolicySelfTest.java
index 2544402..e31a337 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridCacheGgfsPerBlockLruEvictionPolicySelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridCacheGgfsPerBlockLruEvictionPolicySelfTest.java
@@ -459,7 +459,7 @@ public class GridCacheGgfsPerBlockLruEvictionPolicySelfTest extends GridGgfsComm
 
                     return metrics.blocksReadTotal() == blocksRead && metrics.blocksReadRemote() == blocksReadRmt;
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     throw new RuntimeException(e);
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAbstractSelfTest.java
index 517d36a..15194ea 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsAbstractSelfTest.java
@@ -296,7 +296,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
         try {
             paths = ggfs.listFiles(SUBDIR);
         }
-        catch (IgniteCheckedException ignore) {
+        catch (IgniteException ignore) {
             // No-op.
         }
 
@@ -329,7 +329,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
         try {
             info = ggfs.info(DIR);
         }
-        catch (IgniteCheckedException ignore) {
+        catch (IgniteException ignore) {
             // No-op.
         }
 
@@ -686,7 +686,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                     return null;
                 }
-            }, IgniteCheckedException.class, "Failed to delete the path due to secondary file system exception:");
+            }, IgniteException.class, "Failed to delete the path due to secondary file system exception:");
         else {
             GridTestUtils.assertThrows(log, new Callable<Object>() {
                 @Override public Object call() throws Exception {
@@ -1083,7 +1083,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                     checkFileContent(ggfs, path, chunk);
                 }
-                catch (IOException | IgniteCheckedException e) {
+                catch (IOException | IgniteCheckedException | IgniteException e) {
                     err.compareAndSet(null, e); // Log the very first error.
                 }
             }
@@ -1124,7 +1124,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         createCtr.incrementAndGet();
                     }
-                    catch (IgniteCheckedException ignore) {
+                    catch (IgniteException ignore) {
                         try {
                             U.sleep(10);
                         }
@@ -1443,7 +1443,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         chunksCtr.incrementAndGet();
                     }
-                    catch (IgniteCheckedException ignore) {
+                    catch (IgniteException ignore) {
                         try {
                             U.sleep(10);
                         }
@@ -1518,7 +1518,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
                     try {
                         ggfs.mkdirs(SUBSUBDIR);
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
 
@@ -1533,7 +1533,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
                     try {
                         return ggfs.delete(DIR, true);
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1570,7 +1570,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1583,7 +1583,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
                     try {
                         return ggfs.delete(SUBDIR, true);
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1634,7 +1634,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1647,7 +1647,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
                     try {
                         return ggfs.delete(SUBDIR_NEW, true);
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1690,7 +1690,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1705,7 +1705,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1750,7 +1750,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1765,7 +1765,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1978,7 +1978,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         ggfs.rename(fromPath, toPath);
                     }
-                    catch (IgniteCheckedException ignore) {
+                    catch (IgniteException ignore) {
                         // No-op.
                     }
                 }
@@ -2002,7 +2002,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         ggfs.delete(path, true);
                     }
-                    catch (IgniteCheckedException ignore) {
+                    catch (IgniteException ignore) {
                         // No-op.
                     }
                 }
@@ -2026,7 +2026,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
 
                         ggfs.update(path, properties("owner", "group", null));
                     }
-                    catch (IgniteCheckedException ignore) {
+                    catch (IgniteException ignore) {
                         // No-op.
                     }
                 }
@@ -2053,7 +2053,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
                         ggfs.mkdirs(path);
 
                     }
-                    catch (IgniteCheckedException ignore) {
+                    catch (IgniteException ignore) {
                         // No-op.
                     }
                 }
@@ -2088,7 +2088,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
                             U.closeQuiet(os);
                         }
                     }
-                    catch (IOException | IgniteCheckedException ignore) {
+                    catch (IOException | IgniteException ignore) {
                         // No-op.
                     }
                 }
@@ -2240,7 +2240,7 @@ public abstract class GridGgfsAbstractSelfTest extends GridGgfsCommonAbstractTes
         try {
             ggfs.update(file, Collections.singletonMap("prop", "val"));
         }
-        catch (IgniteCheckedException ignore) {
+        catch (IgniteException ignore) {
             // No-op.
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDualAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDualAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDualAbstractSelfTest.java
index 9758af7..2dfa67d 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDualAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsDualAbstractSelfTest.java
@@ -1392,7 +1392,7 @@ public abstract class GridGgfsDualAbstractSelfTest extends GridGgfsAbstractSelfT
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1448,7 +1448,7 @@ public abstract class GridGgfsDualAbstractSelfTest extends GridGgfsAbstractSelfT
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1500,7 +1500,7 @@ public abstract class GridGgfsDualAbstractSelfTest extends GridGgfsAbstractSelfT
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1515,7 +1515,7 @@ public abstract class GridGgfsDualAbstractSelfTest extends GridGgfsAbstractSelfT
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1560,7 +1560,7 @@ public abstract class GridGgfsDualAbstractSelfTest extends GridGgfsAbstractSelfT
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }
@@ -1575,7 +1575,7 @@ public abstract class GridGgfsDualAbstractSelfTest extends GridGgfsAbstractSelfT
 
                         return true;
                     }
-                    catch (IgniteCheckedException ignored) {
+                    catch (IgniteException ignored) {
                         return false;
                     }
                 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileInfoSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileInfoSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileInfoSelfTest.java
index cae7c61..5a16439 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileInfoSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsFileInfoSelfTest.java
@@ -61,9 +61,9 @@ public class GridGgfsFileInfoSelfTest extends GridGgfsCommonAbstractTest {
      * Test node info serialization.
      *
      * @param info Node info to test serialization for.
-     * @throws IgniteCheckedException If failed.
+     * @throws IgniteException If failed.
      */
-    public void testSerialization(GridGgfsFileInfo info) throws IgniteCheckedException {
+    public void testSerialization(GridGgfsFileInfo info) throws IgniteException {
         assertEquals(info, mu(info));
     }
 
@@ -72,9 +72,9 @@ public class GridGgfsFileInfoSelfTest extends GridGgfsCommonAbstractTest {
      *
      * @param obj Object to marshal/unmarshal.
      * @return Marshalled and then unmarshalled object.
-     * @throws IgniteCheckedException In case of any marshalling exception.
+     * @throws IgniteException In case of any marshalling exception.
      */
-    private <T> T mu(T obj) throws IgniteCheckedException {
+    private <T> T mu(T obj) throws IgniteException {
         return marshaller.unmarshal(marshaller.marshal(obj), null);
     }
 }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsProcessorValidationSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsProcessorValidationSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsProcessorValidationSelfTest.java
index d32f6ac..1db76a7 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsProcessorValidationSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/ggfs/GridGgfsProcessorValidationSelfTest.java
@@ -458,7 +458,7 @@ public class GridGgfsProcessorValidationSelfTest extends GridGgfsCommonAbstractT
 
             fail("No exception has been thrown.");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             if (testLoc) {
                 if ("Failed to start processor: GridProcessorAdapter []".equals(e.getMessage()) &&
                     e.getCause().getMessage().contains(excMsgSnippet))

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
index cafff4e..e9e967a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/rest/handlers/cache/GridCacheCommandHandlerSelfTest.java
@@ -82,7 +82,7 @@ public class GridCacheCommandHandlerSelfTest extends GridCommonAbstractTest {
 
             fail("Expected exception not thrown.");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got expected exception: " + e);
         }
     }
@@ -107,7 +107,7 @@ public class GridCacheCommandHandlerSelfTest extends GridCommonAbstractTest {
 
             fail("Expected exception not thrown.");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Got expected exception: " + e);
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessorAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessorAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessorAbstractSelfTest.java
index 835bb26..8811226 100644
--- a/modules/core/src/test/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessorAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/kernal/processors/service/GridServiceProcessorAbstractSelfTest.java
@@ -185,7 +185,7 @@ public abstract class GridServiceProcessorAbstractSelfTest extends GridCommonAbs
 
             fail("Failed to receive mismatching configuration exception.");
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             info("Received mismatching configuration exception: " + e.getMessage());
         }
     }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/lang/GridFutureListenPerformanceTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/lang/GridFutureListenPerformanceTest.java b/modules/core/src/test/java/org/gridgain/grid/lang/GridFutureListenPerformanceTest.java
index 871a25a..1520111 100644
--- a/modules/core/src/test/java/org/gridgain/grid/lang/GridFutureListenPerformanceTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/lang/GridFutureListenPerformanceTest.java
@@ -72,7 +72,7 @@ public class GridFutureListenPerformanceTest {
                                         try {
                                             t.get();
                                         }
-                                        catch (IgniteCheckedException e) {
+                                        catch (IgniteException e) {
                                             e.printStackTrace();
                                         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java b/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
index 4f3e036..726e012 100644
--- a/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/loadtest/swapspace/GridFileSwapSpaceSpiMultithreadedLoadTest.java
@@ -222,7 +222,7 @@ public class GridFileSwapSpaceSpiMultithreadedLoadTest extends GridCommonAbstrac
                         }
                     }
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     e.printStackTrace();
                 }
             }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PLocalDeploymentSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PLocalDeploymentSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PLocalDeploymentSelfTest.java
index 3f3339b..cc41834 100644
--- a/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PLocalDeploymentSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PLocalDeploymentSelfTest.java
@@ -232,7 +232,7 @@ public class GridP2PLocalDeploymentSelfTest extends GridCommonAbstractTest {
                     return Collections.singletonMap(new TestJob(arg), node);
             }
 
-            throw new IgniteCheckedException("Failed to find target node: " + arg);
+            throw new IgniteException("Failed to find target node: " + arg);
         }
 
         /** {@inheritDoc} */

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PMissedResourceCacheSizeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PMissedResourceCacheSizeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PMissedResourceCacheSizeSelfTest.java
index 8fde401..27eb426 100644
--- a/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PMissedResourceCacheSizeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PMissedResourceCacheSizeSelfTest.java
@@ -87,14 +87,14 @@ public class GridP2PMissedResourceCacheSizeSelfTest extends GridCommonAbstractTe
 
             assert false; // Exception must be thrown.
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             // Throwing exception is a correct behaviour.
             info("Received correct exception: " + e);
         }
     }
 
     /**
-     * Querying events here throws {@link IgniteCheckedException}.
+     * Querying events here throws {@link IgniteException}.
      * This is correct behavior.
      *
      * @param g Grid.
@@ -106,7 +106,7 @@ public class GridP2PMissedResourceCacheSizeSelfTest extends GridCommonAbstractTe
 
             assert false; // Exception must be thrown.
         }
-        catch (IgniteCheckedException e) {
+        catch (IgniteException e) {
             // Throwing exception is a correct behaviour.
             info("Received correct exception: " + e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PTimeoutSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PTimeoutSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PTimeoutSelfTest.java
index e1aa70d..388cd7a 100644
--- a/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PTimeoutSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/p2p/GridP2PTimeoutSelfTest.java
@@ -73,7 +73,7 @@ public class GridP2PTimeoutSelfTest extends GridCommonAbstractTest {
 
                 assert false; // Timeout exception must be thrown.
             }
-            catch (IgniteCheckedException ignored) {
+            catch (IgniteException ignored) {
                 // Throwing exception is a correct behaviour.
             }
         }
@@ -115,7 +115,7 @@ public class GridP2PTimeoutSelfTest extends GridCommonAbstractTest {
 
                 assert false; // Timeout exception must be thrown.
             }
-            catch (IgniteCheckedException ignored) {
+            catch (IgniteException ignored) {
                 // Throwing exception is a correct behaviour.
             }
         }

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCheckpointAbstractSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCheckpointAbstractSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCheckpointAbstractSelfTest.java
index 0917224..b695090 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCheckpointAbstractSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCheckpointAbstractSelfTest.java
@@ -194,7 +194,7 @@ public abstract class GridSessionCheckpointAbstractSelfTest extends GridCommonAb
                 Thread.sleep(200);
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Got interrupted during reducing.", e);
+                throw new IgniteException("Got interrupted during reducing.", e);
             }
 
             try {
@@ -214,7 +214,7 @@ public abstract class GridSessionCheckpointAbstractSelfTest extends GridCommonAb
                 }
             }
             catch (Exception e) {
-                throw new IgniteCheckedException("Running state check failure.", e);
+                throw new IgniteException("Running state check failure.", e);
             }
 
             return res;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCollisionSpiSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCollisionSpiSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCollisionSpiSelfTest.java
index b733979..81c0a6c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCollisionSpiSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionCollisionSpiSelfTest.java
@@ -125,7 +125,7 @@ public class GridSessionCollisionSpiSelfTest extends GridCommonAbstractTest {
                     if (log.isInfoEnabled())
                         log.info("Set session attribute for job: " + jobId);
                 }
-                catch (IgniteCheckedException e) {
+                catch (IgniteException e) {
                     log.error("Failed to set session attribute: " + job, e);
                 }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java
index db5d0de..010e8c8 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobFailoverSelfTest.java
@@ -97,7 +97,7 @@ public class GridSessionJobFailoverSelfTest extends GridCommonAbstractTest {
                         fail = ses.waitForAttribute("fail", 0);
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Got interrupted while waiting for attribute to be set.", e);
+                        throw new IgniteException("Got interrupted while waiting for attribute to be set.", e);
                     }
 
                     if (fail) {
@@ -108,7 +108,7 @@ public class GridSessionJobFailoverSelfTest extends GridCommonAbstractTest {
                                 ses.setAttribute("test.job.attr." + i, ii);
                         }
 
-                        throw new IgniteCheckedException("Job exception.");
+                        throw new IgniteException("Job exception.");
                     }
 
                     try {
@@ -125,7 +125,7 @@ public class GridSessionJobFailoverSelfTest extends GridCommonAbstractTest {
                         }
                     }
                     catch (InterruptedException e) {
-                        throw new IgniteCheckedException("Got interrupted while waiting for attribute to be set.", e);
+                        throw new IgniteException("Got interrupted while waiting for attribute to be set.", e);
                     }
 
                     // This job does not return any result.

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobWaitTaskAttributeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobWaitTaskAttributeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobWaitTaskAttributeSelfTest.java
index 66ae464..c18e29b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobWaitTaskAttributeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionJobWaitTaskAttributeSelfTest.java
@@ -159,7 +159,7 @@ public class GridSessionJobWaitTaskAttributeSelfTest extends GridCommonAbstractT
                                 fail("Invalid test session value: " + val);
                             }
                             catch (InterruptedException e) {
-                                throw new IgniteCheckedException("Failed to get attribute due to interruption.", e);
+                                throw new IgniteException("Failed to get attribute due to interruption.", e);
                             }
                         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeSelfTest.java
index ef9cc5e..43a5d6b 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeSelfTest.java
@@ -206,7 +206,7 @@ public class GridSessionSetFutureAttributeSelfTest extends GridCommonAbstractTes
                                 return 1;
                         }
                         catch (InterruptedException e) {
-                            throw new IgniteCheckedException("Failed to get attribute due to interruption.", e);
+                            throw new IgniteException("Failed to get attribute due to interruption.", e);
                         }
 
                         return 0;

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeWaitListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeWaitListenerSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeWaitListenerSelfTest.java
index b5d50d5..98df21c 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeWaitListenerSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetFutureAttributeWaitListenerSelfTest.java
@@ -160,7 +160,7 @@ public class GridSessionSetFutureAttributeWaitListenerSelfTest extends GridCommo
                             return 1;
                         }
                         catch (InterruptedException e) {
-                            throw new IgniteCheckedException("Failed to wait for listener due to interruption.", e);
+                            throw new IgniteException("Failed to wait for listener due to interruption.", e);
                         }
                     }
                 });

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttribute2SelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttribute2SelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttribute2SelfTest.java
index dee2348..3269482 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttribute2SelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttribute2SelfTest.java
@@ -88,7 +88,7 @@ public class GridSessionSetJobAttribute2SelfTest extends GridCommonAbstractTest
                 Thread.sleep(100);
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Got interrupted while while sleeping.", e);
+                throw new IgniteException("Got interrupted while while sleeping.", e);
             }
 
             Serializable ser = taskSes.getAttribute(TEST_ATTR_KEY);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeOrderSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeOrderSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeOrderSelfTest.java
index e7a787e..7d00c59 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeOrderSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeOrderSelfTest.java
@@ -101,7 +101,7 @@ public class GridSessionSetJobAttributeOrderSelfTest extends GridCommonAbstractT
                 }
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Got interrupted while waiting for attribute to be set.", e);
+                throw new IgniteException("Got interrupted while waiting for attribute to be set.", e);
             }
 
             return taskSes.getAttribute(TEST_ATTR_KEY);
@@ -128,7 +128,7 @@ public class GridSessionSetJobAttributeOrderSelfTest extends GridCommonAbstractT
                 assert attr : "Failed to wait for attribute value.";
             }
             catch (InterruptedException e) {
-                throw new IgniteCheckedException("Got interrupted while waiting for attribute to be set.", e);
+                throw new IgniteException("Got interrupted while waiting for attribute to be set.", e);
             }
 
             Integer res = taskSes.getAttribute(TEST_ATTR_KEY);

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeSelfTest.java
index fe98080..318e716 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeSelfTest.java
@@ -160,7 +160,7 @@ public class GridSessionSetJobAttributeSelfTest extends GridCommonAbstractTest {
                                     return 1;
                             }
                             catch (InterruptedException e) {
-                                throw new IgniteCheckedException("Failed to get attribute due to interruption.", e);
+                                throw new IgniteException("Failed to get attribute due to interruption.", e);
                             }
                         }
 

http://git-wip-us.apache.org/repos/asf/incubator-ignite/blob/4307bca1/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeWaitListenerSelfTest.java
----------------------------------------------------------------------
diff --git a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeWaitListenerSelfTest.java b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeWaitListenerSelfTest.java
index 8dddb15..bf0c1f1 100644
--- a/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeWaitListenerSelfTest.java
+++ b/modules/core/src/test/java/org/gridgain/grid/session/GridSessionSetJobAttributeWaitListenerSelfTest.java
@@ -158,7 +158,7 @@ public class GridSessionSetJobAttributeWaitListenerSelfTest extends GridCommonAb
                             return lsnr.getAttributes().size() == 0 ? 0 : 1;
                         }
                         catch (InterruptedException e) {
-                            throw new IgniteCheckedException("Failed to wait for listener due to interruption.", e);
+                            throw new IgniteException("Failed to wait for listener due to interruption.", e);
                         }
                     }
                 });